From dd39efdeb769fbb6b1bd58e008a4262b37eb6308 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?=E8=A2=81=E7=A6=8F=E5=85=83?= Date: Fri, 13 Jan 2023 16:40:16 +0800 Subject: [PATCH 01/21] fix-#4057 info: 1. "$@" is a array, we want use string to compare. so update "$@" => "$*" 2. `tty` mean execute the command, we can use $(tty) replace it 3. param $# is a number, compare number should use -gt/-lt,not >/< --- bin/docker-image-tool.sh | 3 +-- bin/kyuubi | 4 ++-- bin/kyuubi-logo | 8 ++++---- bin/stop-application.sh | 4 ++-- 4 files changed, 9 insertions(+), 10 deletions(-) diff --git a/bin/docker-image-tool.sh b/bin/docker-image-tool.sh index e9e4338b5dc..509da0afb24 100755 --- a/bin/docker-image-tool.sh +++ b/bin/docker-image-tool.sh @@ -227,8 +227,7 @@ Examples: EOF } -# shellcheck disable=SC2199 -if [[ "$@" = *--help ]] || [[ "$@" = *-h ]]; then +if [[ "$*" = *--help ]] || [[ "$*" = *-h ]]; then usage exit 0 fi diff --git a/bin/kyuubi b/bin/kyuubi index 414bdeb86ce..b4e0ca37c83 100755 --- a/bin/kyuubi +++ b/bin/kyuubi @@ -30,8 +30,8 @@ function usage() { echo " -h | --help - Show this help message" } -# shellcheck disable=SC2199 -if [[ "$@" = *--help ]] || [[ "$@" = *-h ]]; then + +if [[ "$*" = *--help ]] || [[ "$*" = *-h ]]; then usage exit 0 fi diff --git a/bin/kyuubi-logo b/bin/kyuubi-logo index 15a45a4bbc0..b59a12d8853 100755 --- a/bin/kyuubi-logo +++ b/bin/kyuubi-logo @@ -18,15 +18,15 @@ # Bugzilla 37848: When no TTY is available, don't output to console have_tty=0 -# shellcheck disable=SC2006 -if [[ "`tty`" != "not a tty" ]]; then + +if [[ "$(tty)" != "not a tty" ]]; then have_tty=1 fi # Bugzilla 37848: When no TTY is available, don't output to console have_tty=0 -# shellcheck disable=SC2006 -if [[ "`tty`" != "not a tty" ]]; then + +if [[ "$(tty)" != "not a tty" ]]; then have_tty=1 fi diff --git a/bin/stop-application.sh b/bin/stop-application.sh index b208ab50527..c11815db63a 100755 --- a/bin/stop-application.sh +++ b/bin/stop-application.sh @@ -16,8 +16,8 @@ # limitations under the License. # -# shellcheck disable=SC2071 -if [[ $# < 1 ]] ; then + +if [[ $# -lt 1 ]] ; then echo "USAGE: $0 " exit 1 fi From 86e4e1ce0baccc4ac56c57c6cc580badc55a1203 Mon Sep 17 00:00:00 2001 From: yuan Date: Fri, 13 Jan 2023 16:46:55 +0800 Subject: [PATCH 02/21] fix-#4057 info: modify the shellcheck errors file in ./bin 1. "$@" is a array, we want use string to compare. so update "$@" => "$*" 2. `tty` mean execute the command, we can use $(tty) replace it 3. param $# is a number, compare number should use -gt/-lt,not >/< 4. not sure the /bin/kyuubi line 63 'exit -1' need modify? so the directory bin only have a shellcheck note in /bin/kyuubi --- bin/docker-image-tool.sh | 4 ++-- bin/kyuubi | 4 ++-- bin/kyuubi-logo | 8 ++++---- bin/stop-application.sh | 4 ++-- 4 files changed, 10 insertions(+), 10 deletions(-) diff --git a/bin/docker-image-tool.sh b/bin/docker-image-tool.sh index e9e4338b5dc..c9a90cf5782 100755 --- a/bin/docker-image-tool.sh +++ b/bin/docker-image-tool.sh @@ -227,8 +227,8 @@ Examples: EOF } -# shellcheck disable=SC2199 -if [[ "$@" = *--help ]] || [[ "$@" = *-h ]]; then + +if [[ "$*" = *--help ]] || [[ "$*" = *-h ]]; then usage exit 0 fi diff --git a/bin/kyuubi b/bin/kyuubi index 414bdeb86ce..b4e0ca37c83 100755 --- a/bin/kyuubi +++ b/bin/kyuubi @@ -30,8 +30,8 @@ function usage() { echo " -h | --help - Show this help message" } -# shellcheck disable=SC2199 -if [[ "$@" = *--help ]] || [[ "$@" = *-h ]]; then + +if [[ "$*" = *--help ]] || [[ "$*" = *-h ]]; then usage exit 0 fi diff --git a/bin/kyuubi-logo b/bin/kyuubi-logo index 15a45a4bbc0..b59a12d8853 100755 --- a/bin/kyuubi-logo +++ b/bin/kyuubi-logo @@ -18,15 +18,15 @@ # Bugzilla 37848: When no TTY is available, don't output to console have_tty=0 -# shellcheck disable=SC2006 -if [[ "`tty`" != "not a tty" ]]; then + +if [[ "$(tty)" != "not a tty" ]]; then have_tty=1 fi # Bugzilla 37848: When no TTY is available, don't output to console have_tty=0 -# shellcheck disable=SC2006 -if [[ "`tty`" != "not a tty" ]]; then + +if [[ "$(tty)" != "not a tty" ]]; then have_tty=1 fi diff --git a/bin/stop-application.sh b/bin/stop-application.sh index b208ab50527..c11815db63a 100755 --- a/bin/stop-application.sh +++ b/bin/stop-application.sh @@ -16,8 +16,8 @@ # limitations under the License. # -# shellcheck disable=SC2071 -if [[ $# < 1 ]] ; then + +if [[ $# -lt 1 ]] ; then echo "USAGE: $0 " exit 1 fi From c48ad38c7a1323a88dd3be1fe8016f98dea8b0b2 Mon Sep 17 00:00:00 2001 From: yuanfuyuan <1406957364@qq.com> Date: Sun, 5 Mar 2023 17:52:44 +0800 Subject: [PATCH 03/21] remove the used blank lines --- bin/docker-image-tool.sh | 1 - bin/kyuubi | 1 - bin/kyuubi-logo | 3 --- bin/stop-application.sh | 1 - 4 files changed, 6 deletions(-) diff --git a/bin/docker-image-tool.sh b/bin/docker-image-tool.sh index c9a90cf5782..509da0afb24 100755 --- a/bin/docker-image-tool.sh +++ b/bin/docker-image-tool.sh @@ -227,7 +227,6 @@ Examples: EOF } - if [[ "$*" = *--help ]] || [[ "$*" = *-h ]]; then usage exit 0 diff --git a/bin/kyuubi b/bin/kyuubi index b4e0ca37c83..3ebfad46fb7 100755 --- a/bin/kyuubi +++ b/bin/kyuubi @@ -30,7 +30,6 @@ function usage() { echo " -h | --help - Show this help message" } - if [[ "$*" = *--help ]] || [[ "$*" = *-h ]]; then usage exit 0 diff --git a/bin/kyuubi-logo b/bin/kyuubi-logo index b59a12d8853..1f95ca02e52 100755 --- a/bin/kyuubi-logo +++ b/bin/kyuubi-logo @@ -15,17 +15,14 @@ # See the License for the specific language governing permissions and # limitations under the License. # - # Bugzilla 37848: When no TTY is available, don't output to console have_tty=0 - if [[ "$(tty)" != "not a tty" ]]; then have_tty=1 fi # Bugzilla 37848: When no TTY is available, don't output to console have_tty=0 - if [[ "$(tty)" != "not a tty" ]]; then have_tty=1 fi diff --git a/bin/stop-application.sh b/bin/stop-application.sh index c11815db63a..000eb4cdd7c 100755 --- a/bin/stop-application.sh +++ b/bin/stop-application.sh @@ -16,7 +16,6 @@ # limitations under the License. # - if [[ $# -lt 1 ]] ; then echo "USAGE: $0 " exit 1 From b61604442177d0a169e118ab0929f24d1d9eb913 Mon Sep 17 00:00:00 2001 From: yuanfuyuan <1406957364@qq.com> Date: Wed, 27 Sep 2023 01:34:02 +0800 Subject: [PATCH 04/21] fix_4186 --- .../spark/kyuubi/SQLOperationListener.scala | 18 ++++++++++++++---- .../spark/kyuubi/SparkConsoleProgressBar.scala | 10 +++++++++- .../org/apache/spark/kyuubi/StageStatus.scala | 3 +++ 3 files changed, 26 insertions(+), 5 deletions(-) diff --git a/externals/kyuubi-spark-sql-engine/src/main/scala/org/apache/spark/kyuubi/SQLOperationListener.scala b/externals/kyuubi-spark-sql-engine/src/main/scala/org/apache/spark/kyuubi/SQLOperationListener.scala index 1a57fcf2994..00bcf591389 100644 --- a/externals/kyuubi-spark-sql-engine/src/main/scala/org/apache/spark/kyuubi/SQLOperationListener.scala +++ b/externals/kyuubi-spark-sql-engine/src/main/scala/org/apache/spark/kyuubi/SQLOperationListener.scala @@ -44,15 +44,15 @@ class SQLOperationListener( spark: SparkSession) extends StatsReportListener with Logging { private val operationId: String = operation.getHandle.identifier.toString - private lazy val activeJobs = new java.util.HashSet[Int]() + private lazy val activeJobs = new ConcurrentHashMap[Int, JobInfo]() private lazy val activeStages = new ConcurrentHashMap[StageAttempt, StageInfo]() private var executionId: Option[Long] = None - private val conf: KyuubiConf = operation.getSession.sessionManager.getConf private lazy val consoleProgressBar = if (conf.get(ENGINE_SPARK_SHOW_PROGRESS)) { Some(new SparkConsoleProgressBar( operation, + activeJobs, activeStages, conf.get(ENGINE_SPARK_SHOW_PROGRESS_UPDATE_INTERVAL), conf.get(ENGINE_SPARK_SHOW_PROGRESS_TIME_FORMAT))) @@ -82,6 +82,7 @@ class SQLOperationListener( override def onJobStart(jobStart: SparkListenerJobStart): Unit = activeJobs.synchronized { if (sameGroupId(jobStart.properties)) { val jobId = jobStart.jobId + val stageIds = jobStart.stageInfos.map(_.stageId).toList val stageSize = jobStart.stageInfos.size if (executionId.isEmpty) { executionId = Option(jobStart.properties.getProperty(SPARK_SQL_EXECUTION_ID_KEY)) @@ -94,7 +95,10 @@ class SQLOperationListener( } } withOperationLog { - activeJobs.add(jobId) + activeJobs.put( + jobId, + new JobInfo(stageSize, stageIds) + ) info(s"Query [$operationId]: Job $jobId started with $stageSize stages," + s" ${activeJobs.size()} active jobs running") } @@ -103,7 +107,7 @@ class SQLOperationListener( override def onJobEnd(jobEnd: SparkListenerJobEnd): Unit = activeJobs.synchronized { val jobId = jobEnd.jobId - if (activeJobs.remove(jobId)) { + if (activeJobs.remove(jobId) != null ) { val hint = jobEnd.jobResult match { case JobSucceeded => "succeeded" case _ => "failed" // TODO: Handle JobFailed(exception: Exception) @@ -135,6 +139,11 @@ class SQLOperationListener( override def onStageCompleted(stageCompleted: SparkListenerStageCompleted): Unit = { val stageInfo = stageCompleted.stageInfo val stageAttempt = StageAttempt(stageInfo.stageId, stageInfo.attemptNumber()) + activeJobs.forEach((_, jobInfo) => { + if (jobInfo.stageIds.contains(stageInfo.stageId)) { + jobInfo.numCompleteStages += 1 + } + }) activeStages.synchronized { if (activeStages.remove(stageAttempt) != null) { withOperationLog(super.onStageCompleted(stageCompleted)) @@ -142,6 +151,7 @@ class SQLOperationListener( } } + override def onTaskStart(taskStart: SparkListenerTaskStart): Unit = activeStages.synchronized { val stageAttempt = StageAttempt(taskStart.stageId, taskStart.stageAttemptId) if (activeStages.containsKey(stageAttempt)) { diff --git a/externals/kyuubi-spark-sql-engine/src/main/scala/org/apache/spark/kyuubi/SparkConsoleProgressBar.scala b/externals/kyuubi-spark-sql-engine/src/main/scala/org/apache/spark/kyuubi/SparkConsoleProgressBar.scala index fc2ebd5f8c8..3fb859617a6 100644 --- a/externals/kyuubi-spark-sql-engine/src/main/scala/org/apache/spark/kyuubi/SparkConsoleProgressBar.scala +++ b/externals/kyuubi-spark-sql-engine/src/main/scala/org/apache/spark/kyuubi/SparkConsoleProgressBar.scala @@ -29,6 +29,7 @@ import org.apache.kyuubi.operation.Operation class SparkConsoleProgressBar( operation: Operation, + liveJobs: ConcurrentHashMap[Int, JobInfo], liveStages: ConcurrentHashMap[StageAttempt, StageInfo], updatePeriodMSec: Long, timeFormat: String) @@ -80,6 +81,13 @@ class SparkConsoleProgressBar( private def show(now: Long, stages: Seq[StageInfo]): Unit = { val width = TerminalWidth / stages.size val bar = stages.map { s => + // build job log info + val jobId: Option[Int] = liveJobs.asScala.find { + case (jobId, jobInfo) => jobInfo.stageIds.contains(s.stageId) + }.map(_._1) + val jobInfoHeader = s"[Job ${jobId} " + + s"(${liveJobs.get(jobId).numCompleteStages} / ${liveJobs.get(jobId).numStages}) Stages] " + // build stage log info val total = s.numTasks val header = s"[Stage ${s.stageId}:" val tailer = s"(${s.numCompleteTasks} + ${s.numActiveTasks}) / $total]" @@ -93,7 +101,7 @@ class SparkConsoleProgressBar( } else { "" } - header + bar + tailer + jobInfoHeader + header + bar + tailer }.mkString("") // only refresh if it's changed OR after 1 minute (or the ssh connection will be closed diff --git a/externals/kyuubi-spark-sql-engine/src/main/scala/org/apache/spark/kyuubi/StageStatus.scala b/externals/kyuubi-spark-sql-engine/src/main/scala/org/apache/spark/kyuubi/StageStatus.scala index 14457086254..ae9f2ff9d82 100644 --- a/externals/kyuubi-spark-sql-engine/src/main/scala/org/apache/spark/kyuubi/StageStatus.scala +++ b/externals/kyuubi-spark-sql-engine/src/main/scala/org/apache/spark/kyuubi/StageStatus.scala @@ -21,6 +21,9 @@ case class StageAttempt(stageId: Int, stageAttemptId: Int) { override def toString: String = s"Stage $stageId (Attempt $stageAttemptId)" } +class JobInfo(val numStages: Int, val stageIds: Seq[Int]) { + var numCompleteStages = 0 +} class StageInfo(val stageId: Int, val numTasks: Int) { var numActiveTasks = 0 var numCompleteTasks = 0 From 40e80d9a89f023577f57d9024b8d4133d0fee57a Mon Sep 17 00:00:00 2001 From: david yuan <51512358+davidyuan1223@users.noreply.github.com> Date: Wed, 27 Sep 2023 12:08:07 +0800 Subject: [PATCH 05/21] Revert "fix_4186" --- .../spark/kyuubi/SQLOperationListener.scala | 25 +++++-------------- .../kyuubi/SparkConsoleProgressBar.scala | 12 ++------- .../org/apache/spark/kyuubi/StageStatus.scala | 10 +++----- 3 files changed, 11 insertions(+), 36 deletions(-) diff --git a/externals/kyuubi-spark-sql-engine/src/main/scala/org/apache/spark/kyuubi/SQLOperationListener.scala b/externals/kyuubi-spark-sql-engine/src/main/scala/org/apache/spark/kyuubi/SQLOperationListener.scala index 5e7e33712ec..4e4a940d295 100644 --- a/externals/kyuubi-spark-sql-engine/src/main/scala/org/apache/spark/kyuubi/SQLOperationListener.scala +++ b/externals/kyuubi-spark-sql-engine/src/main/scala/org/apache/spark/kyuubi/SQLOperationListener.scala @@ -44,17 +44,15 @@ class SQLOperationListener( spark: SparkSession) extends StatsReportListener with Logging { private val operationId: String = operation.getHandle.identifier.toString - - private lazy val activeJobs = new ConcurrentHashMap[Int, JobInfo]() - private lazy val activeStages = new ConcurrentHashMap[StageAttempt, StageInfo]() - + private lazy val activeJobs = new java.util.HashSet[Int]() + private lazy val activeStages = new ConcurrentHashMap[SparkStageAttempt, SparkStageInfo]() private var executionId: Option[Long] = None + private val conf: KyuubiConf = operation.getSession.sessionManager.getConf private lazy val consoleProgressBar = if (conf.get(ENGINE_SPARK_SHOW_PROGRESS)) { Some(new SparkConsoleProgressBar( operation, - activeJobs, activeStages, conf.get(ENGINE_SPARK_SHOW_PROGRESS_UPDATE_INTERVAL), conf.get(ENGINE_SPARK_SHOW_PROGRESS_TIME_FORMAT))) @@ -84,7 +82,6 @@ class SQLOperationListener( override def onJobStart(jobStart: SparkListenerJobStart): Unit = activeJobs.synchronized { if (sameGroupId(jobStart.properties)) { val jobId = jobStart.jobId - val stageIds = jobStart.stageInfos.map(_.stageId).toList val stageSize = jobStart.stageInfos.size if (executionId.isEmpty) { executionId = Option(jobStart.properties.getProperty(SPARK_SQL_EXECUTION_ID_KEY)) @@ -97,10 +94,7 @@ class SQLOperationListener( } } withOperationLog { - activeJobs.put( - jobId, - new JobInfo(stageSize, stageIds) - ) + activeJobs.add(jobId) info(s"Query [$operationId]: Job $jobId started with $stageSize stages," + s" ${activeJobs.size()} active jobs running") } @@ -109,7 +103,7 @@ class SQLOperationListener( override def onJobEnd(jobEnd: SparkListenerJobEnd): Unit = activeJobs.synchronized { val jobId = jobEnd.jobId - if (activeJobs.remove(jobId) != null ) { + if (activeJobs.remove(jobId)) { val hint = jobEnd.jobResult match { case JobSucceeded => "succeeded" case _ => "failed" // TODO: Handle JobFailed(exception: Exception) @@ -140,13 +134,7 @@ class SQLOperationListener( override def onStageCompleted(stageCompleted: SparkListenerStageCompleted): Unit = { val stageInfo = stageCompleted.stageInfo - val stageAttempt = StageAttempt(stageInfo.stageId, stageInfo.attemptNumber()) - activeJobs.forEach((_, jobInfo) => { - if (jobInfo.stageIds.contains(stageInfo.stageId)) { - jobInfo.numCompleteStages += 1 - } - }) - + val stageAttempt = SparkStageAttempt(stageInfo.stageId, stageInfo.attemptNumber()) activeStages.synchronized { if (activeStages.remove(stageAttempt) != null) { withOperationLog(super.onStageCompleted(stageCompleted)) @@ -154,7 +142,6 @@ class SQLOperationListener( } } - override def onTaskStart(taskStart: SparkListenerTaskStart): Unit = activeStages.synchronized { val stageAttempt = SparkStageAttempt(taskStart.stageId, taskStart.stageAttemptId) if (activeStages.containsKey(stageAttempt)) { diff --git a/externals/kyuubi-spark-sql-engine/src/main/scala/org/apache/spark/kyuubi/SparkConsoleProgressBar.scala b/externals/kyuubi-spark-sql-engine/src/main/scala/org/apache/spark/kyuubi/SparkConsoleProgressBar.scala index 148427214fa..dc8b493cc04 100644 --- a/externals/kyuubi-spark-sql-engine/src/main/scala/org/apache/spark/kyuubi/SparkConsoleProgressBar.scala +++ b/externals/kyuubi-spark-sql-engine/src/main/scala/org/apache/spark/kyuubi/SparkConsoleProgressBar.scala @@ -29,8 +29,7 @@ import org.apache.kyuubi.operation.Operation class SparkConsoleProgressBar( operation: Operation, - liveJobs: ConcurrentHashMap[Int, JobInfo], - liveStages: ConcurrentHashMap[StageAttempt, StageInfo], + liveStages: ConcurrentHashMap[SparkStageAttempt, SparkStageInfo], updatePeriodMSec: Long, timeFormat: String) extends Logging { @@ -81,13 +80,6 @@ class SparkConsoleProgressBar( private def show(now: Long, stages: Seq[SparkStageInfo]): Unit = { val width = TerminalWidth / stages.size val bar = stages.map { s => - // build job log info - val jobId: Option[Int] = liveJobs.asScala.find { - case (jobId, jobInfo) => jobInfo.stageIds.contains(s.stageId) - }.map(_._1) - val jobInfoHeader = s"[Job ${jobId} " + - s"(${liveJobs.get(jobId).numCompleteStages} / ${liveJobs.get(jobId).numStages}) Stages] " - // build stage log info val total = s.numTasks val header = s"[Stage ${s.stageId}:" val tailer = s"(${s.numCompleteTasks} + ${s.numActiveTasks}) / $total]" @@ -101,7 +93,7 @@ class SparkConsoleProgressBar( } else { "" } - jobInfoHeader + header + bar + tailer + header + bar + tailer }.mkString("") // only refresh if it's changed OR after 1 minute (or the ssh connection will be closed diff --git a/externals/kyuubi-spark-sql-engine/src/main/scala/org/apache/spark/kyuubi/StageStatus.scala b/externals/kyuubi-spark-sql-engine/src/main/scala/org/apache/spark/kyuubi/StageStatus.scala index 1e78e1c2f4a..2ea9c3fdae6 100644 --- a/externals/kyuubi-spark-sql-engine/src/main/scala/org/apache/spark/kyuubi/StageStatus.scala +++ b/externals/kyuubi-spark-sql-engine/src/main/scala/org/apache/spark/kyuubi/StageStatus.scala @@ -23,11 +23,7 @@ case class SparkStageAttempt(stageId: Int, stageAttemptId: Int) { override def toString: String = s"Stage $stageId (Attempt $stageAttemptId)" } -class JobInfo(val numStages: Int, val stageIds: Seq[Int]) { - var numCompleteStages = 0 -} - -class StageInfo(val stageId: Int, val numTasks: Int) { - var numActiveTasks = 0 - var numCompleteTasks = 0 +class SparkStageInfo(val stageId: Int, val numTasks: Int) { + var numActiveTasks = new AtomicInteger(0) + var numCompleteTasks = new AtomicInteger(0) } From 56b91a321c30d0908f255f4acb22854751c4dfe7 Mon Sep 17 00:00:00 2001 From: yuanfuyuan <1406957364@qq.com> Date: Tue, 10 Oct 2023 22:48:13 +0800 Subject: [PATCH 06/21] fix_4186 --- .../spark/kyuubi/SQLOperationListener.scala | 67 ++++++++++++------- .../kyuubi/SparkConsoleProgressBar.scala | 18 ++++- .../org/apache/spark/kyuubi/StageStatus.scala | 5 ++ 3 files changed, 63 insertions(+), 27 deletions(-) diff --git a/externals/kyuubi-spark-sql-engine/src/main/scala/org/apache/spark/kyuubi/SQLOperationListener.scala b/externals/kyuubi-spark-sql-engine/src/main/scala/org/apache/spark/kyuubi/SQLOperationListener.scala index 4e4a940d295..c362af72870 100644 --- a/externals/kyuubi-spark-sql-engine/src/main/scala/org/apache/spark/kyuubi/SQLOperationListener.scala +++ b/externals/kyuubi-spark-sql-engine/src/main/scala/org/apache/spark/kyuubi/SQLOperationListener.scala @@ -44,7 +44,7 @@ class SQLOperationListener( spark: SparkSession) extends StatsReportListener with Logging { private val operationId: String = operation.getHandle.identifier.toString - private lazy val activeJobs = new java.util.HashSet[Int]() + private lazy val activeJobs = new ConcurrentHashMap[Int, SparkJobInfo]() private lazy val activeStages = new ConcurrentHashMap[SparkStageAttempt, SparkStageInfo]() private var executionId: Option[Long] = None @@ -53,6 +53,7 @@ class SQLOperationListener( if (conf.get(ENGINE_SPARK_SHOW_PROGRESS)) { Some(new SparkConsoleProgressBar( operation, + activeJobs, activeStages, conf.get(ENGINE_SPARK_SHOW_PROGRESS_UPDATE_INTERVAL), conf.get(ENGINE_SPARK_SHOW_PROGRESS_TIME_FORMAT))) @@ -79,37 +80,45 @@ class SQLOperationListener( } } - override def onJobStart(jobStart: SparkListenerJobStart): Unit = activeJobs.synchronized { - if (sameGroupId(jobStart.properties)) { - val jobId = jobStart.jobId - val stageSize = jobStart.stageInfos.size - if (executionId.isEmpty) { - executionId = Option(jobStart.properties.getProperty(SPARK_SQL_EXECUTION_ID_KEY)) - .map(_.toLong) - consoleProgressBar - operation match { - case executeStatement: ExecuteStatement => - executeStatement.setCompiledStateIfNeeded() - case _ => + override def onJobStart(jobStart: SparkListenerJobStart): Unit = { + activeJobs.synchronized { + if (sameGroupId(jobStart.properties)) { + val jobId = jobStart.jobId + val stageIds = jobStart.stageInfos.map(_.stageId) + val stageSize = jobStart.stageInfos.size + if (executionId.isEmpty) { + executionId = Option(jobStart.properties.getProperty(SPARK_SQL_EXECUTION_ID_KEY)) + .map(_.toLong) + consoleProgressBar + operation match { + case executeStatement: ExecuteStatement => + executeStatement.setCompiledStateIfNeeded() + case _ => + } + } + activeJobs.put( + jobId, + new SparkJobInfo(stageSize, stageIds) + ) + withOperationLog { + info(s"Query [$operationId]: Job $jobId started with $stageSize stages," + + s" ${activeJobs.size()} active jobs running") } - } - withOperationLog { - activeJobs.add(jobId) - info(s"Query [$operationId]: Job $jobId started with $stageSize stages," + - s" ${activeJobs.size()} active jobs running") } } } override def onJobEnd(jobEnd: SparkListenerJobEnd): Unit = activeJobs.synchronized { val jobId = jobEnd.jobId - if (activeJobs.remove(jobId)) { - val hint = jobEnd.jobResult match { - case JobSucceeded => "succeeded" - case _ => "failed" // TODO: Handle JobFailed(exception: Exception) - } - withOperationLog { - info(s"Query [$operationId]: Job $jobId $hint, ${activeJobs.size()} active jobs running") + activeJobs.synchronized { + if (activeJobs.remove(jobId) != null) { + val hint = jobEnd.jobResult match { + case JobSucceeded => "succeeded" + case _ => "failed" // TODO: Handle JobFailed(exception: Exception) + } + withOperationLog { + info(s"Query [$operationId]: Job $jobId $hint, ${activeJobs.size()} active jobs running") + } } } } @@ -134,9 +143,17 @@ class SQLOperationListener( override def onStageCompleted(stageCompleted: SparkListenerStageCompleted): Unit = { val stageInfo = stageCompleted.stageInfo + val stageId = stageInfo.stageId val stageAttempt = SparkStageAttempt(stageInfo.stageId, stageInfo.attemptNumber()) activeStages.synchronized { if (activeStages.remove(stageAttempt) != null) { + activeJobs.synchronized { + activeJobs.forEach((jobId, sparkJobInfo) => { + if (sparkJobInfo.stageIds.contains(stageId)) { + sparkJobInfo.numCompleteStages.getAndIncrement() + } + }) + } withOperationLog(super.onStageCompleted(stageCompleted)) } } diff --git a/externals/kyuubi-spark-sql-engine/src/main/scala/org/apache/spark/kyuubi/SparkConsoleProgressBar.scala b/externals/kyuubi-spark-sql-engine/src/main/scala/org/apache/spark/kyuubi/SparkConsoleProgressBar.scala index dc8b493cc04..27b41c2b8b4 100644 --- a/externals/kyuubi-spark-sql-engine/src/main/scala/org/apache/spark/kyuubi/SparkConsoleProgressBar.scala +++ b/externals/kyuubi-spark-sql-engine/src/main/scala/org/apache/spark/kyuubi/SparkConsoleProgressBar.scala @@ -29,6 +29,7 @@ import org.apache.kyuubi.operation.Operation class SparkConsoleProgressBar( operation: Operation, + liveJobs: ConcurrentHashMap[Int, SparkJobInfo], liveStages: ConcurrentHashMap[SparkStageAttempt, SparkStageInfo], updatePeriodMSec: Long, timeFormat: String) @@ -71,7 +72,14 @@ class SparkConsoleProgressBar( show(now, stages.take(3)) // display at most 3 stages in same time } } - + private def findJobId(stageId: Int): Int = { + liveJobs.forEach((jobId, sparkJobInfo) => { + if (sparkJobInfo.stageIds.contains(stageId)) { + return jobId + } + }) + -1 + } /** * Show progress bar in console. The progress bar is displayed in the next line * after your last output, keeps overwriting itself to hold in one line. The logging will follow @@ -81,7 +89,13 @@ class SparkConsoleProgressBar( val width = TerminalWidth / stages.size val bar = stages.map { s => val total = s.numTasks - val header = s"[Stage ${s.stageId}:" + val jobId = findJobId(s.stageId) + var jobHeader = s"[There is no job about this stage]" + if (jobId != -1) { + jobHeader = s"[Job $jobId (${liveJobs.get(jobId).numCompleteStages} " + + s"/ ${liveJobs.get(jobId).numStages}) Stages] " + } + val header = jobHeader + s"[Stage ${s.stageId}:" val tailer = s"(${s.numCompleteTasks} + ${s.numActiveTasks}) / $total]" val w = width - header.length - tailer.length val bar = diff --git a/externals/kyuubi-spark-sql-engine/src/main/scala/org/apache/spark/kyuubi/StageStatus.scala b/externals/kyuubi-spark-sql-engine/src/main/scala/org/apache/spark/kyuubi/StageStatus.scala index 2ea9c3fdae6..cd745ea1921 100644 --- a/externals/kyuubi-spark-sql-engine/src/main/scala/org/apache/spark/kyuubi/StageStatus.scala +++ b/externals/kyuubi-spark-sql-engine/src/main/scala/org/apache/spark/kyuubi/StageStatus.scala @@ -27,3 +27,8 @@ class SparkStageInfo(val stageId: Int, val numTasks: Int) { var numActiveTasks = new AtomicInteger(0) var numCompleteTasks = new AtomicInteger(0) } + + +class SparkJobInfo(val numStages: Int, val stageIds: Seq[Int]) { + var numCompleteStages = new AtomicInteger(0) +} \ No newline at end of file From 8b51840bc43df9fb831f39488567eeb0b3c0e5d5 Mon Sep 17 00:00:00 2001 From: davidyuan Date: Thu, 26 Oct 2023 10:46:07 +0800 Subject: [PATCH 07/21] add common method to get session level config --- .../src/main/scala/org/apache/spark/kyuubi/StageStatus.scala | 5 ----- 1 file changed, 5 deletions(-) diff --git a/externals/kyuubi-spark-sql-engine/src/main/scala/org/apache/spark/kyuubi/StageStatus.scala b/externals/kyuubi-spark-sql-engine/src/main/scala/org/apache/spark/kyuubi/StageStatus.scala index 9e30c69e78f..29644f9f4c7 100644 --- a/externals/kyuubi-spark-sql-engine/src/main/scala/org/apache/spark/kyuubi/StageStatus.scala +++ b/externals/kyuubi-spark-sql-engine/src/main/scala/org/apache/spark/kyuubi/StageStatus.scala @@ -31,8 +31,3 @@ class SparkStageInfo(val stageId: Int, val numTasks: Int) { class SparkJobInfo(val numStages: Int, val stageIds: Set[Int]) { val numCompleteStages = new AtomicInteger(0) } - - -class SparkJobInfo(val numStages: Int, val stageIds: Seq[Int]) { - var numCompleteStages = new AtomicInteger(0) -} \ No newline at end of file From 3d6c53b533fbe22fd7c866d4d7f4cce7c3447b33 Mon Sep 17 00:00:00 2001 From: davidyuan Date: Thu, 23 May 2024 11:18:48 +0800 Subject: [PATCH 08/21] add new module common-grpc --- .../org/apache/kyuubi/config/KyuubiConf.scala | 22 + kyuubi-grpc/pom.xml | 392 ++++++++++++++++++ .../grpc/events/OperationEventsManager.scala | 141 +++++++ .../grpc/events/SessionEventsManager.scala | 60 +++ .../operation/AbstractGrpcOperation.scala | 78 ++++ .../kyuubi/grpc/operation/GrpcOperation.scala | 33 ++ .../grpc/operation/GrpcOperationManager.scala | 83 ++++ .../kyuubi/grpc/operation/OperationKey.scala | 32 ++ .../service/AbstractGrpcBackendService.scala | 29 ++ .../service/AbstractGrpcFrontendService.scala | 153 +++++++ .../grpc/service/GrpcBackendService.scala | 24 ++ .../grpc/service/GrpcFrontendService.scala | 33 ++ .../kyuubi/grpc/service/GrpcSeverable.scala | 66 +++ .../grpc/session/AbstractGrpcSession.scala | 97 +++++ .../kyuubi/grpc/session/GrpcSession.scala | 46 ++ .../grpc/session/GrpcSessionManager.scala | 214 ++++++++++ .../kyuubi/grpc/session/SessionKey.scala | 27 ++ .../org/apache/kyuubi/grpc/utils/Clock.scala | 55 +++ .../apache/kyuubi/grpc/utils/ProtoUtils.scala | 34 ++ .../test/resources/protobuf/test_case.proto | 57 +++ .../kyuubi/grpc/client/SimpleRpcClient.scala | 48 +++ .../event/SimpleOperationEventsManager.scala | 36 ++ .../event/SimpleSessionEventsManager.scala | 36 ++ .../operation/SimpleAddOperationImpl.scala | 45 ++ .../operation/SimpleGrpcOperationImpl.scala | 62 +++ .../SimpleGrpcOperationManager.scala | 45 ++ .../SimpleOpenSessionOperationImpl.scala | 40 ++ .../grpc/server/GrpcSeverableSuite.scala | 73 ++++ .../kyuubi/grpc/server/SimpleGrpcServer.scala | 21 + .../service/SimpleGrpcBackendService.scala | 49 +++ .../service/SimpleGrpcFrontendService.scala | 64 +++ .../grpc/service/SimpleGrpcSeverable.scala | 30 ++ .../grpc/session/SimpleGrpcSessionImpl.scala | 67 +++ .../session/SimpleGrpcSessionManager.scala | 32 ++ pom.xml | 1 + 35 files changed, 2325 insertions(+) create mode 100644 kyuubi-grpc/pom.xml create mode 100644 kyuubi-grpc/src/main/scala/org/apache/kyuubi/grpc/events/OperationEventsManager.scala create mode 100644 kyuubi-grpc/src/main/scala/org/apache/kyuubi/grpc/events/SessionEventsManager.scala create mode 100644 kyuubi-grpc/src/main/scala/org/apache/kyuubi/grpc/operation/AbstractGrpcOperation.scala create mode 100644 kyuubi-grpc/src/main/scala/org/apache/kyuubi/grpc/operation/GrpcOperation.scala create mode 100644 kyuubi-grpc/src/main/scala/org/apache/kyuubi/grpc/operation/GrpcOperationManager.scala create mode 100644 kyuubi-grpc/src/main/scala/org/apache/kyuubi/grpc/operation/OperationKey.scala create mode 100644 kyuubi-grpc/src/main/scala/org/apache/kyuubi/grpc/service/AbstractGrpcBackendService.scala create mode 100644 kyuubi-grpc/src/main/scala/org/apache/kyuubi/grpc/service/AbstractGrpcFrontendService.scala create mode 100644 kyuubi-grpc/src/main/scala/org/apache/kyuubi/grpc/service/GrpcBackendService.scala create mode 100644 kyuubi-grpc/src/main/scala/org/apache/kyuubi/grpc/service/GrpcFrontendService.scala create mode 100644 kyuubi-grpc/src/main/scala/org/apache/kyuubi/grpc/service/GrpcSeverable.scala create mode 100644 kyuubi-grpc/src/main/scala/org/apache/kyuubi/grpc/session/AbstractGrpcSession.scala create mode 100644 kyuubi-grpc/src/main/scala/org/apache/kyuubi/grpc/session/GrpcSession.scala create mode 100644 kyuubi-grpc/src/main/scala/org/apache/kyuubi/grpc/session/GrpcSessionManager.scala create mode 100644 kyuubi-grpc/src/main/scala/org/apache/kyuubi/grpc/session/SessionKey.scala create mode 100644 kyuubi-grpc/src/main/scala/org/apache/kyuubi/grpc/utils/Clock.scala create mode 100644 kyuubi-grpc/src/main/scala/org/apache/kyuubi/grpc/utils/ProtoUtils.scala create mode 100644 kyuubi-grpc/src/test/resources/protobuf/test_case.proto create mode 100644 kyuubi-grpc/src/test/scala/org/apache/kyuubi/grpc/client/SimpleRpcClient.scala create mode 100644 kyuubi-grpc/src/test/scala/org/apache/kyuubi/grpc/event/SimpleOperationEventsManager.scala create mode 100644 kyuubi-grpc/src/test/scala/org/apache/kyuubi/grpc/event/SimpleSessionEventsManager.scala create mode 100644 kyuubi-grpc/src/test/scala/org/apache/kyuubi/grpc/operation/SimpleAddOperationImpl.scala create mode 100644 kyuubi-grpc/src/test/scala/org/apache/kyuubi/grpc/operation/SimpleGrpcOperationImpl.scala create mode 100644 kyuubi-grpc/src/test/scala/org/apache/kyuubi/grpc/operation/SimpleGrpcOperationManager.scala create mode 100644 kyuubi-grpc/src/test/scala/org/apache/kyuubi/grpc/operation/SimpleOpenSessionOperationImpl.scala create mode 100644 kyuubi-grpc/src/test/scala/org/apache/kyuubi/grpc/server/GrpcSeverableSuite.scala create mode 100644 kyuubi-grpc/src/test/scala/org/apache/kyuubi/grpc/server/SimpleGrpcServer.scala create mode 100644 kyuubi-grpc/src/test/scala/org/apache/kyuubi/grpc/service/SimpleGrpcBackendService.scala create mode 100644 kyuubi-grpc/src/test/scala/org/apache/kyuubi/grpc/service/SimpleGrpcFrontendService.scala create mode 100644 kyuubi-grpc/src/test/scala/org/apache/kyuubi/grpc/service/SimpleGrpcSeverable.scala create mode 100644 kyuubi-grpc/src/test/scala/org/apache/kyuubi/grpc/session/SimpleGrpcSessionImpl.scala create mode 100644 kyuubi-grpc/src/test/scala/org/apache/kyuubi/grpc/session/SimpleGrpcSessionManager.scala 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 3eedfdded91..bf0c760cddf 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 @@ -2621,6 +2621,28 @@ object KyuubiConf { .stringConf .createWithDefault("ENGINE") + val ENGINE_SPARK_CONNECT_GRPC_BINDING_PORT: ConfigEntry[Int] = + buildConf("kyuubi.engine.spark.connect.grpc.bind.port") + .doc("The port is used in spark connect frontendService start GrpcServer") + .version("1.9.0") + .intConf + .createWithDefault(15002) + + val ENGINE_SPARK_CONNECT_GRPC_MAX_INBOUND_MESSAGE_SIZE: ConfigEntry[Int] = + buildConf("kyuubi.engine.spark.connect.grpc.max.inbound.message.size") + .doc("Sets the maximum inbound message in bytes size for the gRPC requests." + + "Requests with a larger payload will fail.") + .version("1.9.0") + .intConf + .createWithDefault(128 * 1024 * 1024) + + val ENGINE_SPARK_CONNECT_GRPC_BINDING_HOST: ConfigEntry[Option[String]] = + buildConf("kyuubi.engine.spark.connect.grpc.bind.host") + .doc("Hostname or IP of the machine on which to run the grpc server in frontend service ") + .version("1.9.0") + .serverOnly + .fallbackConf(FRONTEND_BIND_HOST) + val ENGINE_SPARK_SHOW_PROGRESS: ConfigEntry[Boolean] = buildConf("kyuubi.session.engine.spark.showProgress") .doc("When true, show the progress bar in the Spark's engine log.") diff --git a/kyuubi-grpc/pom.xml b/kyuubi-grpc/pom.xml new file mode 100644 index 00000000000..43f73cb1499 --- /dev/null +++ b/kyuubi-grpc/pom.xml @@ -0,0 +1,392 @@ + + + 4.0.0 + + org.apache.kyuubi + kyuubi-parent + 1.9.0-SNAPSHOT + ../pom.xml + + + kyuubi-grpc_${scala.binary.version} + jar + Kyuubi Project Grpc + https://kyuubi.apache.org/ + + + + org.apache.kyuubi + kyuubi-common_${scala.binary.version} + ${project.version} + + + + org.apache.kyuubi + kyuubi-ha_${scala.binary.version} + ${project.version} + + + + io.grpc + grpc-core + + + + io.grpc + grpc-protobuf + + + io.grpc + grpc-util + + + + io.grpc + grpc-stub + + + + com.google.protobuf + protobuf-java + ${protobuf.version} + compile + + + + org.scala-lang + scala-compiler + provided + + + + org.scala-lang + scala-reflect + provided + + + + org.apache.kyuubi + kyuubi-common_${scala.binary.version} + ${project.version} + test-jar + test + + + + org.scala-lang.modules + scala-collection-compat_${scala.binary.version} + + + + commons-collections + commons-collections + test + + + + commons-io + commons-io + test + + + + com.dimafeng + testcontainers-scala-scalatest_${scala.binary.version} + test + + + + + + + + org.codehaus.mojo + build-helper-maven-plugin + + + add-scala-sources + + add-source + + generate-sources + + + src/main/scala-${scala.binary.version} + + + + + add-scala-test-sources + + add-test-source + + generate-test-sources + + + src/test/scala-${scala.binary.version} + + + + + + + org.apache.maven.plugins + maven-shade-plugin + + false + + + com.google.android:annotations + com.google.api.grpc:proto-google-common-protos + com.google.code.gson:gson + com.google.errorprone:error_prone_annotations + com.google.guava:* + com.google.j2objc:j2objc-annotations + com.google.protobuf:* + dev.failsafe:failsafe + io.etcd:* + io.grpc:* + io.netty:* + io.perfmark:perfmark-api + io.vertx:* + org.apache.kyuubi:* + org.checkerframework:checker-qual + org.codehaus.mojo:animal-sniffer-annotations + + + + + *:* + + **/*.proto + META-INF/*.SF + META-INF/*.DSA + META-INF/*.RSA + META-INF/DEPENDENCIES + META-INF/LICENSE.txt + META-INF/NOTICE.txt + META-INF/maven/** + LICENSE.txt + NOTICE.txt + mozilla/** + **/module-info.class + + + + + + dev.failsafe + ${kyuubi.shade.packageName}.dev.failsafe + + dev.failsafe.** + + + + io.etcd + ${kyuubi.shade.packageName}.io.etcd + + io.etcd.** + + + + io.grpc + ${kyuubi.shade.packageName}.io.grpc + + + io.netty + ${kyuubi.shade.packageName}.io.netty + + io.netty.** + + + + io.perfmark + ${kyuubi.shade.packageName}.io.perfmark + + + io.vertx + ${kyuubi.shade.packageName}.io.vertx + + io.vertx.** + + + + android.annotation + ${kyuubi.shade.packageName}.android.annotation + + + com.google.common + ${kyuubi.shade.packageName}.com.google.common + + com.google.common.** + + + + com.google.thirdparty + ${kyuubi.shade.packageName}.com.google.thirdparty + + com.google.thirdparty.** + + + + com.google.protobuf + ${kyuubi.shade.packageName}.com.google.protobuf + + com.google.protobuf.** + + + + org.codehaus.mojo.animal_sniffer + ${kyuubi.shade.packageName}.org.codehaus.mojo.animal_sniffer + + + com.google.j2objc.annotations + ${kyuubi.shade.packageName}.com.google.j2objc.annotations + + + com.google.errorprone.annotations + ${kyuubi.shade.packageName}.com.google.errorprone.annotations + + + org.checkerframework + ${kyuubi.shade.packageName}.org.checkerframework + + + com.google.gson + ${kyuubi.shade.packageName}.com.google.gson + + + + com.google.api + ${kyuubi.shade.packageName}.com.google.api + + + com.google.cloud + ${kyuubi.shade.packageName}.com.google.cloud + + + com.google.geo + ${kyuubi.shade.packageName}.com.google.geo + + + com.google.logging + ${kyuubi.shade.packageName}.com.google.logging + + + com.google.longrunning + ${kyuubi.shade.packageName}.com.google.longrunning + + + com.google.rpc + ${kyuubi.shade.packageName}.com.google.rpc + + + com.google.type + ${kyuubi.shade.packageName}.com.google.type + + + + + + + + + + shade + + package + + + + + + org.apache.maven.plugins + maven-antrun-plugin + + + rename-native-library + + run + + package + + + unpacking netty jar + + renaming netty native libraries + + + + + deleting META-INF/native-image folder + + repackaging netty jar + + + + + + + + + org.apache.maven.plugins + maven-jar-plugin + + + prepare-test-jar + + test-jar + + test-compile + + + + + target/scala-${scala.binary.version}/classes + target/scala-${scala.binary.version}/test-classes + + + kr.motd.maven + os-maven-plugin + 1.6.2 + + + + + + default-protoc + + true + + + + + org.xolstice.maven.plugins + protobuf-maven-plugin + 0.6.1 + + com.google.protobuf:protoc:${protobuf.version}:exe:${os.detected.classifier} + grpc-java + io.grpc:protoc-gen-grpc-java:${grpc.version}:exe:${os.detected.classifier} + src/test/resources/protobuf + + + + + compile + compile-custom + test-compile + + + + + + + + + diff --git a/kyuubi-grpc/src/main/scala/org/apache/kyuubi/grpc/events/OperationEventsManager.scala b/kyuubi-grpc/src/main/scala/org/apache/kyuubi/grpc/events/OperationEventsManager.scala new file mode 100644 index 00000000000..507b944c3be --- /dev/null +++ b/kyuubi-grpc/src/main/scala/org/apache/kyuubi/grpc/events/OperationEventsManager.scala @@ -0,0 +1,141 @@ +/* + * 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.grpc.events + +import org.apache.kyuubi.grpc.operation.GrpcOperation +import org.apache.kyuubi.grpc.session.GrpcSession +import org.apache.kyuubi.grpc.utils.Clock + +object OperationEventsManager { + // TODO: make this configurable + val MAX_STATEMENT_TEXT_SIZE = 65535 +} + +sealed abstract class OperationStatus(value: Int) + +object OperationStatus { + case object Pending extends OperationStatus(0) + case object Started extends OperationStatus(1) + case object Analyzed extends OperationStatus(2) + case object ReadyForExecution extends OperationStatus(3) + case object Finished extends OperationStatus(4) + case object Failed extends OperationStatus(5) + case object Canceled extends OperationStatus(6) + case object Closed extends OperationStatus(7) +} +abstract class OperationEventsManager(operation: GrpcOperation, clock: Clock) { + private def operationId: String = operation.operationKey.operationId + + private def session: GrpcSession = operation.grpcSession + + private def sessionId: String = session.sessionKey.sessionId + + private def sessionStatus = session.sessionEventsManager.status + + protected var _status: OperationStatus = OperationStatus.Pending + + private var error = Option.empty[Boolean] + + private var canceled = Option.empty[Boolean] + + private var producedRowCount = Option.empty[Long] + + private def status: OperationStatus = _status + + private def hasCanceled: Option[Boolean] = canceled + + private def hasError: Option[Boolean] = error + + private def getProduceRowCount: Option[Long] = producedRowCount + + def postStarted(): Unit = { + assertStatus(List(OperationStatus.Pending), OperationStatus.Started) + } + + def postAnalyzed(analyzedPlan: Option[Any] = None): Unit = { + assertStatus(List(OperationStatus.Started, OperationStatus.Analyzed), OperationStatus.Analyzed) + } + + def postReadyForExecution(): Unit = { + assertStatus(List(OperationStatus.Analyzed), OperationStatus.ReadyForExecution) + } + + def postCanceled(): Unit = { + assertStatus( + List( + OperationStatus.Started, + OperationStatus.Analyzed, + OperationStatus.ReadyForExecution, + OperationStatus.Finished, + OperationStatus.Failed), + OperationStatus.Canceled) + canceled = Some(true) + } + + def postFailed(errorMessage: String): Unit = { + assertStatus( + List( + OperationStatus.Started, + OperationStatus.Analyzed, + OperationStatus.ReadyForExecution, + OperationStatus.Finished), + OperationStatus.Failed) + error = Some(true) + } + + def postFinished(producedRowCountOpt: Option[Long] = None): Unit = { + assertStatus( + List( + OperationStatus.Started, + OperationStatus.ReadyForExecution), + OperationStatus.Finished) + producedRowCount = producedRowCountOpt + } + + def postClosed(): Unit = { + assertStatus( + List( + OperationStatus.Finished, + OperationStatus.Failed, + OperationStatus.Canceled), + OperationStatus.Closed) + } + + def status_(operationStatus: OperationStatus): Unit = { + _status = operationStatus + } + + private def assertStatus( + validStatuses: List[OperationStatus], + eventStatus: OperationStatus): Unit = { + if (!validStatuses.contains(status)) { + throw new IllegalStateException( + s""" + |operationId: $operationId with status ${status} + |is not within statuses $validStatuses for event $eventStatus + |""".stripMargin) + } +// if (sessionStatus != SessionStatus.Started) { +// throw new IllegalStateException( +// s""" +// |sessionId: $sessionId with status $sessionStatus +// |is not Started for event $eventStatus +// |""".stripMargin) +// } + _status = eventStatus + } +} diff --git a/kyuubi-grpc/src/main/scala/org/apache/kyuubi/grpc/events/SessionEventsManager.scala b/kyuubi-grpc/src/main/scala/org/apache/kyuubi/grpc/events/SessionEventsManager.scala new file mode 100644 index 00000000000..0d174588bd0 --- /dev/null +++ b/kyuubi-grpc/src/main/scala/org/apache/kyuubi/grpc/events/SessionEventsManager.scala @@ -0,0 +1,60 @@ +/* + * 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.grpc.events + +import org.apache.kyuubi.grpc.session.GrpcSession +import org.apache.kyuubi.grpc.utils.Clock + +sealed abstract class SessionStatus(value: Int) + +object SessionStatus { + case object Pending extends SessionStatus(0) + case object Started extends SessionStatus(1) + case object Closed extends SessionStatus(2) +} + +abstract class SessionEventsManager(session: GrpcSession, clock: Clock) { + private def sessionId: String = session.sessionKey.sessionId + + private var _status: SessionStatus = SessionStatus.Pending + + protected def status_(sessionStatus: SessionStatus): Unit = { + _status = sessionStatus + } + + def status: SessionStatus = _status + + def postStarted(): Unit = { + assertStatus(List(SessionStatus.Pending), SessionStatus.Started) + status_(SessionStatus.Started) + } + + def postClosed(): Unit = { + assertStatus(List(SessionStatus.Started), SessionStatus.Closed) + status_(SessionStatus.Closed) + } + + private def assertStatus(validStatuses: List[SessionStatus], eventStatus: SessionStatus): Unit = { + if (!validStatuses.contains(status)) { + throw new IllegalStateException( + s""" + |sessionId: $sessionId with status ${status} + |is not within statuses $validStatuses for event $eventStatus + |""".stripMargin) + } + } +} diff --git a/kyuubi-grpc/src/main/scala/org/apache/kyuubi/grpc/operation/AbstractGrpcOperation.scala b/kyuubi-grpc/src/main/scala/org/apache/kyuubi/grpc/operation/AbstractGrpcOperation.scala new file mode 100644 index 00000000000..eee82dde59c --- /dev/null +++ b/kyuubi-grpc/src/main/scala/org/apache/kyuubi/grpc/operation/AbstractGrpcOperation.scala @@ -0,0 +1,78 @@ +/* + * 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.grpc.operation + +import java.util.concurrent.locks.ReentrantLock + +import org.apache.kyuubi.{KyuubiSQLException, Logging, Utils} +import org.apache.kyuubi.grpc.session.GrpcSession + +abstract class AbstractGrpcOperation[S <: GrpcSession](session: S) extends GrpcOperation + with Logging { + final protected val opType: String = getClass.getSimpleName + final protected val createTime = System.currentTimeMillis() + protected def key: OperationKey + final private val operationTimeout: Long = 1000 + private var lock: ReentrantLock = new ReentrantLock() + + protected def withLockRequired[T](block: => T): T = Utils.withLockRequired(lock)(block) + + @volatile protected var startTime: Long = _ + @volatile protected var completedTime: Long = _ + @volatile protected var lastAccessTime: Long = createTime + + @volatile protected var operationException: KyuubiSQLException = _ + + protected def setOperationException(ex: KyuubiSQLException): Unit = { + this.operationException = ex + } + + protected def runInternal(): Unit + + protected def beforeRun(): Unit + + protected def afterRun(): Unit + + override def run(): Unit = { + beforeRun() + try { + runInternal() + } finally { + afterRun() + } + } + + override def close(): Unit + + override def operationKey: OperationKey = key + + override def grpcSession: S = session + +} + +object OperationJobTag { + def apply(prefix: String, operationKey: OperationKey): String = { + s"${prefix}_" + + s"User_${operationKey.userId}_" + + s"Session_${operationKey.sessionId}_" + + s"Operation_${operationKey.operationId}" + } + + def unapply(jobTag: String, prefix: String): Option[String] = { + if (jobTag.startsWith(prefix)) Some(jobTag) else None + } +} diff --git a/kyuubi-grpc/src/main/scala/org/apache/kyuubi/grpc/operation/GrpcOperation.scala b/kyuubi-grpc/src/main/scala/org/apache/kyuubi/grpc/operation/GrpcOperation.scala new file mode 100644 index 00000000000..42a3b18ca5a --- /dev/null +++ b/kyuubi-grpc/src/main/scala/org/apache/kyuubi/grpc/operation/GrpcOperation.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.grpc.operation + +import org.apache.kyuubi.grpc.events.OperationEventsManager +import org.apache.kyuubi.grpc.session.GrpcSession +import org.apache.kyuubi.operation.log.OperationLog + +trait GrpcOperation { + def run(): Unit + def interrupt(): Unit + def close(): Unit + + def getOperationLog: Option[OperationLog] + def isTimedOut: Boolean + def grpcSession: GrpcSession + def operationKey: OperationKey + def operationEventsManager: OperationEventsManager +} diff --git a/kyuubi-grpc/src/main/scala/org/apache/kyuubi/grpc/operation/GrpcOperationManager.scala b/kyuubi-grpc/src/main/scala/org/apache/kyuubi/grpc/operation/GrpcOperationManager.scala new file mode 100644 index 00000000000..12913a97c50 --- /dev/null +++ b/kyuubi-grpc/src/main/scala/org/apache/kyuubi/grpc/operation/GrpcOperationManager.scala @@ -0,0 +1,83 @@ +/* + * 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.grpc.operation + +import java.util.concurrent._ + +import scala.collection.JavaConverters._ + +import org.apache.kyuubi.KyuubiSQLException +import org.apache.kyuubi.config.KyuubiConf +import org.apache.kyuubi.operation.log.LogDivertAppender +import org.apache.kyuubi.service.AbstractService + +/** + * The [[GrpcOperationManager]] manages all the grpc operations during their lifecycle + */ +abstract class GrpcOperationManager(name: String) + extends AbstractService(name) { + + private val keyToOperations = new ConcurrentHashMap[OperationKey, GrpcOperation] + + protected def skipOperationLog: Boolean = false + def getOperationCount: Int = keyToOperations.size() + + def allOperations(): Iterable[GrpcOperation] = keyToOperations.values().asScala + + override def initialize(conf: KyuubiConf): Unit = { + LogDivertAppender.initialize(skipOperationLog) + super.initialize(conf) + } + + def close(operationKey: OperationKey): Unit = { + val operation = keyToOperations.get(operationKey) + if (operation == null) throw KyuubiSQLException(s"Invalid $operationKey") + operation.close() + } + + final def addOperation(grpcOperation: GrpcOperation): GrpcOperation = synchronized { + keyToOperations.put(grpcOperation.operationKey, grpcOperation) + grpcOperation + } + + @throws[KyuubiSQLException] + final def getOperation(operationKey: OperationKey): GrpcOperation = { + val operation = synchronized { keyToOperations.get(operationKey) } + if (operation == null) throw KyuubiSQLException(s"Invalid $operationKey") + operation + } + + @throws[KyuubiSQLException] + final def removeOperation(operationKey: OperationKey): GrpcOperation = synchronized { + val operation = keyToOperations.remove(operationKey) + if (operation == null) throw KyuubiSQLException(s"Invalid $operationKey") + operation + } + + @throws[KyuubiSQLException] + final def closeOperation(operationKey: OperationKey): Unit = { + val operation = removeOperation(operationKey) + operation.close() + } + + @throws[KyuubiSQLException] + final def interruptOperation(operationKey: OperationKey): Unit = { + val operation = getOperation(operationKey) + operation.interrupt() + } + +} diff --git a/kyuubi-grpc/src/main/scala/org/apache/kyuubi/grpc/operation/OperationKey.scala b/kyuubi-grpc/src/main/scala/org/apache/kyuubi/grpc/operation/OperationKey.scala new file mode 100644 index 00000000000..badf83de361 --- /dev/null +++ b/kyuubi-grpc/src/main/scala/org/apache/kyuubi/grpc/operation/OperationKey.scala @@ -0,0 +1,32 @@ +/* + * 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.grpc.operation + +import java.util.UUID + +import org.apache.kyuubi.grpc.session.SessionKey + +case class OperationKey(userId: String, sessionId: String, operationId: String) { + override def toString: String = + s"Session: [{$userId}_{$sessionId}], OperationId: [$operationId]" +} + +object OperationKey { + def apply(key: SessionKey): OperationKey = + new OperationKey(key.userId, key.sessionId, UUID.randomUUID().toString) + +} diff --git a/kyuubi-grpc/src/main/scala/org/apache/kyuubi/grpc/service/AbstractGrpcBackendService.scala b/kyuubi-grpc/src/main/scala/org/apache/kyuubi/grpc/service/AbstractGrpcBackendService.scala new file mode 100644 index 00000000000..f7e29627caf --- /dev/null +++ b/kyuubi-grpc/src/main/scala/org/apache/kyuubi/grpc/service/AbstractGrpcBackendService.scala @@ -0,0 +1,29 @@ +/* + * 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.grpc.service + +import org.apache.kyuubi.config.KyuubiConf +import org.apache.kyuubi.service.CompositeService + +abstract class AbstractGrpcBackendService(name: String) + extends CompositeService(name) with GrpcBackendService { + + override def initialize(conf: KyuubiConf): Unit = { + addService(grpcSessionManager) + super.initialize(conf) + } +} diff --git a/kyuubi-grpc/src/main/scala/org/apache/kyuubi/grpc/service/AbstractGrpcFrontendService.scala b/kyuubi-grpc/src/main/scala/org/apache/kyuubi/grpc/service/AbstractGrpcFrontendService.scala new file mode 100644 index 00000000000..4328eab1d62 --- /dev/null +++ b/kyuubi-grpc/src/main/scala/org/apache/kyuubi/grpc/service/AbstractGrpcFrontendService.scala @@ -0,0 +1,153 @@ +/* + * 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.grpc.service + +import java.net.{InetAddress, InetSocketAddress} +import java.util.concurrent.TimeUnit +import java.util.concurrent.atomic.AtomicBoolean + +import com.google.protobuf.MessageLite +import io.grpc._ +import io.grpc.MethodDescriptor.PrototypeMarshaller +import io.grpc.netty.NettyServerBuilder +import io.grpc.protobuf.lite.ProtoLiteUtils + +import org.apache.kyuubi.{KyuubiException, Logging, Utils} +import org.apache.kyuubi.config.KyuubiConf +import org.apache.kyuubi.config.KyuubiConf.{ENGINE_SPARK_CONNECT_GRPC_BINDING_PORT, FRONTEND_ADVERTISED_HOST} +import org.apache.kyuubi.service.CompositeService +import org.apache.kyuubi.util.NamedThreadFactory + +abstract class AbstractGrpcFrontendService(name: String) + extends CompositeService(name) with GrpcFrontendService with Runnable + with BindableService with Logging { + + private val started = new AtomicBoolean(false) + protected var server: Server = _ + protected def portNum: Int = conf.get(ENGINE_SPARK_CONNECT_GRPC_BINDING_PORT) + protected def maxInboundMessageSize: Int = 1024 + + protected def serverHost: Option[String] + protected lazy val serverAddr: InetAddress = + serverHost.map(InetAddress.getByName).getOrElse(Utils.findLocalInetAddress) + + private lazy val serverThread = new NamedThreadFactory(getName, false).newThread(this) + + override def initialize(conf: KyuubiConf): Unit = { + this.conf = conf + try { + val socketAddress = new InetSocketAddress(serverAddr.getHostName, portNum) + val nettyServerBuilder = NettyServerBuilder + .forAddress(socketAddress) + .maxInboundMessageSize(maxInboundMessageSize) + .addService(this) + server = nettyServerBuilder.build() + } catch { + case e: Throwable => + error(e) + throw new KyuubiException( + s"Failed to initialize grpc frontend service on $portNum", + e) + } + super.initialize(conf) + } + + override def bindService(): ServerServiceDefinition + + protected def methodWithCustomMarshallers(methodDesc: MethodDescriptor[MessageLite, MessageLite]) + : MethodDescriptor[MessageLite, MessageLite] = { + // default 1024 + val recursionLimit = 1024 + val requestMarshaller = + ProtoLiteUtils.marshallerWithRecursionLimit( + methodDesc.getRequestMarshaller + .asInstanceOf[PrototypeMarshaller[MessageLite]] + .getMessagePrototype, + recursionLimit) + val responseMarshaller = + ProtoLiteUtils.marshallerWithRecursionLimit( + methodDesc.getResponseMarshaller + .asInstanceOf[PrototypeMarshaller[MessageLite]] + .getMessagePrototype, + recursionLimit) + methodDesc.toBuilder + .setRequestMarshaller(requestMarshaller) + .setResponseMarshaller(responseMarshaller) + .build() + } + + override def start(): Unit = { + try { + if (started.compareAndSet(false, true)) { + serverThread.start() + } + super.start() + } catch { + case e: Throwable => + stopInternal() + throw e + } + } + + private def stopInternal(): Unit = { + if (started.compareAndSet(true, false)) { + serverThread.interrupt() + stopServer(Some(10L), Some(TimeUnit.SECONDS)) + info(getName + " has stoppped") + } + } + + override def stop(): Unit = { + super.stop() + stopInternal() + } + + def stopServer(timeout: Option[Long] = None, unit: Option[TimeUnit] = None): Unit = { + if (server != null) { + if (timeout.isDefined && unit.isDefined) { + server.shutdown() + server.awaitTermination(timeout.get, unit.get) + } else { + server.shutdown() + } + } + } + + override def run(): Unit = { + try { + server.start() + info("Grpc Server Start Success") + } catch { + case _: InterruptedException => error(s"$getName is interrupted") + case t: Throwable => + error(s"Error starting $getName", t) + System.exit(-1) + } + } + + override def connectionUrl: String = { + val host = (conf.get(FRONTEND_ADVERTISED_HOST), serverHost) match { + case (Some(advertisedHost), _) => advertisedHost + case (None, Some(h)) => h + case (None, None) => serverAddr.getHostAddress + } + host + ":" + portNum + } + + protected def isServer(): Boolean = false + +} diff --git a/kyuubi-grpc/src/main/scala/org/apache/kyuubi/grpc/service/GrpcBackendService.scala b/kyuubi-grpc/src/main/scala/org/apache/kyuubi/grpc/service/GrpcBackendService.scala new file mode 100644 index 00000000000..59220edd39f --- /dev/null +++ b/kyuubi-grpc/src/main/scala/org/apache/kyuubi/grpc/service/GrpcBackendService.scala @@ -0,0 +1,24 @@ +/* + * 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.grpc.service + +import org.apache.kyuubi.grpc.session.{GrpcSession, GrpcSessionManager} + +trait GrpcBackendService { + + def grpcSessionManager: GrpcSessionManager[_ <: GrpcSession] +} diff --git a/kyuubi-grpc/src/main/scala/org/apache/kyuubi/grpc/service/GrpcFrontendService.scala b/kyuubi-grpc/src/main/scala/org/apache/kyuubi/grpc/service/GrpcFrontendService.scala new file mode 100644 index 00000000000..d7a11bc2769 --- /dev/null +++ b/kyuubi-grpc/src/main/scala/org/apache/kyuubi/grpc/service/GrpcFrontendService.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.grpc.service + +import org.apache.kyuubi.service.Service + +trait GrpcFrontendService { + + def connectionUrl: String + + val serverable: GrpcSeverable[_ <: GrpcBackendService, _ <: GrpcFrontendService] + + final def be: GrpcBackendService = serverable.backendService + + val discoveryService: Option[Service] + + def attributes: Map[String, String] = Map.empty + +} diff --git a/kyuubi-grpc/src/main/scala/org/apache/kyuubi/grpc/service/GrpcSeverable.scala b/kyuubi-grpc/src/main/scala/org/apache/kyuubi/grpc/service/GrpcSeverable.scala new file mode 100644 index 00000000000..0d5dcfd0bb7 --- /dev/null +++ b/kyuubi-grpc/src/main/scala/org/apache/kyuubi/grpc/service/GrpcSeverable.scala @@ -0,0 +1,66 @@ +/* + * 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.grpc.service + +import java.util.concurrent.atomic.AtomicBoolean + +import org.apache.kyuubi.config.KyuubiConf +import org.apache.kyuubi.service.CompositeService + +abstract class GrpcSeverable[B <: AbstractGrpcBackendService, F <: AbstractGrpcFrontendService]( + name: String) extends CompositeService(name) { + private val started = new AtomicBoolean(false) + + var selfExited = false + + val backendService: B + + val frontendServices: Seq[F] + + override def initialize(conf: KyuubiConf): Unit = synchronized { + this.conf = conf + addService(backendService) + frontendServices.foreach(addService) + super.initialize(conf) + } + + override def start(): Unit = synchronized { + if (!started.getAndSet(true)) { + super.start() + } + } + + protected def stopServer(): Unit + + override def stop(): Unit = synchronized { + try { + if (started.getAndSet(false)) { + super.stop() + } + } catch { + case t: Throwable => + warn(s"Error stopping $name ${t.getMessage}", t) + } finally { + try { + stopServer() + } catch { + case t: Throwable => + warn(s"Error stopping $name ${t.getMessage}", t) + } + } + } +} diff --git a/kyuubi-grpc/src/main/scala/org/apache/kyuubi/grpc/session/AbstractGrpcSession.scala b/kyuubi-grpc/src/main/scala/org/apache/kyuubi/grpc/session/AbstractGrpcSession.scala new file mode 100644 index 00000000000..485d0c01d2b --- /dev/null +++ b/kyuubi-grpc/src/main/scala/org/apache/kyuubi/grpc/session/AbstractGrpcSession.scala @@ -0,0 +1,97 @@ +/* + * 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.grpc.session + +import java.util + +import org.apache.kyuubi.{KyuubiSQLException, Logging} +import org.apache.kyuubi.grpc.operation.{GrpcOperation, OperationKey} +import org.apache.kyuubi.grpc.utils.ProtoUtils + +abstract class AbstractGrpcSession( + val userId: String) extends GrpcSession with Logging { + override val sessionKey: SessionKey = SessionKey(userId) + + final private val _createTime: Long = System.currentTimeMillis() + override def createTime: Long = _createTime + + @volatile private var _lastAccessTime: Long = _createTime + override def lastAccessTime: Long = _lastAccessTime + + @volatile private var closedTimeMs: Option[Long] = None + + @volatile private var _lastIdleTime: Long = _createTime + override def lastIdleTime: Long = _lastIdleTime + + final private val opKeySet = new util.HashSet[OperationKey] + + protected def runGrpcOperation(operation: GrpcOperation): OperationKey = { + if (closedTimeMs.isDefined) { + throw KyuubiSQLException("Cannot build operation because the session is closed") + } + try { + val opKey = operation.operationKey + opKeySet.add(opKey) + operation.run() + opKey + } catch { + case e: KyuubiSQLException => + opKeySet.remove(operation.operationKey) + sessionManager.grpcOperationManager.close(operation.operationKey) + throw e + } + } + + override def removeOperation(operationKey: OperationKey): Unit = { + sessionManager.grpcOperationManager.removeOperation(operationKey) + } + + override def closeOperation(operationKey: OperationKey): Unit = { + sessionManager.grpcOperationManager.closeOperation(operationKey) + } + + override def interruptOperation(operationKey: OperationKey): Unit = { + sessionManager.grpcOperationManager.interruptOperation(operationKey) + } + + override def open(): Unit = { + sessionEventsManager.postStarted() + } + + override def close(): Unit = { + if (closedTimeMs.isDefined) { + throw KyuubiSQLException(s"Session ${sessionKey.sessionId} is already closed.") + } + closedTimeMs = Some(System.currentTimeMillis()) + sessionEventsManager.postClosed() + } + +} + +object SessionTag { + def apply(sessionKey: SessionKey, tag: String, prefix: String): String = { + ProtoUtils.throwIfInvalidTag(tag) + s"${prefix}_" + + s"User_${sessionKey.userId}_" + + s"Session_${sessionKey.sessionId}_" + + s"Tag_${tag}" + } + + def unapply(sessionTag: String, prefix: String): Option[String] = { + if (sessionTag.startsWith(prefix)) Some(sessionTag) else None + } +} diff --git a/kyuubi-grpc/src/main/scala/org/apache/kyuubi/grpc/session/GrpcSession.scala b/kyuubi-grpc/src/main/scala/org/apache/kyuubi/grpc/session/GrpcSession.scala new file mode 100644 index 00000000000..d0a8684448b --- /dev/null +++ b/kyuubi-grpc/src/main/scala/org/apache/kyuubi/grpc/session/GrpcSession.scala @@ -0,0 +1,46 @@ +/* + * 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.grpc.session + +import org.apache.kyuubi.grpc.events.SessionEventsManager +import org.apache.kyuubi.grpc.operation.{GrpcOperation, OperationKey} + +trait GrpcSession { + def sessionKey: SessionKey + def name: Option[String] + + def serverSessionId: String + + def createTime: Long + def lastAccessTime: Long + def lastIdleTime: Long + + def sessionManager: GrpcSessionManager[_ <: GrpcSession] + + def sessionEventsManager: SessionEventsManager + + def open() + def close() + + def removeOperation(operationKey: OperationKey): Unit + + def getOperation(operationKey: OperationKey): GrpcOperation + def closeOperation(operationKey: OperationKey): Unit + + def interruptOperation(operationKey: OperationKey): Unit + +} diff --git a/kyuubi-grpc/src/main/scala/org/apache/kyuubi/grpc/session/GrpcSessionManager.scala b/kyuubi-grpc/src/main/scala/org/apache/kyuubi/grpc/session/GrpcSessionManager.scala new file mode 100644 index 00000000000..1a4b4f6c324 --- /dev/null +++ b/kyuubi-grpc/src/main/scala/org/apache/kyuubi/grpc/session/GrpcSessionManager.scala @@ -0,0 +1,214 @@ +/* + * 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.grpc.session + +import java.io.IOException +import java.nio.file.{Files, Paths} +import java.util.concurrent._ +import scala.concurrent.duration.Duration +import org.apache.kyuubi.{KyuubiSQLException, Utils} +import org.apache.kyuubi.config.KyuubiConf +import org.apache.kyuubi.config.KyuubiConf._ +import org.apache.kyuubi.grpc.operation.GrpcOperationManager +import org.apache.kyuubi.service.CompositeService +import org.apache.kyuubi.util.ThreadUtils + +abstract class GrpcSessionManager[S <: AbstractGrpcSession](name: String) + extends CompositeService(name) { + + @volatile private var shutdown = false + + protected var _operationLogRoot: Option[String] = None + + def operationLogRoot: Option[String] = _operationLogRoot + + private def initOperationLogRootDir(): Unit = { + try { + val logRoot = { + if (isServer) { + conf.get(SERVER_OPERATION_LOG_DIR_ROOT) + } else { + conf.get(ENGINE_OPERATION_LOG_DIR_ROOT) + } + } + val logPath = Files.createDirectories(Utils.getAbsolutePathFromWork(logRoot)) + _operationLogRoot = Some(logPath.toString) + } catch { + case e: IOException => + error(s"Failed to initialize operation log root directory: ${_operationLogRoot}", e) + _operationLogRoot = None + } + } + + private val sessionKeyToSession = new ConcurrentHashMap[SessionKey, S] + + @volatile private var _latestLogoutTime: Long = System.currentTimeMillis() + def latestLogoutTime: Long = _latestLogoutTime + + private val timeoutChecker = + ThreadUtils.newDaemonSingleThreadScheduledExecutor(s"$name-timeout-checker") + + protected def isServer: Boolean + + private var execPool: ThreadPoolExecutor = _ + + def grpcOperationManager: GrpcOperationManager + + protected def getOrCreateSession( + key: SessionKey): S + + def getSession(key: SessionKey): S = { + getSessionOption(key).getOrElse(throw KyuubiSQLException(s"Invalid key $key")) + } + + private def getSessionOption(key: SessionKey): Option[S] = { + Option(sessionKeyToSession.get(key)) + } + def openSession( + key: SessionKey): S = { + info(s"Opening grpc session for ${key.userId}") + val session = getOrCreateSession(key) + try { + val key = session.sessionKey + session.open() + setSession(key, session) + logSessionCountInfo(session, "opened") + session + } catch { + case e: Exception => + try { + session.close() + } catch { + case t: Throwable => + warn(s"Error closing session for ${key.userId}", t) + } + throw KyuubiSQLException(e) + } + } + + protected def removeSession(key: SessionKey): Option[S] = { + val session = sessionKeyToSession.remove(key) + Some(session) + } + + protected def shutdownSession(session: S): Unit = { + session.close() + } + + protected def closeSession(key: SessionKey): Unit = { + _latestLogoutTime = System.currentTimeMillis() + val session = sessionKeyToSession.remove(key) + if (session == null) { + throw KyuubiSQLException(s"Invalid $key") + } + logSessionCountInfo(session, "closed") + try { + session.close() + } finally { + deleteOperationLogSessionDir(key) + } + } + + private def deleteOperationLogSessionDir(key: SessionKey): Unit = { + _operationLogRoot.foreach(logRoot => { + val rootPath = Paths.get(logRoot, key.toString) + try { + Utils.deleteDirectoryRecursively(rootPath.toFile) + } catch { + case e: IOException => + error(s"Failed to delete session operation log directory ${rootPath.toString}", e) + } + }) + } + + final protected def setSession(key: SessionKey, session: S): Unit = { + sessionKeyToSession.put(key, session) + } + + protected def logSessionCountInfo(session: S, action: String): Unit = { + info(s"${session.sessionKey.userId}'s ${session.getClass.getSimpleName} with" + + s" ${session.sessionKey.sessionId}${session.name.map("/" + _).getOrElse("")} is $action," + + s" current opening sessions $getOpenSessionCount") + } + + def getOpenSessionCount: Int = sessionKeyToSession.size() + + def getExecPoolSize: Int = { + assert(execPool != null) + execPool.getPoolSize + } + + def getActiveCount: Int = { + assert(execPool != null) + execPool.getActiveCount + } + + def getWorkQueueSize: Int = { + assert(execPool != null) + execPool.getQueue.size() + } + + override def initialize(conf: KyuubiConf): Unit = { + this.conf = conf + addService(grpcOperationManager) + initOperationLogRootDir() + + val poolSize: Int = { + if (isServer) { + conf.get(SERVER_EXEC_POOL_SIZE) + } else { + conf.get(ENGINE_EXEC_POOL_SIZE) + } + } + + val waitQueueSize: Int = { + if (isServer) { + conf.get(SERVER_EXEC_WAIT_QUEUE_SIZE) + } else { + conf.get(ENGINE_EXEC_WAIT_QUEUE_SIZE) + } + } + val keepAliveMs: Long = { + if (isServer) { + conf.get(SERVER_EXEC_KEEPALIVE_TIME) + } else { + conf.get(ENGINE_EXEC_KEEPALIVE_TIME) + } + } + + execPool = ThreadUtils.newDaemonQueuedThreadPool( + poolSize, + waitQueueSize, + keepAliveMs, + s"$name-exec-pool") + super.initialize(conf) + } + + override def stop(): Unit = synchronized { + super.stop() + shutdown = true + val shutdownTimeout: Long = { + if (isServer) { + conf.get(SERVER_EXEC_POOL_SHUTDOWN_TIMEOUT) + } else { + conf.get(ENGINE_EXEC_POOL_SHUTDOWN_TIMEOUT) + } + } + + ThreadUtils.shutdown(execPool, Duration(shutdownTimeout, TimeUnit.MILLISECONDS)) + } +} diff --git a/kyuubi-grpc/src/main/scala/org/apache/kyuubi/grpc/session/SessionKey.scala b/kyuubi-grpc/src/main/scala/org/apache/kyuubi/grpc/session/SessionKey.scala new file mode 100644 index 00000000000..9d944275a0d --- /dev/null +++ b/kyuubi-grpc/src/main/scala/org/apache/kyuubi/grpc/session/SessionKey.scala @@ -0,0 +1,27 @@ +/* + * 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.grpc.session + +import java.util.UUID + +case class SessionKey(userId: String, sessionId: String) { + override def toString: String = s"UserId: [$userId], SessionId: [$sessionId]" +} + +object SessionKey { + def apply(userId: String): SessionKey = new SessionKey(userId, UUID.randomUUID().toString) +} diff --git a/kyuubi-grpc/src/main/scala/org/apache/kyuubi/grpc/utils/Clock.scala b/kyuubi-grpc/src/main/scala/org/apache/kyuubi/grpc/utils/Clock.scala new file mode 100644 index 00000000000..7de04f4002e --- /dev/null +++ b/kyuubi-grpc/src/main/scala/org/apache/kyuubi/grpc/utils/Clock.scala @@ -0,0 +1,55 @@ +/* + * 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.grpc.utils + +trait Clock { + def getTimeMillis(): Long + + def nanoTime(): Long + + def waitTillTime(targetTime: Long): Long +} + +class SystemClock extends Clock { + val minPollTime = 25L + + override def getTimeMillis(): Long = System.currentTimeMillis() + + override def nanoTime(): Long = System.nanoTime() + + override def waitTillTime(targetTime: Long): Long = { + var currentTime = System.currentTimeMillis() + + var waitTime = targetTime - currentTime + if (waitTime <= 0) { + return currentTime + } + + val pollTime = math.max(waitTime / 10.0, minPollTime).toLong + + while (true) { + currentTime = System.currentTimeMillis() + waitTime = targetTime - currentTime + if (waitTime <= 0) { + return currentTime + } + val sleepTime = math.min(waitTime, pollTime) + Thread.sleep(sleepTime) + } + -1 + } +} diff --git a/kyuubi-grpc/src/main/scala/org/apache/kyuubi/grpc/utils/ProtoUtils.scala b/kyuubi-grpc/src/main/scala/org/apache/kyuubi/grpc/utils/ProtoUtils.scala new file mode 100644 index 00000000000..bad28db54b5 --- /dev/null +++ b/kyuubi-grpc/src/main/scala/org/apache/kyuubi/grpc/utils/ProtoUtils.scala @@ -0,0 +1,34 @@ +/* + * 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.grpc.utils + +object ProtoUtils { + + private val JOB_TAGS_SEP = ',' + def throwIfInvalidTag(tag: String): Unit = { + if (tag == null) { + throw new IllegalArgumentException("Tag cannot be null.") + } + if (tag.contains(JOB_TAGS_SEP)) { + throw new IllegalArgumentException( + s"Tag cannot contain '$JOB_TAGS_SEP'.") + } + if (tag.isEmpty) { + throw new IllegalArgumentException("Tag cannot be an empty string.") + } + } +} diff --git a/kyuubi-grpc/src/test/resources/protobuf/test_case.proto b/kyuubi-grpc/src/test/resources/protobuf/test_case.proto new file mode 100644 index 00000000000..61f02925009 --- /dev/null +++ b/kyuubi-grpc/src/test/resources/protobuf/test_case.proto @@ -0,0 +1,57 @@ +/* + * 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. + */ + +syntax = 'proto3'; + +package kyuubi.grpc; + +option java_multiple_files = true; +option java_package = "org.apache.kyuubi.grpc.proto"; +option go_package = "internal/generated"; + +message TestOpenSessionRequest { + string user_id = 1; + string session_id = 2; +} + +message TestAddRequest { + string user_id = 1; + string session_id = 2; + string operation_id = 3; + int64 first_num = 4; + int64 second_num = 5; +} + +message TestOpenSessionResponse { + string session_id = 1; + string server_side_session_id = 2; + string session_info = 3; +} + +message TestAddResponse { + string session_id = 1; + string server_side_session_id = 2; + string operation_id = 3; + int64 result = 4; +} + +service GrpcTestService { + rpc TestOpenSession(TestOpenSessionRequest) returns (TestOpenSessionResponse) {} + + rpc TestAdd(TestAddRequest) returns (TestAddResponse) {} + +} \ No newline at end of file diff --git a/kyuubi-grpc/src/test/scala/org/apache/kyuubi/grpc/client/SimpleRpcClient.scala b/kyuubi-grpc/src/test/scala/org/apache/kyuubi/grpc/client/SimpleRpcClient.scala new file mode 100644 index 00000000000..25e412be36b --- /dev/null +++ b/kyuubi-grpc/src/test/scala/org/apache/kyuubi/grpc/client/SimpleRpcClient.scala @@ -0,0 +1,48 @@ +/* + * 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.grpc.client + +import java.util.UUID + +import io.grpc.ManagedChannel + +import org.apache.kyuubi.grpc.proto.{GrpcTestServiceGrpc, TestAddRequest, TestAddResponse, TestOpenSessionRequest, TestOpenSessionResponse} + +class SimpleRpcClient(val channel: ManagedChannel) { + private val DEFAULT_USER_ID = "kyuubi_grpc_test" + private val sessionId: String = UUID.randomUUID().toString + private val stub = GrpcTestServiceGrpc.newBlockingStub(channel) + + def openSession(): TestOpenSessionResponse = { + val request = TestOpenSessionRequest.newBuilder() + .setUserId(DEFAULT_USER_ID) + .setSessionId(sessionId) + .build() + stub.testOpenSession(request) + } + + def testAdd(firstNum: Int, secondNum: Int): TestAddResponse = { + val request = TestAddRequest.newBuilder() + .setUserId(DEFAULT_USER_ID) + .setSessionId(sessionId) + .setFirstNum(firstNum) + .setSecondNum(secondNum) + .build() + stub.testAdd(request) + } + +} diff --git a/kyuubi-grpc/src/test/scala/org/apache/kyuubi/grpc/event/SimpleOperationEventsManager.scala b/kyuubi-grpc/src/test/scala/org/apache/kyuubi/grpc/event/SimpleOperationEventsManager.scala new file mode 100644 index 00000000000..8c47cbbc02d --- /dev/null +++ b/kyuubi-grpc/src/test/scala/org/apache/kyuubi/grpc/event/SimpleOperationEventsManager.scala @@ -0,0 +1,36 @@ +/* + * 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.grpc.event + +import org.apache.kyuubi.Logging +import org.apache.kyuubi.grpc.events.OperationEventsManager +import org.apache.kyuubi.grpc.operation.GrpcOperation +import org.apache.kyuubi.grpc.utils.Clock + +class SimpleOperationEventsManager(operation: GrpcOperation, clock: Clock) + extends OperationEventsManager(operation, clock) with Logging { + + override def postStarted(): Unit = { + super.postStarted() + info("Operation Event: post Started") + } + + override def postClosed(): Unit = { + info("Operation Event: post Closed") + super.postClosed() + } +} diff --git a/kyuubi-grpc/src/test/scala/org/apache/kyuubi/grpc/event/SimpleSessionEventsManager.scala b/kyuubi-grpc/src/test/scala/org/apache/kyuubi/grpc/event/SimpleSessionEventsManager.scala new file mode 100644 index 00000000000..9a657d10aa0 --- /dev/null +++ b/kyuubi-grpc/src/test/scala/org/apache/kyuubi/grpc/event/SimpleSessionEventsManager.scala @@ -0,0 +1,36 @@ +/* + * 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.grpc.event + +import org.apache.kyuubi.Logging +import org.apache.kyuubi.grpc.events.SessionEventsManager +import org.apache.kyuubi.grpc.session.SimpleGrpcSessionImpl +import org.apache.kyuubi.grpc.utils.Clock + +class SimpleSessionEventsManager(session: SimpleGrpcSessionImpl, clock: Clock) + extends SessionEventsManager(session, clock) with Logging { + + override def postStarted(): Unit = { + super.postStarted() + info("Session Event: post Started") + } + + override def postClosed(): Unit = { + info("Session Event: post Closed") + super.postClosed() + } +} diff --git a/kyuubi-grpc/src/test/scala/org/apache/kyuubi/grpc/operation/SimpleAddOperationImpl.scala b/kyuubi-grpc/src/test/scala/org/apache/kyuubi/grpc/operation/SimpleAddOperationImpl.scala new file mode 100644 index 00000000000..2f9599b7968 --- /dev/null +++ b/kyuubi-grpc/src/test/scala/org/apache/kyuubi/grpc/operation/SimpleAddOperationImpl.scala @@ -0,0 +1,45 @@ +/* + * 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.grpc.operation + +import io.grpc.stub.StreamObserver + +import org.apache.kyuubi.grpc.proto.{TestAddRequest, TestAddResponse} +import org.apache.kyuubi.grpc.session.SimpleGrpcSessionImpl + +class SimpleAddOperationImpl( + grpcSession: SimpleGrpcSessionImpl, + shouldFail: Boolean, + request: TestAddRequest, + responseObserver: StreamObserver[TestAddResponse]) + extends SimpleGrpcOperationImpl(grpcSession, shouldFail) { + + override protected def key: OperationKey = OperationKey(grpcSession.sessionKey) + + override def runInternal(): Unit = { + super.runInternal() + val result = request.getFirstNum + request.getSecondNum + val builder = TestAddResponse.newBuilder() + .setOperationId(operationKey.operationId) + .setSessionId(operationKey.sessionId) + .setServerSideSessionId(grpcSession.serverSessionId) + .setResult(result) + responseObserver.onNext(builder.build()) + responseObserver.onCompleted() + } + +} diff --git a/kyuubi-grpc/src/test/scala/org/apache/kyuubi/grpc/operation/SimpleGrpcOperationImpl.scala b/kyuubi-grpc/src/test/scala/org/apache/kyuubi/grpc/operation/SimpleGrpcOperationImpl.scala new file mode 100644 index 00000000000..12500f63529 --- /dev/null +++ b/kyuubi-grpc/src/test/scala/org/apache/kyuubi/grpc/operation/SimpleGrpcOperationImpl.scala @@ -0,0 +1,62 @@ +/* + * 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.grpc.operation + +import org.apache.kyuubi.{KyuubiSQLException, Logging} +import org.apache.kyuubi.grpc.event.SimpleOperationEventsManager +import org.apache.kyuubi.grpc.events.OperationEventsManager +import org.apache.kyuubi.grpc.session.SimpleGrpcSessionImpl +import org.apache.kyuubi.grpc.utils.SystemClock +import org.apache.kyuubi.operation.log.OperationLog + +class SimpleGrpcOperationImpl( + grpcSession: SimpleGrpcSessionImpl, + shouldFail: Boolean = false) + extends AbstractGrpcOperation[SimpleGrpcSessionImpl](grpcSession) with Logging { + + override def operationEventsManager: OperationEventsManager = + new SimpleOperationEventsManager(this, new SystemClock()) + + override def runInternal(): Unit = { + if (shouldFail) { + val exception = KyuubiSQLException("noop operation err") + setOperationException(exception) + operationEventsManager.postFailed(exception.getMessage) + } + } + + override def beforeRun(): Unit = { + operationEventsManager.postStarted() + } + + override def close(): Unit = { + operationEventsManager.postClosed() + } + + override def afterRun(): Unit = { + info("afterRun") + } + + override def interrupt(): Unit = { + info("interrupt") + } + + override def isTimedOut: Boolean = false + override def getOperationLog: Option[OperationLog] = None + + override protected def key: OperationKey = OperationKey(grpcSession.sessionKey) +} diff --git a/kyuubi-grpc/src/test/scala/org/apache/kyuubi/grpc/operation/SimpleGrpcOperationManager.scala b/kyuubi-grpc/src/test/scala/org/apache/kyuubi/grpc/operation/SimpleGrpcOperationManager.scala new file mode 100644 index 00000000000..8b69a2d5a64 --- /dev/null +++ b/kyuubi-grpc/src/test/scala/org/apache/kyuubi/grpc/operation/SimpleGrpcOperationManager.scala @@ -0,0 +1,45 @@ +/* + * 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.grpc.operation + +import io.grpc.stub.StreamObserver + +import org.apache.kyuubi.grpc.proto.{TestAddRequest, TestAddResponse, TestOpenSessionRequest, TestOpenSessionResponse} +import org.apache.kyuubi.grpc.session.SimpleGrpcSessionImpl + +class SimpleGrpcOperationManager + extends GrpcOperationManager("SimpleGrpcOperationManager") { + + def newSimpleOpenSessionOperation( + session: SimpleGrpcSessionImpl, + shouldFail: Boolean, + request: TestOpenSessionRequest, + responseObserver: StreamObserver[TestOpenSessionResponse]): GrpcOperation = { + val operation = + new SimpleOpenSessionOperationImpl(session, shouldFail, request, responseObserver) + addOperation(operation) + } + + def newSimpleAddOperation( + session: SimpleGrpcSessionImpl, + shouldFail: Boolean, + request: TestAddRequest, + responseObserver: StreamObserver[TestAddResponse]): GrpcOperation = { + val operation = new SimpleAddOperationImpl(session, shouldFail, request, responseObserver) + addOperation(operation) + } +} diff --git a/kyuubi-grpc/src/test/scala/org/apache/kyuubi/grpc/operation/SimpleOpenSessionOperationImpl.scala b/kyuubi-grpc/src/test/scala/org/apache/kyuubi/grpc/operation/SimpleOpenSessionOperationImpl.scala new file mode 100644 index 00000000000..f40d4cc8471 --- /dev/null +++ b/kyuubi-grpc/src/test/scala/org/apache/kyuubi/grpc/operation/SimpleOpenSessionOperationImpl.scala @@ -0,0 +1,40 @@ +/* + * 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.grpc.operation + +import io.grpc.stub.StreamObserver + +import org.apache.kyuubi.grpc.proto.{TestOpenSessionRequest, TestOpenSessionResponse} +import org.apache.kyuubi.grpc.session.SimpleGrpcSessionImpl + +class SimpleOpenSessionOperationImpl( + grpcSession: SimpleGrpcSessionImpl, + shouldFail: Boolean, + request: TestOpenSessionRequest, + responseObserver: StreamObserver[TestOpenSessionResponse]) + extends SimpleGrpcOperationImpl(grpcSession, shouldFail) { + + override protected def key: OperationKey = OperationKey(grpcSession.sessionKey) + + override def runInternal(): Unit = { + super.runInternal() + val builder = TestOpenSessionResponse.newBuilder() + .setSessionId(grpcSession.sessionKey.sessionId) + responseObserver.onNext(builder.build()) + responseObserver.onCompleted() + } +} diff --git a/kyuubi-grpc/src/test/scala/org/apache/kyuubi/grpc/server/GrpcSeverableSuite.scala b/kyuubi-grpc/src/test/scala/org/apache/kyuubi/grpc/server/GrpcSeverableSuite.scala new file mode 100644 index 00000000000..c85e860e699 --- /dev/null +++ b/kyuubi-grpc/src/test/scala/org/apache/kyuubi/grpc/server/GrpcSeverableSuite.scala @@ -0,0 +1,73 @@ +/* + * 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.grpc.server + +import io.grpc.{Grpc, InsecureChannelCredentials} + +import org.apache.kyuubi.KyuubiFunSuite +import org.apache.kyuubi.config.KyuubiConf +import org.apache.kyuubi.grpc.client.SimpleRpcClient + +class GrpcSeverableSuite extends KyuubiFunSuite { + ignore("GrpcSeverable") { + val severable1 = new SimpleGrpcServer() + val conf = KyuubiConf().set(KyuubiConf.ENGINE_SPARK_CONNECT_GRPC_BINDING_PORT, 0) + severable1.initialize(conf) + assert(severable1.getStartTime === 0) + assert(severable1.getConf === conf) + assert(severable1.frontendServices.head.connectionUrl.nonEmpty) + } + + test("invalid port") { + val conf = KyuubiConf().set(KyuubiConf.ENGINE_SPARK_CONNECT_GRPC_BINDING_PORT, 100000) + val server = new SimpleGrpcServer + server.initialize(conf) + server.start() + } + + test("test openSession") { + val conf = KyuubiConf().set(KyuubiConf.ENGINE_SPARK_CONNECT_GRPC_BINDING_PORT, 10023) + val server = new SimpleGrpcServer + server.initialize(conf) + server.start() + val channel = + Grpc.newChannelBuilderForAddress( + "127.0.0.1", + 10023, + InsecureChannelCredentials.create()).build() + val client = new SimpleRpcClient(channel) + client.openSession() + server.stop() + } + + test("test add") { + val conf = KyuubiConf().set(KyuubiConf.ENGINE_SPARK_CONNECT_GRPC_BINDING_PORT, 10023) + val server = new SimpleGrpcServer + server.initialize(conf) + server.start() + val channel = + Grpc.newChannelBuilderForAddress( + "127.0.0.1", + 10023, + InsecureChannelCredentials.create()).build() + val client = new SimpleRpcClient(channel) + val response = client.testAdd(1, 2) + assert(response.getResult == 3) + server.stop() + } + +} diff --git a/kyuubi-grpc/src/test/scala/org/apache/kyuubi/grpc/server/SimpleGrpcServer.scala b/kyuubi-grpc/src/test/scala/org/apache/kyuubi/grpc/server/SimpleGrpcServer.scala new file mode 100644 index 00000000000..c30236c4441 --- /dev/null +++ b/kyuubi-grpc/src/test/scala/org/apache/kyuubi/grpc/server/SimpleGrpcServer.scala @@ -0,0 +1,21 @@ +/* + * 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.grpc.server + +import org.apache.kyuubi.grpc.service.SimpleGrpcSeverable + +class SimpleGrpcServer extends SimpleGrpcSeverable {} diff --git a/kyuubi-grpc/src/test/scala/org/apache/kyuubi/grpc/service/SimpleGrpcBackendService.scala b/kyuubi-grpc/src/test/scala/org/apache/kyuubi/grpc/service/SimpleGrpcBackendService.scala new file mode 100644 index 00000000000..64f7dd4fedb --- /dev/null +++ b/kyuubi-grpc/src/test/scala/org/apache/kyuubi/grpc/service/SimpleGrpcBackendService.scala @@ -0,0 +1,49 @@ +/* + * 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.grpc.service +import io.grpc.stub.StreamObserver + +import org.apache.kyuubi.KyuubiException +import org.apache.kyuubi.grpc.proto.{TestAddRequest, TestAddResponse, TestOpenSessionRequest, TestOpenSessionResponse} +import org.apache.kyuubi.grpc.session.{SessionKey, SimpleGrpcSessionManager} + +class SimpleGrpcBackendService extends AbstractGrpcBackendService("simpleTest") { + override def grpcSessionManager: SimpleGrpcSessionManager = new SimpleGrpcSessionManager() + + override def start(): Unit = { + if (conf.getOption("kyuubi.test.backend.should.fail").exists(_.toBoolean)) { + throw new KyuubiException("should fail backend") + } + super.start() + } + + def openSessionTesr( + key: SessionKey, + request: TestOpenSessionRequest, + responseObserver: StreamObserver[TestOpenSessionResponse]): Unit = { + grpcSessionManager.openSession(key) + .openSession(key, request, responseObserver) + } + + def add( + key: SessionKey, + request: TestAddRequest, + responseObserver: StreamObserver[TestAddResponse]): Unit = { + grpcSessionManager.openSession(key) + .add(key, request, responseObserver) + } +} diff --git a/kyuubi-grpc/src/test/scala/org/apache/kyuubi/grpc/service/SimpleGrpcFrontendService.scala b/kyuubi-grpc/src/test/scala/org/apache/kyuubi/grpc/service/SimpleGrpcFrontendService.scala new file mode 100644 index 00000000000..9b3fc09f36c --- /dev/null +++ b/kyuubi-grpc/src/test/scala/org/apache/kyuubi/grpc/service/SimpleGrpcFrontendService.scala @@ -0,0 +1,64 @@ +/* + * 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.grpc.service + +import scala.jdk.CollectionConverters.collectionAsScalaIterableConverter + +import com.google.protobuf.MessageLite +import io.grpc.{ServerMethodDefinition, ServerServiceDefinition} +import io.grpc.stub.StreamObserver + +import org.apache.kyuubi.grpc.proto.{GrpcTestServiceGrpc, TestAddRequest, TestAddResponse, TestOpenSessionRequest, TestOpenSessionResponse} +import org.apache.kyuubi.grpc.session.SessionKey +import org.apache.kyuubi.service.Service + +class SimpleGrpcFrontendService(grpcSeverable: SimpleGrpcSeverable) + extends AbstractGrpcFrontendService("SimpleGrpcFrontendService") + with GrpcTestServiceGrpc.AsyncService { + + private def grpcBe = be.asInstanceOf[SimpleGrpcBackendService] + override protected def serverHost: Option[String] = Some("localhost") + + override def bindService(): ServerServiceDefinition = { + val serviceDef = GrpcTestServiceGrpc.bindService(this) + val builder = ServerServiceDefinition.builder(serviceDef.getServiceDescriptor.getName) + serviceDef.getMethods.asScala + .asInstanceOf[Iterable[ServerMethodDefinition[MessageLite, MessageLite]]] + .foreach(method => + builder.addMethod( + methodWithCustomMarshallers(method.getMethodDescriptor), + method.getServerCallHandler)) + builder.build() + } + + override def testAdd( + request: TestAddRequest, + responseObserver: StreamObserver[TestAddResponse]): Unit = { + val key = new SessionKey(request.getUserId, request.getSessionId) + grpcBe.add(key, request, responseObserver) + } + + override def testOpenSession( + request: TestOpenSessionRequest, + responseObserver: StreamObserver[TestOpenSessionResponse]): Unit = { + val key = new SessionKey(request.getUserId, request.getSessionId) + grpcBe.openSessionTesr(key, request, responseObserver) + } + + override val serverable: SimpleGrpcSeverable = grpcSeverable + override val discoveryService: Option[Service] = None +} diff --git a/kyuubi-grpc/src/test/scala/org/apache/kyuubi/grpc/service/SimpleGrpcSeverable.scala b/kyuubi-grpc/src/test/scala/org/apache/kyuubi/grpc/service/SimpleGrpcSeverable.scala new file mode 100644 index 00000000000..51566586b9c --- /dev/null +++ b/kyuubi-grpc/src/test/scala/org/apache/kyuubi/grpc/service/SimpleGrpcSeverable.scala @@ -0,0 +1,30 @@ +/* + * 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.grpc.service + +class SimpleGrpcSeverable + extends GrpcSeverable[SimpleGrpcBackendService, SimpleGrpcFrontendService]( + "SimpleGrpcSeverable") { + + override val backendService: SimpleGrpcBackendService = new SimpleGrpcBackendService + override val frontendServices: Seq[SimpleGrpcFrontendService] = + Seq(new SimpleGrpcFrontendService(this)) + + override protected def stopServer(): Unit = { + info(s"stop") + } +} diff --git a/kyuubi-grpc/src/test/scala/org/apache/kyuubi/grpc/session/SimpleGrpcSessionImpl.scala b/kyuubi-grpc/src/test/scala/org/apache/kyuubi/grpc/session/SimpleGrpcSessionImpl.scala new file mode 100644 index 00000000000..b4b15a16b45 --- /dev/null +++ b/kyuubi-grpc/src/test/scala/org/apache/kyuubi/grpc/session/SimpleGrpcSessionImpl.scala @@ -0,0 +1,67 @@ +/* + * 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.grpc.session + +import scala.util.Random + +import io.grpc.stub.StreamObserver + +import org.apache.kyuubi.grpc.event.SimpleSessionEventsManager +import org.apache.kyuubi.grpc.events.SessionEventsManager +import org.apache.kyuubi.grpc.operation.{GrpcOperation, OperationKey} +import org.apache.kyuubi.grpc.proto.{TestAddRequest, TestAddResponse, TestOpenSessionRequest, TestOpenSessionResponse} +import org.apache.kyuubi.grpc.utils.SystemClock + +class SimpleGrpcSessionImpl( + userId: String, + sessionManager: SimpleGrpcSessionManager) + extends AbstractGrpcSession(userId) { + override def name: Option[String] = Some("SimpleGrpcSessionImpl") + + override def serverSessionId: String = Random.nextString(10) + + override def sessionEventsManager: SessionEventsManager = + new SimpleSessionEventsManager(this, new SystemClock) + + def openSession( + key: SessionKey, + request: TestOpenSessionRequest, + responseObserver: StreamObserver[TestOpenSessionResponse]): Unit = { + val operation = sessionManager.grpcOperationManager + .newSimpleOpenSessionOperation(this, false, request, responseObserver) + runGrpcOperation(operation) + } + + def add( + key: SessionKey, + request: TestAddRequest, + responseObserver: StreamObserver[TestAddResponse]): Unit = { + val operation = sessionManager.grpcOperationManager + .newSimpleAddOperation(this, false, request, responseObserver) + runGrpcOperation(operation) + } + + override def getOperation(operationKey: OperationKey): GrpcOperation = + sessionManager.grpcOperationManager.getOperation(operationKey) + + override def sessionManager: GrpcSessionManager[SimpleGrpcSessionImpl] = sessionManager + + override def close(): Unit = { + sessionEventsManager.postClosed() + super.close() + } +} diff --git a/kyuubi-grpc/src/test/scala/org/apache/kyuubi/grpc/session/SimpleGrpcSessionManager.scala b/kyuubi-grpc/src/test/scala/org/apache/kyuubi/grpc/session/SimpleGrpcSessionManager.scala new file mode 100644 index 00000000000..4c44af3dc1c --- /dev/null +++ b/kyuubi-grpc/src/test/scala/org/apache/kyuubi/grpc/session/SimpleGrpcSessionManager.scala @@ -0,0 +1,32 @@ +/* + * 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.grpc.session + +import org.apache.kyuubi.grpc.operation.SimpleGrpcOperationManager + +class SimpleGrpcSessionManager + extends GrpcSessionManager[SimpleGrpcSessionImpl]("simpleTest") { + override protected def isServer: Boolean = true + + override def grpcOperationManager: SimpleGrpcOperationManager = new SimpleGrpcOperationManager() + + override def getOrCreateSession(key: SessionKey): SimpleGrpcSessionImpl = { + val session = new SimpleGrpcSessionImpl(key.userId, this) + session + } + +} diff --git a/pom.xml b/pom.xml index a04c34ff3a9..2b9468d4bdd 100644 --- a/pom.xml +++ b/pom.xml @@ -83,6 +83,7 @@ kyuubi-util kyuubi-util-scala kyuubi-zookeeper + kyuubi-grpc From e16fd7f5a3953c29a3909b06a224da3fd411f7cc Mon Sep 17 00:00:00 2001 From: yuanfuyuan <1406957364@qq.com> Date: Tue, 10 Oct 2023 22:48:13 +0800 Subject: [PATCH 09/21] fix_4186 --- .../src/main/scala/org/apache/spark/kyuubi/StageStatus.scala | 5 +++++ 1 file changed, 5 insertions(+) diff --git a/externals/kyuubi-spark-sql-engine/src/main/scala/org/apache/spark/kyuubi/StageStatus.scala b/externals/kyuubi-spark-sql-engine/src/main/scala/org/apache/spark/kyuubi/StageStatus.scala index 29644f9f4c7..9e30c69e78f 100644 --- a/externals/kyuubi-spark-sql-engine/src/main/scala/org/apache/spark/kyuubi/StageStatus.scala +++ b/externals/kyuubi-spark-sql-engine/src/main/scala/org/apache/spark/kyuubi/StageStatus.scala @@ -31,3 +31,8 @@ class SparkStageInfo(val stageId: Int, val numTasks: Int) { class SparkJobInfo(val numStages: Int, val stageIds: Set[Int]) { val numCompleteStages = new AtomicInteger(0) } + + +class SparkJobInfo(val numStages: Int, val stageIds: Seq[Int]) { + var numCompleteStages = new AtomicInteger(0) +} \ No newline at end of file From a3a4ebdc369ddd540e5944736b33c8a2dbbfaead Mon Sep 17 00:00:00 2001 From: davidyuan Date: Thu, 26 Oct 2023 10:46:07 +0800 Subject: [PATCH 10/21] add common method to get session level config --- .../src/main/scala/org/apache/spark/kyuubi/StageStatus.scala | 5 ----- 1 file changed, 5 deletions(-) diff --git a/externals/kyuubi-spark-sql-engine/src/main/scala/org/apache/spark/kyuubi/StageStatus.scala b/externals/kyuubi-spark-sql-engine/src/main/scala/org/apache/spark/kyuubi/StageStatus.scala index 9e30c69e78f..29644f9f4c7 100644 --- a/externals/kyuubi-spark-sql-engine/src/main/scala/org/apache/spark/kyuubi/StageStatus.scala +++ b/externals/kyuubi-spark-sql-engine/src/main/scala/org/apache/spark/kyuubi/StageStatus.scala @@ -31,8 +31,3 @@ class SparkStageInfo(val stageId: Int, val numTasks: Int) { class SparkJobInfo(val numStages: Int, val stageIds: Set[Int]) { val numCompleteStages = new AtomicInteger(0) } - - -class SparkJobInfo(val numStages: Int, val stageIds: Seq[Int]) { - var numCompleteStages = new AtomicInteger(0) -} \ No newline at end of file From 631ac9a7aa9a12bf5b977b03c1250e4981fb2eca Mon Sep 17 00:00:00 2001 From: davidyuan Date: Thu, 23 May 2024 11:18:48 +0800 Subject: [PATCH 11/21] add new module common-grpc --- .../org/apache/kyuubi/config/KyuubiConf.scala | 22 + kyuubi-grpc/pom.xml | 392 ++++++++++++++++++ .../grpc/events/OperationEventsManager.scala | 141 +++++++ .../grpc/events/SessionEventsManager.scala | 60 +++ .../operation/AbstractGrpcOperation.scala | 78 ++++ .../kyuubi/grpc/operation/GrpcOperation.scala | 33 ++ .../grpc/operation/GrpcOperationManager.scala | 83 ++++ .../kyuubi/grpc/operation/OperationKey.scala | 32 ++ .../service/AbstractGrpcBackendService.scala | 29 ++ .../service/AbstractGrpcFrontendService.scala | 153 +++++++ .../grpc/service/GrpcBackendService.scala | 24 ++ .../grpc/service/GrpcFrontendService.scala | 33 ++ .../kyuubi/grpc/service/GrpcSeverable.scala | 66 +++ .../grpc/session/AbstractGrpcSession.scala | 97 +++++ .../kyuubi/grpc/session/GrpcSession.scala | 46 ++ .../grpc/session/GrpcSessionManager.scala | 214 ++++++++++ .../kyuubi/grpc/session/SessionKey.scala | 27 ++ .../org/apache/kyuubi/grpc/utils/Clock.scala | 55 +++ .../apache/kyuubi/grpc/utils/ProtoUtils.scala | 34 ++ .../test/resources/protobuf/test_case.proto | 57 +++ .../kyuubi/grpc/client/SimpleRpcClient.scala | 48 +++ .../event/SimpleOperationEventsManager.scala | 36 ++ .../event/SimpleSessionEventsManager.scala | 36 ++ .../operation/SimpleAddOperationImpl.scala | 45 ++ .../operation/SimpleGrpcOperationImpl.scala | 62 +++ .../SimpleGrpcOperationManager.scala | 45 ++ .../SimpleOpenSessionOperationImpl.scala | 40 ++ .../grpc/server/GrpcSeverableSuite.scala | 73 ++++ .../kyuubi/grpc/server/SimpleGrpcServer.scala | 21 + .../service/SimpleGrpcBackendService.scala | 49 +++ .../service/SimpleGrpcFrontendService.scala | 64 +++ .../grpc/service/SimpleGrpcSeverable.scala | 30 ++ .../grpc/session/SimpleGrpcSessionImpl.scala | 67 +++ .../session/SimpleGrpcSessionManager.scala | 32 ++ pom.xml | 1 + 35 files changed, 2325 insertions(+) create mode 100644 kyuubi-grpc/pom.xml create mode 100644 kyuubi-grpc/src/main/scala/org/apache/kyuubi/grpc/events/OperationEventsManager.scala create mode 100644 kyuubi-grpc/src/main/scala/org/apache/kyuubi/grpc/events/SessionEventsManager.scala create mode 100644 kyuubi-grpc/src/main/scala/org/apache/kyuubi/grpc/operation/AbstractGrpcOperation.scala create mode 100644 kyuubi-grpc/src/main/scala/org/apache/kyuubi/grpc/operation/GrpcOperation.scala create mode 100644 kyuubi-grpc/src/main/scala/org/apache/kyuubi/grpc/operation/GrpcOperationManager.scala create mode 100644 kyuubi-grpc/src/main/scala/org/apache/kyuubi/grpc/operation/OperationKey.scala create mode 100644 kyuubi-grpc/src/main/scala/org/apache/kyuubi/grpc/service/AbstractGrpcBackendService.scala create mode 100644 kyuubi-grpc/src/main/scala/org/apache/kyuubi/grpc/service/AbstractGrpcFrontendService.scala create mode 100644 kyuubi-grpc/src/main/scala/org/apache/kyuubi/grpc/service/GrpcBackendService.scala create mode 100644 kyuubi-grpc/src/main/scala/org/apache/kyuubi/grpc/service/GrpcFrontendService.scala create mode 100644 kyuubi-grpc/src/main/scala/org/apache/kyuubi/grpc/service/GrpcSeverable.scala create mode 100644 kyuubi-grpc/src/main/scala/org/apache/kyuubi/grpc/session/AbstractGrpcSession.scala create mode 100644 kyuubi-grpc/src/main/scala/org/apache/kyuubi/grpc/session/GrpcSession.scala create mode 100644 kyuubi-grpc/src/main/scala/org/apache/kyuubi/grpc/session/GrpcSessionManager.scala create mode 100644 kyuubi-grpc/src/main/scala/org/apache/kyuubi/grpc/session/SessionKey.scala create mode 100644 kyuubi-grpc/src/main/scala/org/apache/kyuubi/grpc/utils/Clock.scala create mode 100644 kyuubi-grpc/src/main/scala/org/apache/kyuubi/grpc/utils/ProtoUtils.scala create mode 100644 kyuubi-grpc/src/test/resources/protobuf/test_case.proto create mode 100644 kyuubi-grpc/src/test/scala/org/apache/kyuubi/grpc/client/SimpleRpcClient.scala create mode 100644 kyuubi-grpc/src/test/scala/org/apache/kyuubi/grpc/event/SimpleOperationEventsManager.scala create mode 100644 kyuubi-grpc/src/test/scala/org/apache/kyuubi/grpc/event/SimpleSessionEventsManager.scala create mode 100644 kyuubi-grpc/src/test/scala/org/apache/kyuubi/grpc/operation/SimpleAddOperationImpl.scala create mode 100644 kyuubi-grpc/src/test/scala/org/apache/kyuubi/grpc/operation/SimpleGrpcOperationImpl.scala create mode 100644 kyuubi-grpc/src/test/scala/org/apache/kyuubi/grpc/operation/SimpleGrpcOperationManager.scala create mode 100644 kyuubi-grpc/src/test/scala/org/apache/kyuubi/grpc/operation/SimpleOpenSessionOperationImpl.scala create mode 100644 kyuubi-grpc/src/test/scala/org/apache/kyuubi/grpc/server/GrpcSeverableSuite.scala create mode 100644 kyuubi-grpc/src/test/scala/org/apache/kyuubi/grpc/server/SimpleGrpcServer.scala create mode 100644 kyuubi-grpc/src/test/scala/org/apache/kyuubi/grpc/service/SimpleGrpcBackendService.scala create mode 100644 kyuubi-grpc/src/test/scala/org/apache/kyuubi/grpc/service/SimpleGrpcFrontendService.scala create mode 100644 kyuubi-grpc/src/test/scala/org/apache/kyuubi/grpc/service/SimpleGrpcSeverable.scala create mode 100644 kyuubi-grpc/src/test/scala/org/apache/kyuubi/grpc/session/SimpleGrpcSessionImpl.scala create mode 100644 kyuubi-grpc/src/test/scala/org/apache/kyuubi/grpc/session/SimpleGrpcSessionManager.scala 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 344806dc943..a3b2a96ba22 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 @@ -2731,6 +2731,28 @@ object KyuubiConf { .stringConf .createWithDefault("ENGINE") + val ENGINE_SPARK_CONNECT_GRPC_BINDING_PORT: ConfigEntry[Int] = + buildConf("kyuubi.engine.spark.connect.grpc.bind.port") + .doc("The port is used in spark connect frontendService start GrpcServer") + .version("1.9.0") + .intConf + .createWithDefault(15002) + + val ENGINE_SPARK_CONNECT_GRPC_MAX_INBOUND_MESSAGE_SIZE: ConfigEntry[Int] = + buildConf("kyuubi.engine.spark.connect.grpc.max.inbound.message.size") + .doc("Sets the maximum inbound message in bytes size for the gRPC requests." + + "Requests with a larger payload will fail.") + .version("1.9.0") + .intConf + .createWithDefault(128 * 1024 * 1024) + + val ENGINE_SPARK_CONNECT_GRPC_BINDING_HOST: ConfigEntry[Option[String]] = + buildConf("kyuubi.engine.spark.connect.grpc.bind.host") + .doc("Hostname or IP of the machine on which to run the grpc server in frontend service ") + .version("1.9.0") + .serverOnly + .fallbackConf(FRONTEND_BIND_HOST) + val ENGINE_SPARK_SHOW_PROGRESS: ConfigEntry[Boolean] = buildConf("kyuubi.session.engine.spark.showProgress") .doc("When true, show the progress bar in the Spark's engine log.") diff --git a/kyuubi-grpc/pom.xml b/kyuubi-grpc/pom.xml new file mode 100644 index 00000000000..43f73cb1499 --- /dev/null +++ b/kyuubi-grpc/pom.xml @@ -0,0 +1,392 @@ + + + 4.0.0 + + org.apache.kyuubi + kyuubi-parent + 1.9.0-SNAPSHOT + ../pom.xml + + + kyuubi-grpc_${scala.binary.version} + jar + Kyuubi Project Grpc + https://kyuubi.apache.org/ + + + + org.apache.kyuubi + kyuubi-common_${scala.binary.version} + ${project.version} + + + + org.apache.kyuubi + kyuubi-ha_${scala.binary.version} + ${project.version} + + + + io.grpc + grpc-core + + + + io.grpc + grpc-protobuf + + + io.grpc + grpc-util + + + + io.grpc + grpc-stub + + + + com.google.protobuf + protobuf-java + ${protobuf.version} + compile + + + + org.scala-lang + scala-compiler + provided + + + + org.scala-lang + scala-reflect + provided + + + + org.apache.kyuubi + kyuubi-common_${scala.binary.version} + ${project.version} + test-jar + test + + + + org.scala-lang.modules + scala-collection-compat_${scala.binary.version} + + + + commons-collections + commons-collections + test + + + + commons-io + commons-io + test + + + + com.dimafeng + testcontainers-scala-scalatest_${scala.binary.version} + test + + + + + + + + org.codehaus.mojo + build-helper-maven-plugin + + + add-scala-sources + + add-source + + generate-sources + + + src/main/scala-${scala.binary.version} + + + + + add-scala-test-sources + + add-test-source + + generate-test-sources + + + src/test/scala-${scala.binary.version} + + + + + + + org.apache.maven.plugins + maven-shade-plugin + + false + + + com.google.android:annotations + com.google.api.grpc:proto-google-common-protos + com.google.code.gson:gson + com.google.errorprone:error_prone_annotations + com.google.guava:* + com.google.j2objc:j2objc-annotations + com.google.protobuf:* + dev.failsafe:failsafe + io.etcd:* + io.grpc:* + io.netty:* + io.perfmark:perfmark-api + io.vertx:* + org.apache.kyuubi:* + org.checkerframework:checker-qual + org.codehaus.mojo:animal-sniffer-annotations + + + + + *:* + + **/*.proto + META-INF/*.SF + META-INF/*.DSA + META-INF/*.RSA + META-INF/DEPENDENCIES + META-INF/LICENSE.txt + META-INF/NOTICE.txt + META-INF/maven/** + LICENSE.txt + NOTICE.txt + mozilla/** + **/module-info.class + + + + + + dev.failsafe + ${kyuubi.shade.packageName}.dev.failsafe + + dev.failsafe.** + + + + io.etcd + ${kyuubi.shade.packageName}.io.etcd + + io.etcd.** + + + + io.grpc + ${kyuubi.shade.packageName}.io.grpc + + + io.netty + ${kyuubi.shade.packageName}.io.netty + + io.netty.** + + + + io.perfmark + ${kyuubi.shade.packageName}.io.perfmark + + + io.vertx + ${kyuubi.shade.packageName}.io.vertx + + io.vertx.** + + + + android.annotation + ${kyuubi.shade.packageName}.android.annotation + + + com.google.common + ${kyuubi.shade.packageName}.com.google.common + + com.google.common.** + + + + com.google.thirdparty + ${kyuubi.shade.packageName}.com.google.thirdparty + + com.google.thirdparty.** + + + + com.google.protobuf + ${kyuubi.shade.packageName}.com.google.protobuf + + com.google.protobuf.** + + + + org.codehaus.mojo.animal_sniffer + ${kyuubi.shade.packageName}.org.codehaus.mojo.animal_sniffer + + + com.google.j2objc.annotations + ${kyuubi.shade.packageName}.com.google.j2objc.annotations + + + com.google.errorprone.annotations + ${kyuubi.shade.packageName}.com.google.errorprone.annotations + + + org.checkerframework + ${kyuubi.shade.packageName}.org.checkerframework + + + com.google.gson + ${kyuubi.shade.packageName}.com.google.gson + + + + com.google.api + ${kyuubi.shade.packageName}.com.google.api + + + com.google.cloud + ${kyuubi.shade.packageName}.com.google.cloud + + + com.google.geo + ${kyuubi.shade.packageName}.com.google.geo + + + com.google.logging + ${kyuubi.shade.packageName}.com.google.logging + + + com.google.longrunning + ${kyuubi.shade.packageName}.com.google.longrunning + + + com.google.rpc + ${kyuubi.shade.packageName}.com.google.rpc + + + com.google.type + ${kyuubi.shade.packageName}.com.google.type + + + + + + + + + + shade + + package + + + + + + org.apache.maven.plugins + maven-antrun-plugin + + + rename-native-library + + run + + package + + + unpacking netty jar + + renaming netty native libraries + + + + + deleting META-INF/native-image folder + + repackaging netty jar + + + + + + + + + org.apache.maven.plugins + maven-jar-plugin + + + prepare-test-jar + + test-jar + + test-compile + + + + + target/scala-${scala.binary.version}/classes + target/scala-${scala.binary.version}/test-classes + + + kr.motd.maven + os-maven-plugin + 1.6.2 + + + + + + default-protoc + + true + + + + + org.xolstice.maven.plugins + protobuf-maven-plugin + 0.6.1 + + com.google.protobuf:protoc:${protobuf.version}:exe:${os.detected.classifier} + grpc-java + io.grpc:protoc-gen-grpc-java:${grpc.version}:exe:${os.detected.classifier} + src/test/resources/protobuf + + + + + compile + compile-custom + test-compile + + + + + + + + + diff --git a/kyuubi-grpc/src/main/scala/org/apache/kyuubi/grpc/events/OperationEventsManager.scala b/kyuubi-grpc/src/main/scala/org/apache/kyuubi/grpc/events/OperationEventsManager.scala new file mode 100644 index 00000000000..507b944c3be --- /dev/null +++ b/kyuubi-grpc/src/main/scala/org/apache/kyuubi/grpc/events/OperationEventsManager.scala @@ -0,0 +1,141 @@ +/* + * 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.grpc.events + +import org.apache.kyuubi.grpc.operation.GrpcOperation +import org.apache.kyuubi.grpc.session.GrpcSession +import org.apache.kyuubi.grpc.utils.Clock + +object OperationEventsManager { + // TODO: make this configurable + val MAX_STATEMENT_TEXT_SIZE = 65535 +} + +sealed abstract class OperationStatus(value: Int) + +object OperationStatus { + case object Pending extends OperationStatus(0) + case object Started extends OperationStatus(1) + case object Analyzed extends OperationStatus(2) + case object ReadyForExecution extends OperationStatus(3) + case object Finished extends OperationStatus(4) + case object Failed extends OperationStatus(5) + case object Canceled extends OperationStatus(6) + case object Closed extends OperationStatus(7) +} +abstract class OperationEventsManager(operation: GrpcOperation, clock: Clock) { + private def operationId: String = operation.operationKey.operationId + + private def session: GrpcSession = operation.grpcSession + + private def sessionId: String = session.sessionKey.sessionId + + private def sessionStatus = session.sessionEventsManager.status + + protected var _status: OperationStatus = OperationStatus.Pending + + private var error = Option.empty[Boolean] + + private var canceled = Option.empty[Boolean] + + private var producedRowCount = Option.empty[Long] + + private def status: OperationStatus = _status + + private def hasCanceled: Option[Boolean] = canceled + + private def hasError: Option[Boolean] = error + + private def getProduceRowCount: Option[Long] = producedRowCount + + def postStarted(): Unit = { + assertStatus(List(OperationStatus.Pending), OperationStatus.Started) + } + + def postAnalyzed(analyzedPlan: Option[Any] = None): Unit = { + assertStatus(List(OperationStatus.Started, OperationStatus.Analyzed), OperationStatus.Analyzed) + } + + def postReadyForExecution(): Unit = { + assertStatus(List(OperationStatus.Analyzed), OperationStatus.ReadyForExecution) + } + + def postCanceled(): Unit = { + assertStatus( + List( + OperationStatus.Started, + OperationStatus.Analyzed, + OperationStatus.ReadyForExecution, + OperationStatus.Finished, + OperationStatus.Failed), + OperationStatus.Canceled) + canceled = Some(true) + } + + def postFailed(errorMessage: String): Unit = { + assertStatus( + List( + OperationStatus.Started, + OperationStatus.Analyzed, + OperationStatus.ReadyForExecution, + OperationStatus.Finished), + OperationStatus.Failed) + error = Some(true) + } + + def postFinished(producedRowCountOpt: Option[Long] = None): Unit = { + assertStatus( + List( + OperationStatus.Started, + OperationStatus.ReadyForExecution), + OperationStatus.Finished) + producedRowCount = producedRowCountOpt + } + + def postClosed(): Unit = { + assertStatus( + List( + OperationStatus.Finished, + OperationStatus.Failed, + OperationStatus.Canceled), + OperationStatus.Closed) + } + + def status_(operationStatus: OperationStatus): Unit = { + _status = operationStatus + } + + private def assertStatus( + validStatuses: List[OperationStatus], + eventStatus: OperationStatus): Unit = { + if (!validStatuses.contains(status)) { + throw new IllegalStateException( + s""" + |operationId: $operationId with status ${status} + |is not within statuses $validStatuses for event $eventStatus + |""".stripMargin) + } +// if (sessionStatus != SessionStatus.Started) { +// throw new IllegalStateException( +// s""" +// |sessionId: $sessionId with status $sessionStatus +// |is not Started for event $eventStatus +// |""".stripMargin) +// } + _status = eventStatus + } +} diff --git a/kyuubi-grpc/src/main/scala/org/apache/kyuubi/grpc/events/SessionEventsManager.scala b/kyuubi-grpc/src/main/scala/org/apache/kyuubi/grpc/events/SessionEventsManager.scala new file mode 100644 index 00000000000..0d174588bd0 --- /dev/null +++ b/kyuubi-grpc/src/main/scala/org/apache/kyuubi/grpc/events/SessionEventsManager.scala @@ -0,0 +1,60 @@ +/* + * 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.grpc.events + +import org.apache.kyuubi.grpc.session.GrpcSession +import org.apache.kyuubi.grpc.utils.Clock + +sealed abstract class SessionStatus(value: Int) + +object SessionStatus { + case object Pending extends SessionStatus(0) + case object Started extends SessionStatus(1) + case object Closed extends SessionStatus(2) +} + +abstract class SessionEventsManager(session: GrpcSession, clock: Clock) { + private def sessionId: String = session.sessionKey.sessionId + + private var _status: SessionStatus = SessionStatus.Pending + + protected def status_(sessionStatus: SessionStatus): Unit = { + _status = sessionStatus + } + + def status: SessionStatus = _status + + def postStarted(): Unit = { + assertStatus(List(SessionStatus.Pending), SessionStatus.Started) + status_(SessionStatus.Started) + } + + def postClosed(): Unit = { + assertStatus(List(SessionStatus.Started), SessionStatus.Closed) + status_(SessionStatus.Closed) + } + + private def assertStatus(validStatuses: List[SessionStatus], eventStatus: SessionStatus): Unit = { + if (!validStatuses.contains(status)) { + throw new IllegalStateException( + s""" + |sessionId: $sessionId with status ${status} + |is not within statuses $validStatuses for event $eventStatus + |""".stripMargin) + } + } +} diff --git a/kyuubi-grpc/src/main/scala/org/apache/kyuubi/grpc/operation/AbstractGrpcOperation.scala b/kyuubi-grpc/src/main/scala/org/apache/kyuubi/grpc/operation/AbstractGrpcOperation.scala new file mode 100644 index 00000000000..eee82dde59c --- /dev/null +++ b/kyuubi-grpc/src/main/scala/org/apache/kyuubi/grpc/operation/AbstractGrpcOperation.scala @@ -0,0 +1,78 @@ +/* + * 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.grpc.operation + +import java.util.concurrent.locks.ReentrantLock + +import org.apache.kyuubi.{KyuubiSQLException, Logging, Utils} +import org.apache.kyuubi.grpc.session.GrpcSession + +abstract class AbstractGrpcOperation[S <: GrpcSession](session: S) extends GrpcOperation + with Logging { + final protected val opType: String = getClass.getSimpleName + final protected val createTime = System.currentTimeMillis() + protected def key: OperationKey + final private val operationTimeout: Long = 1000 + private var lock: ReentrantLock = new ReentrantLock() + + protected def withLockRequired[T](block: => T): T = Utils.withLockRequired(lock)(block) + + @volatile protected var startTime: Long = _ + @volatile protected var completedTime: Long = _ + @volatile protected var lastAccessTime: Long = createTime + + @volatile protected var operationException: KyuubiSQLException = _ + + protected def setOperationException(ex: KyuubiSQLException): Unit = { + this.operationException = ex + } + + protected def runInternal(): Unit + + protected def beforeRun(): Unit + + protected def afterRun(): Unit + + override def run(): Unit = { + beforeRun() + try { + runInternal() + } finally { + afterRun() + } + } + + override def close(): Unit + + override def operationKey: OperationKey = key + + override def grpcSession: S = session + +} + +object OperationJobTag { + def apply(prefix: String, operationKey: OperationKey): String = { + s"${prefix}_" + + s"User_${operationKey.userId}_" + + s"Session_${operationKey.sessionId}_" + + s"Operation_${operationKey.operationId}" + } + + def unapply(jobTag: String, prefix: String): Option[String] = { + if (jobTag.startsWith(prefix)) Some(jobTag) else None + } +} diff --git a/kyuubi-grpc/src/main/scala/org/apache/kyuubi/grpc/operation/GrpcOperation.scala b/kyuubi-grpc/src/main/scala/org/apache/kyuubi/grpc/operation/GrpcOperation.scala new file mode 100644 index 00000000000..42a3b18ca5a --- /dev/null +++ b/kyuubi-grpc/src/main/scala/org/apache/kyuubi/grpc/operation/GrpcOperation.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.grpc.operation + +import org.apache.kyuubi.grpc.events.OperationEventsManager +import org.apache.kyuubi.grpc.session.GrpcSession +import org.apache.kyuubi.operation.log.OperationLog + +trait GrpcOperation { + def run(): Unit + def interrupt(): Unit + def close(): Unit + + def getOperationLog: Option[OperationLog] + def isTimedOut: Boolean + def grpcSession: GrpcSession + def operationKey: OperationKey + def operationEventsManager: OperationEventsManager +} diff --git a/kyuubi-grpc/src/main/scala/org/apache/kyuubi/grpc/operation/GrpcOperationManager.scala b/kyuubi-grpc/src/main/scala/org/apache/kyuubi/grpc/operation/GrpcOperationManager.scala new file mode 100644 index 00000000000..12913a97c50 --- /dev/null +++ b/kyuubi-grpc/src/main/scala/org/apache/kyuubi/grpc/operation/GrpcOperationManager.scala @@ -0,0 +1,83 @@ +/* + * 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.grpc.operation + +import java.util.concurrent._ + +import scala.collection.JavaConverters._ + +import org.apache.kyuubi.KyuubiSQLException +import org.apache.kyuubi.config.KyuubiConf +import org.apache.kyuubi.operation.log.LogDivertAppender +import org.apache.kyuubi.service.AbstractService + +/** + * The [[GrpcOperationManager]] manages all the grpc operations during their lifecycle + */ +abstract class GrpcOperationManager(name: String) + extends AbstractService(name) { + + private val keyToOperations = new ConcurrentHashMap[OperationKey, GrpcOperation] + + protected def skipOperationLog: Boolean = false + def getOperationCount: Int = keyToOperations.size() + + def allOperations(): Iterable[GrpcOperation] = keyToOperations.values().asScala + + override def initialize(conf: KyuubiConf): Unit = { + LogDivertAppender.initialize(skipOperationLog) + super.initialize(conf) + } + + def close(operationKey: OperationKey): Unit = { + val operation = keyToOperations.get(operationKey) + if (operation == null) throw KyuubiSQLException(s"Invalid $operationKey") + operation.close() + } + + final def addOperation(grpcOperation: GrpcOperation): GrpcOperation = synchronized { + keyToOperations.put(grpcOperation.operationKey, grpcOperation) + grpcOperation + } + + @throws[KyuubiSQLException] + final def getOperation(operationKey: OperationKey): GrpcOperation = { + val operation = synchronized { keyToOperations.get(operationKey) } + if (operation == null) throw KyuubiSQLException(s"Invalid $operationKey") + operation + } + + @throws[KyuubiSQLException] + final def removeOperation(operationKey: OperationKey): GrpcOperation = synchronized { + val operation = keyToOperations.remove(operationKey) + if (operation == null) throw KyuubiSQLException(s"Invalid $operationKey") + operation + } + + @throws[KyuubiSQLException] + final def closeOperation(operationKey: OperationKey): Unit = { + val operation = removeOperation(operationKey) + operation.close() + } + + @throws[KyuubiSQLException] + final def interruptOperation(operationKey: OperationKey): Unit = { + val operation = getOperation(operationKey) + operation.interrupt() + } + +} diff --git a/kyuubi-grpc/src/main/scala/org/apache/kyuubi/grpc/operation/OperationKey.scala b/kyuubi-grpc/src/main/scala/org/apache/kyuubi/grpc/operation/OperationKey.scala new file mode 100644 index 00000000000..badf83de361 --- /dev/null +++ b/kyuubi-grpc/src/main/scala/org/apache/kyuubi/grpc/operation/OperationKey.scala @@ -0,0 +1,32 @@ +/* + * 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.grpc.operation + +import java.util.UUID + +import org.apache.kyuubi.grpc.session.SessionKey + +case class OperationKey(userId: String, sessionId: String, operationId: String) { + override def toString: String = + s"Session: [{$userId}_{$sessionId}], OperationId: [$operationId]" +} + +object OperationKey { + def apply(key: SessionKey): OperationKey = + new OperationKey(key.userId, key.sessionId, UUID.randomUUID().toString) + +} diff --git a/kyuubi-grpc/src/main/scala/org/apache/kyuubi/grpc/service/AbstractGrpcBackendService.scala b/kyuubi-grpc/src/main/scala/org/apache/kyuubi/grpc/service/AbstractGrpcBackendService.scala new file mode 100644 index 00000000000..f7e29627caf --- /dev/null +++ b/kyuubi-grpc/src/main/scala/org/apache/kyuubi/grpc/service/AbstractGrpcBackendService.scala @@ -0,0 +1,29 @@ +/* + * 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.grpc.service + +import org.apache.kyuubi.config.KyuubiConf +import org.apache.kyuubi.service.CompositeService + +abstract class AbstractGrpcBackendService(name: String) + extends CompositeService(name) with GrpcBackendService { + + override def initialize(conf: KyuubiConf): Unit = { + addService(grpcSessionManager) + super.initialize(conf) + } +} diff --git a/kyuubi-grpc/src/main/scala/org/apache/kyuubi/grpc/service/AbstractGrpcFrontendService.scala b/kyuubi-grpc/src/main/scala/org/apache/kyuubi/grpc/service/AbstractGrpcFrontendService.scala new file mode 100644 index 00000000000..4328eab1d62 --- /dev/null +++ b/kyuubi-grpc/src/main/scala/org/apache/kyuubi/grpc/service/AbstractGrpcFrontendService.scala @@ -0,0 +1,153 @@ +/* + * 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.grpc.service + +import java.net.{InetAddress, InetSocketAddress} +import java.util.concurrent.TimeUnit +import java.util.concurrent.atomic.AtomicBoolean + +import com.google.protobuf.MessageLite +import io.grpc._ +import io.grpc.MethodDescriptor.PrototypeMarshaller +import io.grpc.netty.NettyServerBuilder +import io.grpc.protobuf.lite.ProtoLiteUtils + +import org.apache.kyuubi.{KyuubiException, Logging, Utils} +import org.apache.kyuubi.config.KyuubiConf +import org.apache.kyuubi.config.KyuubiConf.{ENGINE_SPARK_CONNECT_GRPC_BINDING_PORT, FRONTEND_ADVERTISED_HOST} +import org.apache.kyuubi.service.CompositeService +import org.apache.kyuubi.util.NamedThreadFactory + +abstract class AbstractGrpcFrontendService(name: String) + extends CompositeService(name) with GrpcFrontendService with Runnable + with BindableService with Logging { + + private val started = new AtomicBoolean(false) + protected var server: Server = _ + protected def portNum: Int = conf.get(ENGINE_SPARK_CONNECT_GRPC_BINDING_PORT) + protected def maxInboundMessageSize: Int = 1024 + + protected def serverHost: Option[String] + protected lazy val serverAddr: InetAddress = + serverHost.map(InetAddress.getByName).getOrElse(Utils.findLocalInetAddress) + + private lazy val serverThread = new NamedThreadFactory(getName, false).newThread(this) + + override def initialize(conf: KyuubiConf): Unit = { + this.conf = conf + try { + val socketAddress = new InetSocketAddress(serverAddr.getHostName, portNum) + val nettyServerBuilder = NettyServerBuilder + .forAddress(socketAddress) + .maxInboundMessageSize(maxInboundMessageSize) + .addService(this) + server = nettyServerBuilder.build() + } catch { + case e: Throwable => + error(e) + throw new KyuubiException( + s"Failed to initialize grpc frontend service on $portNum", + e) + } + super.initialize(conf) + } + + override def bindService(): ServerServiceDefinition + + protected def methodWithCustomMarshallers(methodDesc: MethodDescriptor[MessageLite, MessageLite]) + : MethodDescriptor[MessageLite, MessageLite] = { + // default 1024 + val recursionLimit = 1024 + val requestMarshaller = + ProtoLiteUtils.marshallerWithRecursionLimit( + methodDesc.getRequestMarshaller + .asInstanceOf[PrototypeMarshaller[MessageLite]] + .getMessagePrototype, + recursionLimit) + val responseMarshaller = + ProtoLiteUtils.marshallerWithRecursionLimit( + methodDesc.getResponseMarshaller + .asInstanceOf[PrototypeMarshaller[MessageLite]] + .getMessagePrototype, + recursionLimit) + methodDesc.toBuilder + .setRequestMarshaller(requestMarshaller) + .setResponseMarshaller(responseMarshaller) + .build() + } + + override def start(): Unit = { + try { + if (started.compareAndSet(false, true)) { + serverThread.start() + } + super.start() + } catch { + case e: Throwable => + stopInternal() + throw e + } + } + + private def stopInternal(): Unit = { + if (started.compareAndSet(true, false)) { + serverThread.interrupt() + stopServer(Some(10L), Some(TimeUnit.SECONDS)) + info(getName + " has stoppped") + } + } + + override def stop(): Unit = { + super.stop() + stopInternal() + } + + def stopServer(timeout: Option[Long] = None, unit: Option[TimeUnit] = None): Unit = { + if (server != null) { + if (timeout.isDefined && unit.isDefined) { + server.shutdown() + server.awaitTermination(timeout.get, unit.get) + } else { + server.shutdown() + } + } + } + + override def run(): Unit = { + try { + server.start() + info("Grpc Server Start Success") + } catch { + case _: InterruptedException => error(s"$getName is interrupted") + case t: Throwable => + error(s"Error starting $getName", t) + System.exit(-1) + } + } + + override def connectionUrl: String = { + val host = (conf.get(FRONTEND_ADVERTISED_HOST), serverHost) match { + case (Some(advertisedHost), _) => advertisedHost + case (None, Some(h)) => h + case (None, None) => serverAddr.getHostAddress + } + host + ":" + portNum + } + + protected def isServer(): Boolean = false + +} diff --git a/kyuubi-grpc/src/main/scala/org/apache/kyuubi/grpc/service/GrpcBackendService.scala b/kyuubi-grpc/src/main/scala/org/apache/kyuubi/grpc/service/GrpcBackendService.scala new file mode 100644 index 00000000000..59220edd39f --- /dev/null +++ b/kyuubi-grpc/src/main/scala/org/apache/kyuubi/grpc/service/GrpcBackendService.scala @@ -0,0 +1,24 @@ +/* + * 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.grpc.service + +import org.apache.kyuubi.grpc.session.{GrpcSession, GrpcSessionManager} + +trait GrpcBackendService { + + def grpcSessionManager: GrpcSessionManager[_ <: GrpcSession] +} diff --git a/kyuubi-grpc/src/main/scala/org/apache/kyuubi/grpc/service/GrpcFrontendService.scala b/kyuubi-grpc/src/main/scala/org/apache/kyuubi/grpc/service/GrpcFrontendService.scala new file mode 100644 index 00000000000..d7a11bc2769 --- /dev/null +++ b/kyuubi-grpc/src/main/scala/org/apache/kyuubi/grpc/service/GrpcFrontendService.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.grpc.service + +import org.apache.kyuubi.service.Service + +trait GrpcFrontendService { + + def connectionUrl: String + + val serverable: GrpcSeverable[_ <: GrpcBackendService, _ <: GrpcFrontendService] + + final def be: GrpcBackendService = serverable.backendService + + val discoveryService: Option[Service] + + def attributes: Map[String, String] = Map.empty + +} diff --git a/kyuubi-grpc/src/main/scala/org/apache/kyuubi/grpc/service/GrpcSeverable.scala b/kyuubi-grpc/src/main/scala/org/apache/kyuubi/grpc/service/GrpcSeverable.scala new file mode 100644 index 00000000000..0d5dcfd0bb7 --- /dev/null +++ b/kyuubi-grpc/src/main/scala/org/apache/kyuubi/grpc/service/GrpcSeverable.scala @@ -0,0 +1,66 @@ +/* + * 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.grpc.service + +import java.util.concurrent.atomic.AtomicBoolean + +import org.apache.kyuubi.config.KyuubiConf +import org.apache.kyuubi.service.CompositeService + +abstract class GrpcSeverable[B <: AbstractGrpcBackendService, F <: AbstractGrpcFrontendService]( + name: String) extends CompositeService(name) { + private val started = new AtomicBoolean(false) + + var selfExited = false + + val backendService: B + + val frontendServices: Seq[F] + + override def initialize(conf: KyuubiConf): Unit = synchronized { + this.conf = conf + addService(backendService) + frontendServices.foreach(addService) + super.initialize(conf) + } + + override def start(): Unit = synchronized { + if (!started.getAndSet(true)) { + super.start() + } + } + + protected def stopServer(): Unit + + override def stop(): Unit = synchronized { + try { + if (started.getAndSet(false)) { + super.stop() + } + } catch { + case t: Throwable => + warn(s"Error stopping $name ${t.getMessage}", t) + } finally { + try { + stopServer() + } catch { + case t: Throwable => + warn(s"Error stopping $name ${t.getMessage}", t) + } + } + } +} diff --git a/kyuubi-grpc/src/main/scala/org/apache/kyuubi/grpc/session/AbstractGrpcSession.scala b/kyuubi-grpc/src/main/scala/org/apache/kyuubi/grpc/session/AbstractGrpcSession.scala new file mode 100644 index 00000000000..485d0c01d2b --- /dev/null +++ b/kyuubi-grpc/src/main/scala/org/apache/kyuubi/grpc/session/AbstractGrpcSession.scala @@ -0,0 +1,97 @@ +/* + * 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.grpc.session + +import java.util + +import org.apache.kyuubi.{KyuubiSQLException, Logging} +import org.apache.kyuubi.grpc.operation.{GrpcOperation, OperationKey} +import org.apache.kyuubi.grpc.utils.ProtoUtils + +abstract class AbstractGrpcSession( + val userId: String) extends GrpcSession with Logging { + override val sessionKey: SessionKey = SessionKey(userId) + + final private val _createTime: Long = System.currentTimeMillis() + override def createTime: Long = _createTime + + @volatile private var _lastAccessTime: Long = _createTime + override def lastAccessTime: Long = _lastAccessTime + + @volatile private var closedTimeMs: Option[Long] = None + + @volatile private var _lastIdleTime: Long = _createTime + override def lastIdleTime: Long = _lastIdleTime + + final private val opKeySet = new util.HashSet[OperationKey] + + protected def runGrpcOperation(operation: GrpcOperation): OperationKey = { + if (closedTimeMs.isDefined) { + throw KyuubiSQLException("Cannot build operation because the session is closed") + } + try { + val opKey = operation.operationKey + opKeySet.add(opKey) + operation.run() + opKey + } catch { + case e: KyuubiSQLException => + opKeySet.remove(operation.operationKey) + sessionManager.grpcOperationManager.close(operation.operationKey) + throw e + } + } + + override def removeOperation(operationKey: OperationKey): Unit = { + sessionManager.grpcOperationManager.removeOperation(operationKey) + } + + override def closeOperation(operationKey: OperationKey): Unit = { + sessionManager.grpcOperationManager.closeOperation(operationKey) + } + + override def interruptOperation(operationKey: OperationKey): Unit = { + sessionManager.grpcOperationManager.interruptOperation(operationKey) + } + + override def open(): Unit = { + sessionEventsManager.postStarted() + } + + override def close(): Unit = { + if (closedTimeMs.isDefined) { + throw KyuubiSQLException(s"Session ${sessionKey.sessionId} is already closed.") + } + closedTimeMs = Some(System.currentTimeMillis()) + sessionEventsManager.postClosed() + } + +} + +object SessionTag { + def apply(sessionKey: SessionKey, tag: String, prefix: String): String = { + ProtoUtils.throwIfInvalidTag(tag) + s"${prefix}_" + + s"User_${sessionKey.userId}_" + + s"Session_${sessionKey.sessionId}_" + + s"Tag_${tag}" + } + + def unapply(sessionTag: String, prefix: String): Option[String] = { + if (sessionTag.startsWith(prefix)) Some(sessionTag) else None + } +} diff --git a/kyuubi-grpc/src/main/scala/org/apache/kyuubi/grpc/session/GrpcSession.scala b/kyuubi-grpc/src/main/scala/org/apache/kyuubi/grpc/session/GrpcSession.scala new file mode 100644 index 00000000000..d0a8684448b --- /dev/null +++ b/kyuubi-grpc/src/main/scala/org/apache/kyuubi/grpc/session/GrpcSession.scala @@ -0,0 +1,46 @@ +/* + * 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.grpc.session + +import org.apache.kyuubi.grpc.events.SessionEventsManager +import org.apache.kyuubi.grpc.operation.{GrpcOperation, OperationKey} + +trait GrpcSession { + def sessionKey: SessionKey + def name: Option[String] + + def serverSessionId: String + + def createTime: Long + def lastAccessTime: Long + def lastIdleTime: Long + + def sessionManager: GrpcSessionManager[_ <: GrpcSession] + + def sessionEventsManager: SessionEventsManager + + def open() + def close() + + def removeOperation(operationKey: OperationKey): Unit + + def getOperation(operationKey: OperationKey): GrpcOperation + def closeOperation(operationKey: OperationKey): Unit + + def interruptOperation(operationKey: OperationKey): Unit + +} diff --git a/kyuubi-grpc/src/main/scala/org/apache/kyuubi/grpc/session/GrpcSessionManager.scala b/kyuubi-grpc/src/main/scala/org/apache/kyuubi/grpc/session/GrpcSessionManager.scala new file mode 100644 index 00000000000..1a4b4f6c324 --- /dev/null +++ b/kyuubi-grpc/src/main/scala/org/apache/kyuubi/grpc/session/GrpcSessionManager.scala @@ -0,0 +1,214 @@ +/* + * 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.grpc.session + +import java.io.IOException +import java.nio.file.{Files, Paths} +import java.util.concurrent._ +import scala.concurrent.duration.Duration +import org.apache.kyuubi.{KyuubiSQLException, Utils} +import org.apache.kyuubi.config.KyuubiConf +import org.apache.kyuubi.config.KyuubiConf._ +import org.apache.kyuubi.grpc.operation.GrpcOperationManager +import org.apache.kyuubi.service.CompositeService +import org.apache.kyuubi.util.ThreadUtils + +abstract class GrpcSessionManager[S <: AbstractGrpcSession](name: String) + extends CompositeService(name) { + + @volatile private var shutdown = false + + protected var _operationLogRoot: Option[String] = None + + def operationLogRoot: Option[String] = _operationLogRoot + + private def initOperationLogRootDir(): Unit = { + try { + val logRoot = { + if (isServer) { + conf.get(SERVER_OPERATION_LOG_DIR_ROOT) + } else { + conf.get(ENGINE_OPERATION_LOG_DIR_ROOT) + } + } + val logPath = Files.createDirectories(Utils.getAbsolutePathFromWork(logRoot)) + _operationLogRoot = Some(logPath.toString) + } catch { + case e: IOException => + error(s"Failed to initialize operation log root directory: ${_operationLogRoot}", e) + _operationLogRoot = None + } + } + + private val sessionKeyToSession = new ConcurrentHashMap[SessionKey, S] + + @volatile private var _latestLogoutTime: Long = System.currentTimeMillis() + def latestLogoutTime: Long = _latestLogoutTime + + private val timeoutChecker = + ThreadUtils.newDaemonSingleThreadScheduledExecutor(s"$name-timeout-checker") + + protected def isServer: Boolean + + private var execPool: ThreadPoolExecutor = _ + + def grpcOperationManager: GrpcOperationManager + + protected def getOrCreateSession( + key: SessionKey): S + + def getSession(key: SessionKey): S = { + getSessionOption(key).getOrElse(throw KyuubiSQLException(s"Invalid key $key")) + } + + private def getSessionOption(key: SessionKey): Option[S] = { + Option(sessionKeyToSession.get(key)) + } + def openSession( + key: SessionKey): S = { + info(s"Opening grpc session for ${key.userId}") + val session = getOrCreateSession(key) + try { + val key = session.sessionKey + session.open() + setSession(key, session) + logSessionCountInfo(session, "opened") + session + } catch { + case e: Exception => + try { + session.close() + } catch { + case t: Throwable => + warn(s"Error closing session for ${key.userId}", t) + } + throw KyuubiSQLException(e) + } + } + + protected def removeSession(key: SessionKey): Option[S] = { + val session = sessionKeyToSession.remove(key) + Some(session) + } + + protected def shutdownSession(session: S): Unit = { + session.close() + } + + protected def closeSession(key: SessionKey): Unit = { + _latestLogoutTime = System.currentTimeMillis() + val session = sessionKeyToSession.remove(key) + if (session == null) { + throw KyuubiSQLException(s"Invalid $key") + } + logSessionCountInfo(session, "closed") + try { + session.close() + } finally { + deleteOperationLogSessionDir(key) + } + } + + private def deleteOperationLogSessionDir(key: SessionKey): Unit = { + _operationLogRoot.foreach(logRoot => { + val rootPath = Paths.get(logRoot, key.toString) + try { + Utils.deleteDirectoryRecursively(rootPath.toFile) + } catch { + case e: IOException => + error(s"Failed to delete session operation log directory ${rootPath.toString}", e) + } + }) + } + + final protected def setSession(key: SessionKey, session: S): Unit = { + sessionKeyToSession.put(key, session) + } + + protected def logSessionCountInfo(session: S, action: String): Unit = { + info(s"${session.sessionKey.userId}'s ${session.getClass.getSimpleName} with" + + s" ${session.sessionKey.sessionId}${session.name.map("/" + _).getOrElse("")} is $action," + + s" current opening sessions $getOpenSessionCount") + } + + def getOpenSessionCount: Int = sessionKeyToSession.size() + + def getExecPoolSize: Int = { + assert(execPool != null) + execPool.getPoolSize + } + + def getActiveCount: Int = { + assert(execPool != null) + execPool.getActiveCount + } + + def getWorkQueueSize: Int = { + assert(execPool != null) + execPool.getQueue.size() + } + + override def initialize(conf: KyuubiConf): Unit = { + this.conf = conf + addService(grpcOperationManager) + initOperationLogRootDir() + + val poolSize: Int = { + if (isServer) { + conf.get(SERVER_EXEC_POOL_SIZE) + } else { + conf.get(ENGINE_EXEC_POOL_SIZE) + } + } + + val waitQueueSize: Int = { + if (isServer) { + conf.get(SERVER_EXEC_WAIT_QUEUE_SIZE) + } else { + conf.get(ENGINE_EXEC_WAIT_QUEUE_SIZE) + } + } + val keepAliveMs: Long = { + if (isServer) { + conf.get(SERVER_EXEC_KEEPALIVE_TIME) + } else { + conf.get(ENGINE_EXEC_KEEPALIVE_TIME) + } + } + + execPool = ThreadUtils.newDaemonQueuedThreadPool( + poolSize, + waitQueueSize, + keepAliveMs, + s"$name-exec-pool") + super.initialize(conf) + } + + override def stop(): Unit = synchronized { + super.stop() + shutdown = true + val shutdownTimeout: Long = { + if (isServer) { + conf.get(SERVER_EXEC_POOL_SHUTDOWN_TIMEOUT) + } else { + conf.get(ENGINE_EXEC_POOL_SHUTDOWN_TIMEOUT) + } + } + + ThreadUtils.shutdown(execPool, Duration(shutdownTimeout, TimeUnit.MILLISECONDS)) + } +} diff --git a/kyuubi-grpc/src/main/scala/org/apache/kyuubi/grpc/session/SessionKey.scala b/kyuubi-grpc/src/main/scala/org/apache/kyuubi/grpc/session/SessionKey.scala new file mode 100644 index 00000000000..9d944275a0d --- /dev/null +++ b/kyuubi-grpc/src/main/scala/org/apache/kyuubi/grpc/session/SessionKey.scala @@ -0,0 +1,27 @@ +/* + * 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.grpc.session + +import java.util.UUID + +case class SessionKey(userId: String, sessionId: String) { + override def toString: String = s"UserId: [$userId], SessionId: [$sessionId]" +} + +object SessionKey { + def apply(userId: String): SessionKey = new SessionKey(userId, UUID.randomUUID().toString) +} diff --git a/kyuubi-grpc/src/main/scala/org/apache/kyuubi/grpc/utils/Clock.scala b/kyuubi-grpc/src/main/scala/org/apache/kyuubi/grpc/utils/Clock.scala new file mode 100644 index 00000000000..7de04f4002e --- /dev/null +++ b/kyuubi-grpc/src/main/scala/org/apache/kyuubi/grpc/utils/Clock.scala @@ -0,0 +1,55 @@ +/* + * 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.grpc.utils + +trait Clock { + def getTimeMillis(): Long + + def nanoTime(): Long + + def waitTillTime(targetTime: Long): Long +} + +class SystemClock extends Clock { + val minPollTime = 25L + + override def getTimeMillis(): Long = System.currentTimeMillis() + + override def nanoTime(): Long = System.nanoTime() + + override def waitTillTime(targetTime: Long): Long = { + var currentTime = System.currentTimeMillis() + + var waitTime = targetTime - currentTime + if (waitTime <= 0) { + return currentTime + } + + val pollTime = math.max(waitTime / 10.0, minPollTime).toLong + + while (true) { + currentTime = System.currentTimeMillis() + waitTime = targetTime - currentTime + if (waitTime <= 0) { + return currentTime + } + val sleepTime = math.min(waitTime, pollTime) + Thread.sleep(sleepTime) + } + -1 + } +} diff --git a/kyuubi-grpc/src/main/scala/org/apache/kyuubi/grpc/utils/ProtoUtils.scala b/kyuubi-grpc/src/main/scala/org/apache/kyuubi/grpc/utils/ProtoUtils.scala new file mode 100644 index 00000000000..bad28db54b5 --- /dev/null +++ b/kyuubi-grpc/src/main/scala/org/apache/kyuubi/grpc/utils/ProtoUtils.scala @@ -0,0 +1,34 @@ +/* + * 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.grpc.utils + +object ProtoUtils { + + private val JOB_TAGS_SEP = ',' + def throwIfInvalidTag(tag: String): Unit = { + if (tag == null) { + throw new IllegalArgumentException("Tag cannot be null.") + } + if (tag.contains(JOB_TAGS_SEP)) { + throw new IllegalArgumentException( + s"Tag cannot contain '$JOB_TAGS_SEP'.") + } + if (tag.isEmpty) { + throw new IllegalArgumentException("Tag cannot be an empty string.") + } + } +} diff --git a/kyuubi-grpc/src/test/resources/protobuf/test_case.proto b/kyuubi-grpc/src/test/resources/protobuf/test_case.proto new file mode 100644 index 00000000000..61f02925009 --- /dev/null +++ b/kyuubi-grpc/src/test/resources/protobuf/test_case.proto @@ -0,0 +1,57 @@ +/* + * 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. + */ + +syntax = 'proto3'; + +package kyuubi.grpc; + +option java_multiple_files = true; +option java_package = "org.apache.kyuubi.grpc.proto"; +option go_package = "internal/generated"; + +message TestOpenSessionRequest { + string user_id = 1; + string session_id = 2; +} + +message TestAddRequest { + string user_id = 1; + string session_id = 2; + string operation_id = 3; + int64 first_num = 4; + int64 second_num = 5; +} + +message TestOpenSessionResponse { + string session_id = 1; + string server_side_session_id = 2; + string session_info = 3; +} + +message TestAddResponse { + string session_id = 1; + string server_side_session_id = 2; + string operation_id = 3; + int64 result = 4; +} + +service GrpcTestService { + rpc TestOpenSession(TestOpenSessionRequest) returns (TestOpenSessionResponse) {} + + rpc TestAdd(TestAddRequest) returns (TestAddResponse) {} + +} \ No newline at end of file diff --git a/kyuubi-grpc/src/test/scala/org/apache/kyuubi/grpc/client/SimpleRpcClient.scala b/kyuubi-grpc/src/test/scala/org/apache/kyuubi/grpc/client/SimpleRpcClient.scala new file mode 100644 index 00000000000..25e412be36b --- /dev/null +++ b/kyuubi-grpc/src/test/scala/org/apache/kyuubi/grpc/client/SimpleRpcClient.scala @@ -0,0 +1,48 @@ +/* + * 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.grpc.client + +import java.util.UUID + +import io.grpc.ManagedChannel + +import org.apache.kyuubi.grpc.proto.{GrpcTestServiceGrpc, TestAddRequest, TestAddResponse, TestOpenSessionRequest, TestOpenSessionResponse} + +class SimpleRpcClient(val channel: ManagedChannel) { + private val DEFAULT_USER_ID = "kyuubi_grpc_test" + private val sessionId: String = UUID.randomUUID().toString + private val stub = GrpcTestServiceGrpc.newBlockingStub(channel) + + def openSession(): TestOpenSessionResponse = { + val request = TestOpenSessionRequest.newBuilder() + .setUserId(DEFAULT_USER_ID) + .setSessionId(sessionId) + .build() + stub.testOpenSession(request) + } + + def testAdd(firstNum: Int, secondNum: Int): TestAddResponse = { + val request = TestAddRequest.newBuilder() + .setUserId(DEFAULT_USER_ID) + .setSessionId(sessionId) + .setFirstNum(firstNum) + .setSecondNum(secondNum) + .build() + stub.testAdd(request) + } + +} diff --git a/kyuubi-grpc/src/test/scala/org/apache/kyuubi/grpc/event/SimpleOperationEventsManager.scala b/kyuubi-grpc/src/test/scala/org/apache/kyuubi/grpc/event/SimpleOperationEventsManager.scala new file mode 100644 index 00000000000..8c47cbbc02d --- /dev/null +++ b/kyuubi-grpc/src/test/scala/org/apache/kyuubi/grpc/event/SimpleOperationEventsManager.scala @@ -0,0 +1,36 @@ +/* + * 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.grpc.event + +import org.apache.kyuubi.Logging +import org.apache.kyuubi.grpc.events.OperationEventsManager +import org.apache.kyuubi.grpc.operation.GrpcOperation +import org.apache.kyuubi.grpc.utils.Clock + +class SimpleOperationEventsManager(operation: GrpcOperation, clock: Clock) + extends OperationEventsManager(operation, clock) with Logging { + + override def postStarted(): Unit = { + super.postStarted() + info("Operation Event: post Started") + } + + override def postClosed(): Unit = { + info("Operation Event: post Closed") + super.postClosed() + } +} diff --git a/kyuubi-grpc/src/test/scala/org/apache/kyuubi/grpc/event/SimpleSessionEventsManager.scala b/kyuubi-grpc/src/test/scala/org/apache/kyuubi/grpc/event/SimpleSessionEventsManager.scala new file mode 100644 index 00000000000..9a657d10aa0 --- /dev/null +++ b/kyuubi-grpc/src/test/scala/org/apache/kyuubi/grpc/event/SimpleSessionEventsManager.scala @@ -0,0 +1,36 @@ +/* + * 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.grpc.event + +import org.apache.kyuubi.Logging +import org.apache.kyuubi.grpc.events.SessionEventsManager +import org.apache.kyuubi.grpc.session.SimpleGrpcSessionImpl +import org.apache.kyuubi.grpc.utils.Clock + +class SimpleSessionEventsManager(session: SimpleGrpcSessionImpl, clock: Clock) + extends SessionEventsManager(session, clock) with Logging { + + override def postStarted(): Unit = { + super.postStarted() + info("Session Event: post Started") + } + + override def postClosed(): Unit = { + info("Session Event: post Closed") + super.postClosed() + } +} diff --git a/kyuubi-grpc/src/test/scala/org/apache/kyuubi/grpc/operation/SimpleAddOperationImpl.scala b/kyuubi-grpc/src/test/scala/org/apache/kyuubi/grpc/operation/SimpleAddOperationImpl.scala new file mode 100644 index 00000000000..2f9599b7968 --- /dev/null +++ b/kyuubi-grpc/src/test/scala/org/apache/kyuubi/grpc/operation/SimpleAddOperationImpl.scala @@ -0,0 +1,45 @@ +/* + * 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.grpc.operation + +import io.grpc.stub.StreamObserver + +import org.apache.kyuubi.grpc.proto.{TestAddRequest, TestAddResponse} +import org.apache.kyuubi.grpc.session.SimpleGrpcSessionImpl + +class SimpleAddOperationImpl( + grpcSession: SimpleGrpcSessionImpl, + shouldFail: Boolean, + request: TestAddRequest, + responseObserver: StreamObserver[TestAddResponse]) + extends SimpleGrpcOperationImpl(grpcSession, shouldFail) { + + override protected def key: OperationKey = OperationKey(grpcSession.sessionKey) + + override def runInternal(): Unit = { + super.runInternal() + val result = request.getFirstNum + request.getSecondNum + val builder = TestAddResponse.newBuilder() + .setOperationId(operationKey.operationId) + .setSessionId(operationKey.sessionId) + .setServerSideSessionId(grpcSession.serverSessionId) + .setResult(result) + responseObserver.onNext(builder.build()) + responseObserver.onCompleted() + } + +} diff --git a/kyuubi-grpc/src/test/scala/org/apache/kyuubi/grpc/operation/SimpleGrpcOperationImpl.scala b/kyuubi-grpc/src/test/scala/org/apache/kyuubi/grpc/operation/SimpleGrpcOperationImpl.scala new file mode 100644 index 00000000000..12500f63529 --- /dev/null +++ b/kyuubi-grpc/src/test/scala/org/apache/kyuubi/grpc/operation/SimpleGrpcOperationImpl.scala @@ -0,0 +1,62 @@ +/* + * 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.grpc.operation + +import org.apache.kyuubi.{KyuubiSQLException, Logging} +import org.apache.kyuubi.grpc.event.SimpleOperationEventsManager +import org.apache.kyuubi.grpc.events.OperationEventsManager +import org.apache.kyuubi.grpc.session.SimpleGrpcSessionImpl +import org.apache.kyuubi.grpc.utils.SystemClock +import org.apache.kyuubi.operation.log.OperationLog + +class SimpleGrpcOperationImpl( + grpcSession: SimpleGrpcSessionImpl, + shouldFail: Boolean = false) + extends AbstractGrpcOperation[SimpleGrpcSessionImpl](grpcSession) with Logging { + + override def operationEventsManager: OperationEventsManager = + new SimpleOperationEventsManager(this, new SystemClock()) + + override def runInternal(): Unit = { + if (shouldFail) { + val exception = KyuubiSQLException("noop operation err") + setOperationException(exception) + operationEventsManager.postFailed(exception.getMessage) + } + } + + override def beforeRun(): Unit = { + operationEventsManager.postStarted() + } + + override def close(): Unit = { + operationEventsManager.postClosed() + } + + override def afterRun(): Unit = { + info("afterRun") + } + + override def interrupt(): Unit = { + info("interrupt") + } + + override def isTimedOut: Boolean = false + override def getOperationLog: Option[OperationLog] = None + + override protected def key: OperationKey = OperationKey(grpcSession.sessionKey) +} diff --git a/kyuubi-grpc/src/test/scala/org/apache/kyuubi/grpc/operation/SimpleGrpcOperationManager.scala b/kyuubi-grpc/src/test/scala/org/apache/kyuubi/grpc/operation/SimpleGrpcOperationManager.scala new file mode 100644 index 00000000000..8b69a2d5a64 --- /dev/null +++ b/kyuubi-grpc/src/test/scala/org/apache/kyuubi/grpc/operation/SimpleGrpcOperationManager.scala @@ -0,0 +1,45 @@ +/* + * 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.grpc.operation + +import io.grpc.stub.StreamObserver + +import org.apache.kyuubi.grpc.proto.{TestAddRequest, TestAddResponse, TestOpenSessionRequest, TestOpenSessionResponse} +import org.apache.kyuubi.grpc.session.SimpleGrpcSessionImpl + +class SimpleGrpcOperationManager + extends GrpcOperationManager("SimpleGrpcOperationManager") { + + def newSimpleOpenSessionOperation( + session: SimpleGrpcSessionImpl, + shouldFail: Boolean, + request: TestOpenSessionRequest, + responseObserver: StreamObserver[TestOpenSessionResponse]): GrpcOperation = { + val operation = + new SimpleOpenSessionOperationImpl(session, shouldFail, request, responseObserver) + addOperation(operation) + } + + def newSimpleAddOperation( + session: SimpleGrpcSessionImpl, + shouldFail: Boolean, + request: TestAddRequest, + responseObserver: StreamObserver[TestAddResponse]): GrpcOperation = { + val operation = new SimpleAddOperationImpl(session, shouldFail, request, responseObserver) + addOperation(operation) + } +} diff --git a/kyuubi-grpc/src/test/scala/org/apache/kyuubi/grpc/operation/SimpleOpenSessionOperationImpl.scala b/kyuubi-grpc/src/test/scala/org/apache/kyuubi/grpc/operation/SimpleOpenSessionOperationImpl.scala new file mode 100644 index 00000000000..f40d4cc8471 --- /dev/null +++ b/kyuubi-grpc/src/test/scala/org/apache/kyuubi/grpc/operation/SimpleOpenSessionOperationImpl.scala @@ -0,0 +1,40 @@ +/* + * 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.grpc.operation + +import io.grpc.stub.StreamObserver + +import org.apache.kyuubi.grpc.proto.{TestOpenSessionRequest, TestOpenSessionResponse} +import org.apache.kyuubi.grpc.session.SimpleGrpcSessionImpl + +class SimpleOpenSessionOperationImpl( + grpcSession: SimpleGrpcSessionImpl, + shouldFail: Boolean, + request: TestOpenSessionRequest, + responseObserver: StreamObserver[TestOpenSessionResponse]) + extends SimpleGrpcOperationImpl(grpcSession, shouldFail) { + + override protected def key: OperationKey = OperationKey(grpcSession.sessionKey) + + override def runInternal(): Unit = { + super.runInternal() + val builder = TestOpenSessionResponse.newBuilder() + .setSessionId(grpcSession.sessionKey.sessionId) + responseObserver.onNext(builder.build()) + responseObserver.onCompleted() + } +} diff --git a/kyuubi-grpc/src/test/scala/org/apache/kyuubi/grpc/server/GrpcSeverableSuite.scala b/kyuubi-grpc/src/test/scala/org/apache/kyuubi/grpc/server/GrpcSeverableSuite.scala new file mode 100644 index 00000000000..c85e860e699 --- /dev/null +++ b/kyuubi-grpc/src/test/scala/org/apache/kyuubi/grpc/server/GrpcSeverableSuite.scala @@ -0,0 +1,73 @@ +/* + * 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.grpc.server + +import io.grpc.{Grpc, InsecureChannelCredentials} + +import org.apache.kyuubi.KyuubiFunSuite +import org.apache.kyuubi.config.KyuubiConf +import org.apache.kyuubi.grpc.client.SimpleRpcClient + +class GrpcSeverableSuite extends KyuubiFunSuite { + ignore("GrpcSeverable") { + val severable1 = new SimpleGrpcServer() + val conf = KyuubiConf().set(KyuubiConf.ENGINE_SPARK_CONNECT_GRPC_BINDING_PORT, 0) + severable1.initialize(conf) + assert(severable1.getStartTime === 0) + assert(severable1.getConf === conf) + assert(severable1.frontendServices.head.connectionUrl.nonEmpty) + } + + test("invalid port") { + val conf = KyuubiConf().set(KyuubiConf.ENGINE_SPARK_CONNECT_GRPC_BINDING_PORT, 100000) + val server = new SimpleGrpcServer + server.initialize(conf) + server.start() + } + + test("test openSession") { + val conf = KyuubiConf().set(KyuubiConf.ENGINE_SPARK_CONNECT_GRPC_BINDING_PORT, 10023) + val server = new SimpleGrpcServer + server.initialize(conf) + server.start() + val channel = + Grpc.newChannelBuilderForAddress( + "127.0.0.1", + 10023, + InsecureChannelCredentials.create()).build() + val client = new SimpleRpcClient(channel) + client.openSession() + server.stop() + } + + test("test add") { + val conf = KyuubiConf().set(KyuubiConf.ENGINE_SPARK_CONNECT_GRPC_BINDING_PORT, 10023) + val server = new SimpleGrpcServer + server.initialize(conf) + server.start() + val channel = + Grpc.newChannelBuilderForAddress( + "127.0.0.1", + 10023, + InsecureChannelCredentials.create()).build() + val client = new SimpleRpcClient(channel) + val response = client.testAdd(1, 2) + assert(response.getResult == 3) + server.stop() + } + +} diff --git a/kyuubi-grpc/src/test/scala/org/apache/kyuubi/grpc/server/SimpleGrpcServer.scala b/kyuubi-grpc/src/test/scala/org/apache/kyuubi/grpc/server/SimpleGrpcServer.scala new file mode 100644 index 00000000000..c30236c4441 --- /dev/null +++ b/kyuubi-grpc/src/test/scala/org/apache/kyuubi/grpc/server/SimpleGrpcServer.scala @@ -0,0 +1,21 @@ +/* + * 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.grpc.server + +import org.apache.kyuubi.grpc.service.SimpleGrpcSeverable + +class SimpleGrpcServer extends SimpleGrpcSeverable {} diff --git a/kyuubi-grpc/src/test/scala/org/apache/kyuubi/grpc/service/SimpleGrpcBackendService.scala b/kyuubi-grpc/src/test/scala/org/apache/kyuubi/grpc/service/SimpleGrpcBackendService.scala new file mode 100644 index 00000000000..64f7dd4fedb --- /dev/null +++ b/kyuubi-grpc/src/test/scala/org/apache/kyuubi/grpc/service/SimpleGrpcBackendService.scala @@ -0,0 +1,49 @@ +/* + * 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.grpc.service +import io.grpc.stub.StreamObserver + +import org.apache.kyuubi.KyuubiException +import org.apache.kyuubi.grpc.proto.{TestAddRequest, TestAddResponse, TestOpenSessionRequest, TestOpenSessionResponse} +import org.apache.kyuubi.grpc.session.{SessionKey, SimpleGrpcSessionManager} + +class SimpleGrpcBackendService extends AbstractGrpcBackendService("simpleTest") { + override def grpcSessionManager: SimpleGrpcSessionManager = new SimpleGrpcSessionManager() + + override def start(): Unit = { + if (conf.getOption("kyuubi.test.backend.should.fail").exists(_.toBoolean)) { + throw new KyuubiException("should fail backend") + } + super.start() + } + + def openSessionTesr( + key: SessionKey, + request: TestOpenSessionRequest, + responseObserver: StreamObserver[TestOpenSessionResponse]): Unit = { + grpcSessionManager.openSession(key) + .openSession(key, request, responseObserver) + } + + def add( + key: SessionKey, + request: TestAddRequest, + responseObserver: StreamObserver[TestAddResponse]): Unit = { + grpcSessionManager.openSession(key) + .add(key, request, responseObserver) + } +} diff --git a/kyuubi-grpc/src/test/scala/org/apache/kyuubi/grpc/service/SimpleGrpcFrontendService.scala b/kyuubi-grpc/src/test/scala/org/apache/kyuubi/grpc/service/SimpleGrpcFrontendService.scala new file mode 100644 index 00000000000..9b3fc09f36c --- /dev/null +++ b/kyuubi-grpc/src/test/scala/org/apache/kyuubi/grpc/service/SimpleGrpcFrontendService.scala @@ -0,0 +1,64 @@ +/* + * 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.grpc.service + +import scala.jdk.CollectionConverters.collectionAsScalaIterableConverter + +import com.google.protobuf.MessageLite +import io.grpc.{ServerMethodDefinition, ServerServiceDefinition} +import io.grpc.stub.StreamObserver + +import org.apache.kyuubi.grpc.proto.{GrpcTestServiceGrpc, TestAddRequest, TestAddResponse, TestOpenSessionRequest, TestOpenSessionResponse} +import org.apache.kyuubi.grpc.session.SessionKey +import org.apache.kyuubi.service.Service + +class SimpleGrpcFrontendService(grpcSeverable: SimpleGrpcSeverable) + extends AbstractGrpcFrontendService("SimpleGrpcFrontendService") + with GrpcTestServiceGrpc.AsyncService { + + private def grpcBe = be.asInstanceOf[SimpleGrpcBackendService] + override protected def serverHost: Option[String] = Some("localhost") + + override def bindService(): ServerServiceDefinition = { + val serviceDef = GrpcTestServiceGrpc.bindService(this) + val builder = ServerServiceDefinition.builder(serviceDef.getServiceDescriptor.getName) + serviceDef.getMethods.asScala + .asInstanceOf[Iterable[ServerMethodDefinition[MessageLite, MessageLite]]] + .foreach(method => + builder.addMethod( + methodWithCustomMarshallers(method.getMethodDescriptor), + method.getServerCallHandler)) + builder.build() + } + + override def testAdd( + request: TestAddRequest, + responseObserver: StreamObserver[TestAddResponse]): Unit = { + val key = new SessionKey(request.getUserId, request.getSessionId) + grpcBe.add(key, request, responseObserver) + } + + override def testOpenSession( + request: TestOpenSessionRequest, + responseObserver: StreamObserver[TestOpenSessionResponse]): Unit = { + val key = new SessionKey(request.getUserId, request.getSessionId) + grpcBe.openSessionTesr(key, request, responseObserver) + } + + override val serverable: SimpleGrpcSeverable = grpcSeverable + override val discoveryService: Option[Service] = None +} diff --git a/kyuubi-grpc/src/test/scala/org/apache/kyuubi/grpc/service/SimpleGrpcSeverable.scala b/kyuubi-grpc/src/test/scala/org/apache/kyuubi/grpc/service/SimpleGrpcSeverable.scala new file mode 100644 index 00000000000..51566586b9c --- /dev/null +++ b/kyuubi-grpc/src/test/scala/org/apache/kyuubi/grpc/service/SimpleGrpcSeverable.scala @@ -0,0 +1,30 @@ +/* + * 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.grpc.service + +class SimpleGrpcSeverable + extends GrpcSeverable[SimpleGrpcBackendService, SimpleGrpcFrontendService]( + "SimpleGrpcSeverable") { + + override val backendService: SimpleGrpcBackendService = new SimpleGrpcBackendService + override val frontendServices: Seq[SimpleGrpcFrontendService] = + Seq(new SimpleGrpcFrontendService(this)) + + override protected def stopServer(): Unit = { + info(s"stop") + } +} diff --git a/kyuubi-grpc/src/test/scala/org/apache/kyuubi/grpc/session/SimpleGrpcSessionImpl.scala b/kyuubi-grpc/src/test/scala/org/apache/kyuubi/grpc/session/SimpleGrpcSessionImpl.scala new file mode 100644 index 00000000000..b4b15a16b45 --- /dev/null +++ b/kyuubi-grpc/src/test/scala/org/apache/kyuubi/grpc/session/SimpleGrpcSessionImpl.scala @@ -0,0 +1,67 @@ +/* + * 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.grpc.session + +import scala.util.Random + +import io.grpc.stub.StreamObserver + +import org.apache.kyuubi.grpc.event.SimpleSessionEventsManager +import org.apache.kyuubi.grpc.events.SessionEventsManager +import org.apache.kyuubi.grpc.operation.{GrpcOperation, OperationKey} +import org.apache.kyuubi.grpc.proto.{TestAddRequest, TestAddResponse, TestOpenSessionRequest, TestOpenSessionResponse} +import org.apache.kyuubi.grpc.utils.SystemClock + +class SimpleGrpcSessionImpl( + userId: String, + sessionManager: SimpleGrpcSessionManager) + extends AbstractGrpcSession(userId) { + override def name: Option[String] = Some("SimpleGrpcSessionImpl") + + override def serverSessionId: String = Random.nextString(10) + + override def sessionEventsManager: SessionEventsManager = + new SimpleSessionEventsManager(this, new SystemClock) + + def openSession( + key: SessionKey, + request: TestOpenSessionRequest, + responseObserver: StreamObserver[TestOpenSessionResponse]): Unit = { + val operation = sessionManager.grpcOperationManager + .newSimpleOpenSessionOperation(this, false, request, responseObserver) + runGrpcOperation(operation) + } + + def add( + key: SessionKey, + request: TestAddRequest, + responseObserver: StreamObserver[TestAddResponse]): Unit = { + val operation = sessionManager.grpcOperationManager + .newSimpleAddOperation(this, false, request, responseObserver) + runGrpcOperation(operation) + } + + override def getOperation(operationKey: OperationKey): GrpcOperation = + sessionManager.grpcOperationManager.getOperation(operationKey) + + override def sessionManager: GrpcSessionManager[SimpleGrpcSessionImpl] = sessionManager + + override def close(): Unit = { + sessionEventsManager.postClosed() + super.close() + } +} diff --git a/kyuubi-grpc/src/test/scala/org/apache/kyuubi/grpc/session/SimpleGrpcSessionManager.scala b/kyuubi-grpc/src/test/scala/org/apache/kyuubi/grpc/session/SimpleGrpcSessionManager.scala new file mode 100644 index 00000000000..4c44af3dc1c --- /dev/null +++ b/kyuubi-grpc/src/test/scala/org/apache/kyuubi/grpc/session/SimpleGrpcSessionManager.scala @@ -0,0 +1,32 @@ +/* + * 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.grpc.session + +import org.apache.kyuubi.grpc.operation.SimpleGrpcOperationManager + +class SimpleGrpcSessionManager + extends GrpcSessionManager[SimpleGrpcSessionImpl]("simpleTest") { + override protected def isServer: Boolean = true + + override def grpcOperationManager: SimpleGrpcOperationManager = new SimpleGrpcOperationManager() + + override def getOrCreateSession(key: SessionKey): SimpleGrpcSessionImpl = { + val session = new SimpleGrpcSessionImpl(key.userId, this) + session + } + +} diff --git a/pom.xml b/pom.xml index 513271d0d3c..dd0266aa8b9 100644 --- a/pom.xml +++ b/pom.xml @@ -83,6 +83,7 @@ kyuubi-util kyuubi-util-scala kyuubi-zookeeper + kyuubi-grpc From 0a52c2ad684bb1907e7aabaa9c02978f725ab6d8 Mon Sep 17 00:00:00 2001 From: davidyuan Date: Thu, 23 May 2024 11:28:30 +0800 Subject: [PATCH 12/21] fork master new version --- kyuubi-grpc/pom.xml | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/kyuubi-grpc/pom.xml b/kyuubi-grpc/pom.xml index 43f73cb1499..48b8768f9a5 100644 --- a/kyuubi-grpc/pom.xml +++ b/kyuubi-grpc/pom.xml @@ -5,7 +5,7 @@ org.apache.kyuubi kyuubi-parent - 1.9.0-SNAPSHOT + 1.10.0-SNAPSHOT ../pom.xml From 4c5a4ae62ff6ba6cb34a58bb95437bfba576db7c Mon Sep 17 00:00:00 2001 From: davidyuan Date: Thu, 23 May 2024 11:33:34 +0800 Subject: [PATCH 13/21] fix format error --- .../kyuubi/grpc/session/GrpcSessionManager.scala | 2 ++ .../kyuubi/grpc/server/GrpcSeverableSuite.scala | 15 --------------- 2 files changed, 2 insertions(+), 15 deletions(-) diff --git a/kyuubi-grpc/src/main/scala/org/apache/kyuubi/grpc/session/GrpcSessionManager.scala b/kyuubi-grpc/src/main/scala/org/apache/kyuubi/grpc/session/GrpcSessionManager.scala index 1a4b4f6c324..d482a1dbfbe 100644 --- a/kyuubi-grpc/src/main/scala/org/apache/kyuubi/grpc/session/GrpcSessionManager.scala +++ b/kyuubi-grpc/src/main/scala/org/apache/kyuubi/grpc/session/GrpcSessionManager.scala @@ -19,7 +19,9 @@ package org.apache.kyuubi.grpc.session import java.io.IOException import java.nio.file.{Files, Paths} import java.util.concurrent._ + import scala.concurrent.duration.Duration + import org.apache.kyuubi.{KyuubiSQLException, Utils} import org.apache.kyuubi.config.KyuubiConf import org.apache.kyuubi.config.KyuubiConf._ diff --git a/kyuubi-grpc/src/test/scala/org/apache/kyuubi/grpc/server/GrpcSeverableSuite.scala b/kyuubi-grpc/src/test/scala/org/apache/kyuubi/grpc/server/GrpcSeverableSuite.scala index c85e860e699..1c751c5de92 100644 --- a/kyuubi-grpc/src/test/scala/org/apache/kyuubi/grpc/server/GrpcSeverableSuite.scala +++ b/kyuubi-grpc/src/test/scala/org/apache/kyuubi/grpc/server/GrpcSeverableSuite.scala @@ -23,21 +23,6 @@ import org.apache.kyuubi.config.KyuubiConf import org.apache.kyuubi.grpc.client.SimpleRpcClient class GrpcSeverableSuite extends KyuubiFunSuite { - ignore("GrpcSeverable") { - val severable1 = new SimpleGrpcServer() - val conf = KyuubiConf().set(KyuubiConf.ENGINE_SPARK_CONNECT_GRPC_BINDING_PORT, 0) - severable1.initialize(conf) - assert(severable1.getStartTime === 0) - assert(severable1.getConf === conf) - assert(severable1.frontendServices.head.connectionUrl.nonEmpty) - } - - test("invalid port") { - val conf = KyuubiConf().set(KyuubiConf.ENGINE_SPARK_CONNECT_GRPC_BINDING_PORT, 100000) - val server = new SimpleGrpcServer - server.initialize(conf) - server.start() - } test("test openSession") { val conf = KyuubiConf().set(KyuubiConf.ENGINE_SPARK_CONNECT_GRPC_BINDING_PORT, 10023) From 3e302aa1ccd2ff26bf8ba414c2a55074baa383cb Mon Sep 17 00:00:00 2001 From: davidyuan Date: Thu, 23 May 2024 11:47:07 +0800 Subject: [PATCH 14/21] fix format error --- kyuubi-grpc/pom.xml | 16 ++++++++++++++++ .../grpc/events/OperationEventsManager.scala | 1 + .../grpc/events/SessionEventsManager.scala | 1 + .../grpc/operation/AbstractGrpcOperation.scala | 1 + .../kyuubi/grpc/operation/GrpcOperation.scala | 1 + .../grpc/operation/GrpcOperationManager.scala | 1 + .../kyuubi/grpc/operation/OperationKey.scala | 1 + .../service/AbstractGrpcBackendService.scala | 1 + .../service/AbstractGrpcFrontendService.scala | 1 + .../kyuubi/grpc/service/GrpcBackendService.scala | 1 + .../grpc/service/GrpcFrontendService.scala | 1 + .../kyuubi/grpc/service/GrpcSeverable.scala | 1 + .../grpc/session/AbstractGrpcSession.scala | 1 + .../apache/kyuubi/grpc/session/GrpcSession.scala | 1 + .../kyuubi/grpc/session/GrpcSessionManager.scala | 1 + .../apache/kyuubi/grpc/session/SessionKey.scala | 1 + .../org/apache/kyuubi/grpc/utils/Clock.scala | 1 + .../apache/kyuubi/grpc/utils/ProtoUtils.scala | 1 + .../kyuubi/grpc/client/SimpleRpcClient.scala | 1 + .../event/SimpleOperationEventsManager.scala | 1 + .../grpc/event/SimpleSessionEventsManager.scala | 1 + .../grpc/operation/SimpleAddOperationImpl.scala | 1 + .../grpc/operation/SimpleGrpcOperationImpl.scala | 1 + .../operation/SimpleGrpcOperationManager.scala | 1 + .../SimpleOpenSessionOperationImpl.scala | 1 + .../kyuubi/grpc/server/GrpcSeverableSuite.scala | 1 + .../kyuubi/grpc/server/SimpleGrpcServer.scala | 1 + .../grpc/service/SimpleGrpcBackendService.scala | 2 ++ .../grpc/service/SimpleGrpcFrontendService.scala | 1 + .../grpc/service/SimpleGrpcSeverable.scala | 1 + .../grpc/session/SimpleGrpcSessionImpl.scala | 1 + .../grpc/session/SimpleGrpcSessionManager.scala | 1 + 32 files changed, 48 insertions(+) diff --git a/kyuubi-grpc/pom.xml b/kyuubi-grpc/pom.xml index 48b8768f9a5..63b220695ff 100644 --- a/kyuubi-grpc/pom.xml +++ b/kyuubi-grpc/pom.xml @@ -1,4 +1,20 @@ + 4.0.0 diff --git a/kyuubi-grpc/src/main/scala/org/apache/kyuubi/grpc/events/OperationEventsManager.scala b/kyuubi-grpc/src/main/scala/org/apache/kyuubi/grpc/events/OperationEventsManager.scala index 507b944c3be..e427dbc20d6 100644 --- a/kyuubi-grpc/src/main/scala/org/apache/kyuubi/grpc/events/OperationEventsManager.scala +++ b/kyuubi-grpc/src/main/scala/org/apache/kyuubi/grpc/events/OperationEventsManager.scala @@ -14,6 +14,7 @@ * See the License for the specific language governing permissions and * limitations under the License. */ + package org.apache.kyuubi.grpc.events import org.apache.kyuubi.grpc.operation.GrpcOperation diff --git a/kyuubi-grpc/src/main/scala/org/apache/kyuubi/grpc/events/SessionEventsManager.scala b/kyuubi-grpc/src/main/scala/org/apache/kyuubi/grpc/events/SessionEventsManager.scala index 0d174588bd0..e182a62c783 100644 --- a/kyuubi-grpc/src/main/scala/org/apache/kyuubi/grpc/events/SessionEventsManager.scala +++ b/kyuubi-grpc/src/main/scala/org/apache/kyuubi/grpc/events/SessionEventsManager.scala @@ -14,6 +14,7 @@ * See the License for the specific language governing permissions and * limitations under the License. */ + package org.apache.kyuubi.grpc.events import org.apache.kyuubi.grpc.session.GrpcSession diff --git a/kyuubi-grpc/src/main/scala/org/apache/kyuubi/grpc/operation/AbstractGrpcOperation.scala b/kyuubi-grpc/src/main/scala/org/apache/kyuubi/grpc/operation/AbstractGrpcOperation.scala index eee82dde59c..3502802e7bc 100644 --- a/kyuubi-grpc/src/main/scala/org/apache/kyuubi/grpc/operation/AbstractGrpcOperation.scala +++ b/kyuubi-grpc/src/main/scala/org/apache/kyuubi/grpc/operation/AbstractGrpcOperation.scala @@ -14,6 +14,7 @@ * See the License for the specific language governing permissions and * limitations under the License. */ + package org.apache.kyuubi.grpc.operation import java.util.concurrent.locks.ReentrantLock diff --git a/kyuubi-grpc/src/main/scala/org/apache/kyuubi/grpc/operation/GrpcOperation.scala b/kyuubi-grpc/src/main/scala/org/apache/kyuubi/grpc/operation/GrpcOperation.scala index 42a3b18ca5a..c0d3715c176 100644 --- a/kyuubi-grpc/src/main/scala/org/apache/kyuubi/grpc/operation/GrpcOperation.scala +++ b/kyuubi-grpc/src/main/scala/org/apache/kyuubi/grpc/operation/GrpcOperation.scala @@ -14,6 +14,7 @@ * See the License for the specific language governing permissions and * limitations under the License. */ + package org.apache.kyuubi.grpc.operation import org.apache.kyuubi.grpc.events.OperationEventsManager diff --git a/kyuubi-grpc/src/main/scala/org/apache/kyuubi/grpc/operation/GrpcOperationManager.scala b/kyuubi-grpc/src/main/scala/org/apache/kyuubi/grpc/operation/GrpcOperationManager.scala index 12913a97c50..964158ee4d9 100644 --- a/kyuubi-grpc/src/main/scala/org/apache/kyuubi/grpc/operation/GrpcOperationManager.scala +++ b/kyuubi-grpc/src/main/scala/org/apache/kyuubi/grpc/operation/GrpcOperationManager.scala @@ -14,6 +14,7 @@ * See the License for the specific language governing permissions and * limitations under the License. */ + package org.apache.kyuubi.grpc.operation import java.util.concurrent._ diff --git a/kyuubi-grpc/src/main/scala/org/apache/kyuubi/grpc/operation/OperationKey.scala b/kyuubi-grpc/src/main/scala/org/apache/kyuubi/grpc/operation/OperationKey.scala index badf83de361..c1cc2e83474 100644 --- a/kyuubi-grpc/src/main/scala/org/apache/kyuubi/grpc/operation/OperationKey.scala +++ b/kyuubi-grpc/src/main/scala/org/apache/kyuubi/grpc/operation/OperationKey.scala @@ -14,6 +14,7 @@ * See the License for the specific language governing permissions and * limitations under the License. */ + package org.apache.kyuubi.grpc.operation import java.util.UUID diff --git a/kyuubi-grpc/src/main/scala/org/apache/kyuubi/grpc/service/AbstractGrpcBackendService.scala b/kyuubi-grpc/src/main/scala/org/apache/kyuubi/grpc/service/AbstractGrpcBackendService.scala index f7e29627caf..502ad3d057d 100644 --- a/kyuubi-grpc/src/main/scala/org/apache/kyuubi/grpc/service/AbstractGrpcBackendService.scala +++ b/kyuubi-grpc/src/main/scala/org/apache/kyuubi/grpc/service/AbstractGrpcBackendService.scala @@ -14,6 +14,7 @@ * See the License for the specific language governing permissions and * limitations under the License. */ + package org.apache.kyuubi.grpc.service import org.apache.kyuubi.config.KyuubiConf diff --git a/kyuubi-grpc/src/main/scala/org/apache/kyuubi/grpc/service/AbstractGrpcFrontendService.scala b/kyuubi-grpc/src/main/scala/org/apache/kyuubi/grpc/service/AbstractGrpcFrontendService.scala index 4328eab1d62..7d057220b65 100644 --- a/kyuubi-grpc/src/main/scala/org/apache/kyuubi/grpc/service/AbstractGrpcFrontendService.scala +++ b/kyuubi-grpc/src/main/scala/org/apache/kyuubi/grpc/service/AbstractGrpcFrontendService.scala @@ -14,6 +14,7 @@ * See the License for the specific language governing permissions and * limitations under the License. */ + package org.apache.kyuubi.grpc.service import java.net.{InetAddress, InetSocketAddress} diff --git a/kyuubi-grpc/src/main/scala/org/apache/kyuubi/grpc/service/GrpcBackendService.scala b/kyuubi-grpc/src/main/scala/org/apache/kyuubi/grpc/service/GrpcBackendService.scala index 59220edd39f..9dbdebcd1d0 100644 --- a/kyuubi-grpc/src/main/scala/org/apache/kyuubi/grpc/service/GrpcBackendService.scala +++ b/kyuubi-grpc/src/main/scala/org/apache/kyuubi/grpc/service/GrpcBackendService.scala @@ -14,6 +14,7 @@ * See the License for the specific language governing permissions and * limitations under the License. */ + package org.apache.kyuubi.grpc.service import org.apache.kyuubi.grpc.session.{GrpcSession, GrpcSessionManager} diff --git a/kyuubi-grpc/src/main/scala/org/apache/kyuubi/grpc/service/GrpcFrontendService.scala b/kyuubi-grpc/src/main/scala/org/apache/kyuubi/grpc/service/GrpcFrontendService.scala index d7a11bc2769..7feebb067fd 100644 --- a/kyuubi-grpc/src/main/scala/org/apache/kyuubi/grpc/service/GrpcFrontendService.scala +++ b/kyuubi-grpc/src/main/scala/org/apache/kyuubi/grpc/service/GrpcFrontendService.scala @@ -14,6 +14,7 @@ * See the License for the specific language governing permissions and * limitations under the License. */ + package org.apache.kyuubi.grpc.service import org.apache.kyuubi.service.Service diff --git a/kyuubi-grpc/src/main/scala/org/apache/kyuubi/grpc/service/GrpcSeverable.scala b/kyuubi-grpc/src/main/scala/org/apache/kyuubi/grpc/service/GrpcSeverable.scala index 0d5dcfd0bb7..64ea8347607 100644 --- a/kyuubi-grpc/src/main/scala/org/apache/kyuubi/grpc/service/GrpcSeverable.scala +++ b/kyuubi-grpc/src/main/scala/org/apache/kyuubi/grpc/service/GrpcSeverable.scala @@ -14,6 +14,7 @@ * See the License for the specific language governing permissions and * limitations under the License. */ + package org.apache.kyuubi.grpc.service import java.util.concurrent.atomic.AtomicBoolean diff --git a/kyuubi-grpc/src/main/scala/org/apache/kyuubi/grpc/session/AbstractGrpcSession.scala b/kyuubi-grpc/src/main/scala/org/apache/kyuubi/grpc/session/AbstractGrpcSession.scala index 485d0c01d2b..1035a5e9116 100644 --- a/kyuubi-grpc/src/main/scala/org/apache/kyuubi/grpc/session/AbstractGrpcSession.scala +++ b/kyuubi-grpc/src/main/scala/org/apache/kyuubi/grpc/session/AbstractGrpcSession.scala @@ -14,6 +14,7 @@ * See the License for the specific language governing permissions and * limitations under the License. */ + package org.apache.kyuubi.grpc.session import java.util diff --git a/kyuubi-grpc/src/main/scala/org/apache/kyuubi/grpc/session/GrpcSession.scala b/kyuubi-grpc/src/main/scala/org/apache/kyuubi/grpc/session/GrpcSession.scala index d0a8684448b..00e3666cca7 100644 --- a/kyuubi-grpc/src/main/scala/org/apache/kyuubi/grpc/session/GrpcSession.scala +++ b/kyuubi-grpc/src/main/scala/org/apache/kyuubi/grpc/session/GrpcSession.scala @@ -14,6 +14,7 @@ * See the License for the specific language governing permissions and * limitations under the License. */ + package org.apache.kyuubi.grpc.session import org.apache.kyuubi.grpc.events.SessionEventsManager diff --git a/kyuubi-grpc/src/main/scala/org/apache/kyuubi/grpc/session/GrpcSessionManager.scala b/kyuubi-grpc/src/main/scala/org/apache/kyuubi/grpc/session/GrpcSessionManager.scala index d482a1dbfbe..2708e542fb1 100644 --- a/kyuubi-grpc/src/main/scala/org/apache/kyuubi/grpc/session/GrpcSessionManager.scala +++ b/kyuubi-grpc/src/main/scala/org/apache/kyuubi/grpc/session/GrpcSessionManager.scala @@ -14,6 +14,7 @@ * See the License for the specific language governing permissions and * limitations under the License. */ + package org.apache.kyuubi.grpc.session import java.io.IOException diff --git a/kyuubi-grpc/src/main/scala/org/apache/kyuubi/grpc/session/SessionKey.scala b/kyuubi-grpc/src/main/scala/org/apache/kyuubi/grpc/session/SessionKey.scala index 9d944275a0d..e02582de331 100644 --- a/kyuubi-grpc/src/main/scala/org/apache/kyuubi/grpc/session/SessionKey.scala +++ b/kyuubi-grpc/src/main/scala/org/apache/kyuubi/grpc/session/SessionKey.scala @@ -14,6 +14,7 @@ * See the License for the specific language governing permissions and * limitations under the License. */ + package org.apache.kyuubi.grpc.session import java.util.UUID diff --git a/kyuubi-grpc/src/main/scala/org/apache/kyuubi/grpc/utils/Clock.scala b/kyuubi-grpc/src/main/scala/org/apache/kyuubi/grpc/utils/Clock.scala index 7de04f4002e..9a471443d59 100644 --- a/kyuubi-grpc/src/main/scala/org/apache/kyuubi/grpc/utils/Clock.scala +++ b/kyuubi-grpc/src/main/scala/org/apache/kyuubi/grpc/utils/Clock.scala @@ -14,6 +14,7 @@ * See the License for the specific language governing permissions and * limitations under the License. */ + package org.apache.kyuubi.grpc.utils trait Clock { diff --git a/kyuubi-grpc/src/main/scala/org/apache/kyuubi/grpc/utils/ProtoUtils.scala b/kyuubi-grpc/src/main/scala/org/apache/kyuubi/grpc/utils/ProtoUtils.scala index bad28db54b5..08528eaacad 100644 --- a/kyuubi-grpc/src/main/scala/org/apache/kyuubi/grpc/utils/ProtoUtils.scala +++ b/kyuubi-grpc/src/main/scala/org/apache/kyuubi/grpc/utils/ProtoUtils.scala @@ -14,6 +14,7 @@ * See the License for the specific language governing permissions and * limitations under the License. */ + package org.apache.kyuubi.grpc.utils object ProtoUtils { diff --git a/kyuubi-grpc/src/test/scala/org/apache/kyuubi/grpc/client/SimpleRpcClient.scala b/kyuubi-grpc/src/test/scala/org/apache/kyuubi/grpc/client/SimpleRpcClient.scala index 25e412be36b..e131a608720 100644 --- a/kyuubi-grpc/src/test/scala/org/apache/kyuubi/grpc/client/SimpleRpcClient.scala +++ b/kyuubi-grpc/src/test/scala/org/apache/kyuubi/grpc/client/SimpleRpcClient.scala @@ -14,6 +14,7 @@ * See the License for the specific language governing permissions and * limitations under the License. */ + package org.apache.kyuubi.grpc.client import java.util.UUID diff --git a/kyuubi-grpc/src/test/scala/org/apache/kyuubi/grpc/event/SimpleOperationEventsManager.scala b/kyuubi-grpc/src/test/scala/org/apache/kyuubi/grpc/event/SimpleOperationEventsManager.scala index 8c47cbbc02d..a2329ac9a31 100644 --- a/kyuubi-grpc/src/test/scala/org/apache/kyuubi/grpc/event/SimpleOperationEventsManager.scala +++ b/kyuubi-grpc/src/test/scala/org/apache/kyuubi/grpc/event/SimpleOperationEventsManager.scala @@ -14,6 +14,7 @@ * See the License for the specific language governing permissions and * limitations under the License. */ + package org.apache.kyuubi.grpc.event import org.apache.kyuubi.Logging diff --git a/kyuubi-grpc/src/test/scala/org/apache/kyuubi/grpc/event/SimpleSessionEventsManager.scala b/kyuubi-grpc/src/test/scala/org/apache/kyuubi/grpc/event/SimpleSessionEventsManager.scala index 9a657d10aa0..a03abc2b0d3 100644 --- a/kyuubi-grpc/src/test/scala/org/apache/kyuubi/grpc/event/SimpleSessionEventsManager.scala +++ b/kyuubi-grpc/src/test/scala/org/apache/kyuubi/grpc/event/SimpleSessionEventsManager.scala @@ -14,6 +14,7 @@ * See the License for the specific language governing permissions and * limitations under the License. */ + package org.apache.kyuubi.grpc.event import org.apache.kyuubi.Logging diff --git a/kyuubi-grpc/src/test/scala/org/apache/kyuubi/grpc/operation/SimpleAddOperationImpl.scala b/kyuubi-grpc/src/test/scala/org/apache/kyuubi/grpc/operation/SimpleAddOperationImpl.scala index 2f9599b7968..28ab5cdee75 100644 --- a/kyuubi-grpc/src/test/scala/org/apache/kyuubi/grpc/operation/SimpleAddOperationImpl.scala +++ b/kyuubi-grpc/src/test/scala/org/apache/kyuubi/grpc/operation/SimpleAddOperationImpl.scala @@ -14,6 +14,7 @@ * See the License for the specific language governing permissions and * limitations under the License. */ + package org.apache.kyuubi.grpc.operation import io.grpc.stub.StreamObserver diff --git a/kyuubi-grpc/src/test/scala/org/apache/kyuubi/grpc/operation/SimpleGrpcOperationImpl.scala b/kyuubi-grpc/src/test/scala/org/apache/kyuubi/grpc/operation/SimpleGrpcOperationImpl.scala index 12500f63529..ae57c68cc1c 100644 --- a/kyuubi-grpc/src/test/scala/org/apache/kyuubi/grpc/operation/SimpleGrpcOperationImpl.scala +++ b/kyuubi-grpc/src/test/scala/org/apache/kyuubi/grpc/operation/SimpleGrpcOperationImpl.scala @@ -14,6 +14,7 @@ * See the License for the specific language governing permissions and * limitations under the License. */ + package org.apache.kyuubi.grpc.operation import org.apache.kyuubi.{KyuubiSQLException, Logging} diff --git a/kyuubi-grpc/src/test/scala/org/apache/kyuubi/grpc/operation/SimpleGrpcOperationManager.scala b/kyuubi-grpc/src/test/scala/org/apache/kyuubi/grpc/operation/SimpleGrpcOperationManager.scala index 8b69a2d5a64..a4ce69d3fd1 100644 --- a/kyuubi-grpc/src/test/scala/org/apache/kyuubi/grpc/operation/SimpleGrpcOperationManager.scala +++ b/kyuubi-grpc/src/test/scala/org/apache/kyuubi/grpc/operation/SimpleGrpcOperationManager.scala @@ -14,6 +14,7 @@ * See the License for the specific language governing permissions and * limitations under the License. */ + package org.apache.kyuubi.grpc.operation import io.grpc.stub.StreamObserver diff --git a/kyuubi-grpc/src/test/scala/org/apache/kyuubi/grpc/operation/SimpleOpenSessionOperationImpl.scala b/kyuubi-grpc/src/test/scala/org/apache/kyuubi/grpc/operation/SimpleOpenSessionOperationImpl.scala index f40d4cc8471..b349e941645 100644 --- a/kyuubi-grpc/src/test/scala/org/apache/kyuubi/grpc/operation/SimpleOpenSessionOperationImpl.scala +++ b/kyuubi-grpc/src/test/scala/org/apache/kyuubi/grpc/operation/SimpleOpenSessionOperationImpl.scala @@ -14,6 +14,7 @@ * See the License for the specific language governing permissions and * limitations under the License. */ + package org.apache.kyuubi.grpc.operation import io.grpc.stub.StreamObserver diff --git a/kyuubi-grpc/src/test/scala/org/apache/kyuubi/grpc/server/GrpcSeverableSuite.scala b/kyuubi-grpc/src/test/scala/org/apache/kyuubi/grpc/server/GrpcSeverableSuite.scala index 1c751c5de92..fc49f190066 100644 --- a/kyuubi-grpc/src/test/scala/org/apache/kyuubi/grpc/server/GrpcSeverableSuite.scala +++ b/kyuubi-grpc/src/test/scala/org/apache/kyuubi/grpc/server/GrpcSeverableSuite.scala @@ -14,6 +14,7 @@ * See the License for the specific language governing permissions and * limitations under the License. */ + package org.apache.kyuubi.grpc.server import io.grpc.{Grpc, InsecureChannelCredentials} diff --git a/kyuubi-grpc/src/test/scala/org/apache/kyuubi/grpc/server/SimpleGrpcServer.scala b/kyuubi-grpc/src/test/scala/org/apache/kyuubi/grpc/server/SimpleGrpcServer.scala index c30236c4441..8dd569da862 100644 --- a/kyuubi-grpc/src/test/scala/org/apache/kyuubi/grpc/server/SimpleGrpcServer.scala +++ b/kyuubi-grpc/src/test/scala/org/apache/kyuubi/grpc/server/SimpleGrpcServer.scala @@ -14,6 +14,7 @@ * See the License for the specific language governing permissions and * limitations under the License. */ + package org.apache.kyuubi.grpc.server import org.apache.kyuubi.grpc.service.SimpleGrpcSeverable diff --git a/kyuubi-grpc/src/test/scala/org/apache/kyuubi/grpc/service/SimpleGrpcBackendService.scala b/kyuubi-grpc/src/test/scala/org/apache/kyuubi/grpc/service/SimpleGrpcBackendService.scala index 64f7dd4fedb..fedab4fd817 100644 --- a/kyuubi-grpc/src/test/scala/org/apache/kyuubi/grpc/service/SimpleGrpcBackendService.scala +++ b/kyuubi-grpc/src/test/scala/org/apache/kyuubi/grpc/service/SimpleGrpcBackendService.scala @@ -14,7 +14,9 @@ * See the License for the specific language governing permissions and * limitations under the License. */ + package org.apache.kyuubi.grpc.service + import io.grpc.stub.StreamObserver import org.apache.kyuubi.KyuubiException diff --git a/kyuubi-grpc/src/test/scala/org/apache/kyuubi/grpc/service/SimpleGrpcFrontendService.scala b/kyuubi-grpc/src/test/scala/org/apache/kyuubi/grpc/service/SimpleGrpcFrontendService.scala index 9b3fc09f36c..cc6f1e7add5 100644 --- a/kyuubi-grpc/src/test/scala/org/apache/kyuubi/grpc/service/SimpleGrpcFrontendService.scala +++ b/kyuubi-grpc/src/test/scala/org/apache/kyuubi/grpc/service/SimpleGrpcFrontendService.scala @@ -14,6 +14,7 @@ * See the License for the specific language governing permissions and * limitations under the License. */ + package org.apache.kyuubi.grpc.service import scala.jdk.CollectionConverters.collectionAsScalaIterableConverter diff --git a/kyuubi-grpc/src/test/scala/org/apache/kyuubi/grpc/service/SimpleGrpcSeverable.scala b/kyuubi-grpc/src/test/scala/org/apache/kyuubi/grpc/service/SimpleGrpcSeverable.scala index 51566586b9c..e8157a62385 100644 --- a/kyuubi-grpc/src/test/scala/org/apache/kyuubi/grpc/service/SimpleGrpcSeverable.scala +++ b/kyuubi-grpc/src/test/scala/org/apache/kyuubi/grpc/service/SimpleGrpcSeverable.scala @@ -14,6 +14,7 @@ * See the License for the specific language governing permissions and * limitations under the License. */ + package org.apache.kyuubi.grpc.service class SimpleGrpcSeverable diff --git a/kyuubi-grpc/src/test/scala/org/apache/kyuubi/grpc/session/SimpleGrpcSessionImpl.scala b/kyuubi-grpc/src/test/scala/org/apache/kyuubi/grpc/session/SimpleGrpcSessionImpl.scala index b4b15a16b45..13a9f6816b6 100644 --- a/kyuubi-grpc/src/test/scala/org/apache/kyuubi/grpc/session/SimpleGrpcSessionImpl.scala +++ b/kyuubi-grpc/src/test/scala/org/apache/kyuubi/grpc/session/SimpleGrpcSessionImpl.scala @@ -14,6 +14,7 @@ * See the License for the specific language governing permissions and * limitations under the License. */ + package org.apache.kyuubi.grpc.session import scala.util.Random diff --git a/kyuubi-grpc/src/test/scala/org/apache/kyuubi/grpc/session/SimpleGrpcSessionManager.scala b/kyuubi-grpc/src/test/scala/org/apache/kyuubi/grpc/session/SimpleGrpcSessionManager.scala index 4c44af3dc1c..72de94e991f 100644 --- a/kyuubi-grpc/src/test/scala/org/apache/kyuubi/grpc/session/SimpleGrpcSessionManager.scala +++ b/kyuubi-grpc/src/test/scala/org/apache/kyuubi/grpc/session/SimpleGrpcSessionManager.scala @@ -14,6 +14,7 @@ * See the License for the specific language governing permissions and * limitations under the License. */ + package org.apache.kyuubi.grpc.session import org.apache.kyuubi.grpc.operation.SimpleGrpcOperationManager From c6af8ed1176f88d7a253fc483cad1cc841fa19cf Mon Sep 17 00:00:00 2001 From: davidyuan Date: Thu, 23 May 2024 19:20:14 +0800 Subject: [PATCH 15/21] update settings.md --- docs/configuration/settings.md | 207 +++++++++++++++++---------------- 1 file changed, 105 insertions(+), 102 deletions(-) diff --git a/docs/configuration/settings.md b/docs/configuration/settings.md index 3d4177e86a7..6d9e5b57406 100644 --- a/docs/configuration/settings.md +++ b/docs/configuration/settings.md @@ -120,108 +120,111 @@ You can configure the Kyuubi properties in `$KYUUBI_HOME/conf/kyuubi-defaults.co ### Engine -| Key | Default | Meaning | Type | Since | -|----------------------------------------------------------|---------------------------|-----------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------|----------|--------| -| kyuubi.engine.chat.ernie.http.connect.timeout | PT2M | The timeout[ms] for establishing the connection with the ernie bot server. A timeout value of zero is interpreted as an infinite timeout. | duration | 1.9.0 | -| kyuubi.engine.chat.ernie.http.proxy | <undefined> | HTTP proxy url for API calling in ernie bot engine. e.g. http://127.0.0.1:1088 | string | 1.9.0 | -| kyuubi.engine.chat.ernie.http.socket.timeout | PT2M | The timeout[ms] for waiting for data packets after ernie bot server connection is established. A timeout value of zero is interpreted as an infinite timeout. | duration | 1.9.0 | -| kyuubi.engine.chat.ernie.model | completions | ID of the model used in ernie bot. Available models are completions_pro, ernie_bot_8k, completions and eb-instant[Model overview](https://cloud.baidu.com/doc/WENXINWORKSHOP/s/6lp69is2a). | string | 1.9.0 | -| kyuubi.engine.chat.ernie.token | <undefined> | The token to access ernie bot open API, which could be got at https://cloud.baidu.com/doc/WENXINWORKSHOP/s/Ilkkrb0i5 | string | 1.9.0 | -| kyuubi.engine.chat.extra.classpath | <undefined> | The extra classpath for the Chat engine, for configuring the location of the SDK and etc. | string | 1.8.0 | -| kyuubi.engine.chat.gpt.apiKey | <undefined> | The key to access OpenAI open API, which could be got at https://platform.openai.com/account/api-keys | string | 1.8.0 | -| kyuubi.engine.chat.gpt.http.connect.timeout | PT2M | The timeout[ms] for establishing the connection with the Chat GPT server. A timeout value of zero is interpreted as an infinite timeout. | duration | 1.8.0 | -| kyuubi.engine.chat.gpt.http.proxy | <undefined> | HTTP proxy url for API calling in Chat GPT engine. e.g. http://127.0.0.1:1087 | string | 1.8.0 | -| kyuubi.engine.chat.gpt.http.socket.timeout | PT2M | The timeout[ms] for waiting for data packets after Chat GPT server connection is established. A timeout value of zero is interpreted as an infinite timeout. | duration | 1.8.0 | -| kyuubi.engine.chat.gpt.model | gpt-3.5-turbo | ID of the model used in ChatGPT. Available models refer to OpenAI's [Model overview](https://platform.openai.com/docs/models/overview). | string | 1.8.0 | -| kyuubi.engine.chat.java.options | <undefined> | The extra Java options for the Chat engine | string | 1.8.0 | -| kyuubi.engine.chat.memory | 1g | The heap memory for the Chat engine | string | 1.8.0 | -| kyuubi.engine.chat.provider | ECHO | The provider for the Chat engine. Candidates:
  • ECHO: simply replies a welcome message.
  • GPT: a.k.a ChatGPT, powered by OpenAI.
  • ERNIE: ErnieBot, powered by Baidu.
| string | 1.8.0 | -| kyuubi.engine.connection.url.use.hostname | true | (deprecated) When true, the engine registers with hostname to zookeeper. When Spark runs on K8s with cluster mode, set to false to ensure that server can connect to engine | boolean | 1.3.0 | -| kyuubi.engine.deregister.exception.classes || A comma-separated list of exception classes. If there is any exception thrown, whose class matches the specified classes, the engine would deregister itself. | set | 1.2.0 | -| kyuubi.engine.deregister.exception.messages || A comma-separated list of exception messages. If there is any exception thrown, whose message or stacktrace matches the specified message list, the engine would deregister itself. | set | 1.2.0 | -| kyuubi.engine.deregister.exception.ttl | PT30M | Time to live(TTL) for exceptions pattern specified in kyuubi.engine.deregister.exception.classes and kyuubi.engine.deregister.exception.messages to deregister engines. Once the total error count hits the kyuubi.engine.deregister.job.max.failures within the TTL, an engine will deregister itself and wait for self-terminated. Otherwise, we suppose that the engine has recovered from temporary failures. | duration | 1.2.0 | -| kyuubi.engine.deregister.job.max.failures | 4 | Number of failures of job before deregistering the engine. | int | 1.2.0 | -| kyuubi.engine.doAs.enabled | true | Whether to enable user impersonation on launching engine. When enabled, for engines which supports user impersonation, e.g. SPARK, depends on the `kyuubi.engine.share.level`, different users will be used to launch the engine. Otherwise, Kyuubi Server's user will always be used to launch the engine. | boolean | 1.9.0 | -| kyuubi.engine.event.json.log.path | file:///tmp/kyuubi/events | The location where all the engine events go for the built-in JSON logger.
  • Local Path: start with 'file://'
  • HDFS Path: start with 'hdfs://'
| string | 1.3.0 | -| kyuubi.engine.event.loggers | SPARK | A comma-separated list of engine history loggers, where engine/session/operation etc events go.
  • SPARK: the events will be written to the Spark listener bus.
  • JSON: the events will be written to the location of kyuubi.engine.event.json.log.path
  • JDBC: to be done
  • CUSTOM: User-defined event handlers.
Note that: Kyuubi supports custom event handlers with the Java SPI. To register a custom event handler, the user needs to implement a subclass of `org.apache.kyuubi.events.handler.CustomEventHandlerProvider` which has a zero-arg constructor. | seq | 1.3.0 | -| kyuubi.engine.flink.application.jars | <undefined> | A comma-separated list of the local jars to be shipped with the job to the cluster. For example, SQL UDF jars. Only effective in yarn application mode. | string | 1.8.0 | -| kyuubi.engine.flink.extra.classpath | <undefined> | The extra classpath for the Flink SQL engine, for configuring the location of hadoop client jars, etc. Only effective in yarn session mode. | string | 1.6.0 | -| kyuubi.engine.flink.initialize.sql | SHOW DATABASES | The initialize sql for Flink engine. It fallback to `kyuubi.engine.initialize.sql`. | seq | 1.8.1 | -| kyuubi.engine.flink.java.options | <undefined> | The extra Java options for the Flink SQL engine. Only effective in yarn session mode. | string | 1.6.0 | -| kyuubi.engine.flink.memory | 1g | The heap memory for the Flink SQL engine. Only effective in yarn session mode. | string | 1.6.0 | -| kyuubi.engine.hive.deploy.mode | LOCAL | Configures the hive engine deploy mode, The value can be 'local', 'yarn'. In local mode, the engine operates on the same node as the KyuubiServer. In YARN mode, the engine runs within the Application Master (AM) container of YARN. | string | 1.9.0 | -| kyuubi.engine.hive.event.loggers | JSON | A comma-separated list of engine history loggers, where engine/session/operation etc events go.
  • JSON: the events will be written to the location of kyuubi.engine.event.json.log.path
  • JDBC: to be done
  • CUSTOM: to be done.
| seq | 1.7.0 | -| kyuubi.engine.hive.extra.classpath | <undefined> | The extra classpath for the Hive query engine, for configuring location of the hadoop client jars and etc. | string | 1.6.0 | -| kyuubi.engine.hive.java.options | <undefined> | The extra Java options for the Hive query engine | string | 1.6.0 | -| kyuubi.engine.hive.memory | 1g | The heap memory for the Hive query engine | string | 1.6.0 | -| kyuubi.engine.initialize.sql | SHOW DATABASES | SemiColon-separated list of SQL statements to be initialized in the newly created engine before queries. i.e. use `SHOW DATABASES` to eagerly active HiveClient. This configuration can not be used in JDBC url due to the limitation of Beeline/JDBC driver. | seq | 1.2.0 | -| kyuubi.engine.jdbc.connection.password | <undefined> | The password is used for connecting to server | string | 1.6.0 | -| kyuubi.engine.jdbc.connection.propagateCredential | false | Whether to use the session's user and password to connect to database | boolean | 1.8.0 | -| kyuubi.engine.jdbc.connection.properties || The additional properties are used for connecting to server | seq | 1.6.0 | -| kyuubi.engine.jdbc.connection.provider | <undefined> | A JDBC connection provider plugin for the Kyuubi Server to establish a connection to the JDBC URL. The configuration value should be a subclass of `org.apache.kyuubi.engine.jdbc.connection.JdbcConnectionProvider`. Kyuubi provides the following built-in implementations:
  • doris: For establishing Doris connections.
  • mysql: For establishing MySQL connections.
  • phoenix: For establishing Phoenix connections.
  • postgresql: For establishing PostgreSQL connections.
  • starrocks: For establishing StarRocks connections.
  • impala: For establishing Impala connections.
  • clickhouse: For establishing clickhouse connections.
  • | string | 1.6.0 | -| kyuubi.engine.jdbc.connection.url | <undefined> | The server url that engine will connect to | string | 1.6.0 | -| kyuubi.engine.jdbc.connection.user | <undefined> | The user is used for connecting to server | string | 1.6.0 | -| kyuubi.engine.jdbc.deploy.mode | LOCAL | Configures the jdbc engine deploy mode, The value can be 'local', 'yarn'. In local mode, the engine operates on the same node as the KyuubiServer. In YARN mode, the engine runs within the Application Master (AM) container of YARN. | string | 1.10.0 | -| kyuubi.engine.jdbc.driver.class | <undefined> | The driver class for JDBC engine connection | string | 1.6.0 | -| kyuubi.engine.jdbc.extra.classpath | <undefined> | The extra classpath for the JDBC query engine, for configuring the location of the JDBC driver and etc. | string | 1.6.0 | -| kyuubi.engine.jdbc.fetch.size | 1000 | The fetch size of JDBC engine | int | 1.9.0 | -| kyuubi.engine.jdbc.initialize.sql | SELECT 1 | SemiColon-separated list of SQL statements to be initialized in the newly created engine before queries. i.e. use `SELECT 1` to eagerly active JDBCClient. | seq | 1.8.0 | -| kyuubi.engine.jdbc.java.options | <undefined> | The extra Java options for the JDBC query engine | string | 1.6.0 | -| kyuubi.engine.jdbc.memory | 1g | The heap memory for the JDBC query engine | string | 1.6.0 | -| kyuubi.engine.jdbc.operation.incremental.collect | false | When true, the result will be sequentially calculated and returned to the JDBC engine. It fallback to `kyuubi.operation.incremental.collect` | boolean | 1.10.0 | -| kyuubi.engine.jdbc.session.initialize.sql || SemiColon-separated list of SQL statements to be initialized in the newly created engine session before queries. | seq | 1.8.0 | -| kyuubi.engine.jdbc.type | <undefined> | The short name of JDBC type | string | 1.6.0 | -| kyuubi.engine.keytab | <undefined> | Kerberos keytab for the kyuubi engine. | string | 1.10.0 | -| kyuubi.engine.kubernetes.submit.timeout | PT30S | The engine submit timeout for Kubernetes application. | duration | 1.7.2 | -| kyuubi.engine.operation.convert.catalog.database.enabled | true | When set to true, The engine converts the JDBC methods of set/get Catalog and set/get Schema to the implementation of different engines | boolean | 1.6.0 | -| kyuubi.engine.operation.log.dir.root | engine_operation_logs | Root directory for query operation log at engine-side. | string | 1.4.0 | -| kyuubi.engine.pool.name | engine-pool | The name of the engine pool. | string | 1.5.0 | -| kyuubi.engine.pool.selectPolicy | RANDOM | The select policy of an engine from the corresponding engine pool engine for a session.
    • RANDOM - Randomly use the engine in the pool
    • POLLING - Polling use the engine in the pool
    | string | 1.7.0 | -| kyuubi.engine.pool.size | -1 | The size of the engine pool. Note that, if the size is less than 1, the engine pool will not be enabled; otherwise, the size of the engine pool will be min(this, kyuubi.engine.pool.size.threshold). | int | 1.4.0 | -| kyuubi.engine.pool.size.threshold | 9 | This parameter is introduced as a server-side parameter controlling the upper limit of the engine pool. | int | 1.4.0 | -| kyuubi.engine.principal | <undefined> | Kerberos principal for the kyuubi engine. | string | 1.10.0 | -| kyuubi.engine.session.initialize.sql || SemiColon-separated list of SQL statements to be initialized in the newly created engine session before queries. This configuration can not be used in JDBC url due to the limitation of Beeline/JDBC driver. | seq | 1.3.0 | -| kyuubi.engine.share.level | USER | Engines will be shared in different levels, available configs are:
    • CONNECTION: the engine will not be shared but only used by the current client connection, and the engine will be launched by session user.
    • USER: the engine will be shared by all sessions created by a unique username, and the engine will be launched by session user.
    • GROUP: the engine will be shared by all sessions created by all users belong to the same primary group name. The engine will be launched by the primary group name as the effective username, so here the group name is in value of special user who is able to visit the computing resources/data of the team. It follows the [Hadoop GroupsMapping](https://reurl.cc/xE61Y5) to map user to a primary group. If the primary group is not found, it fallback to the USER level.
    • SERVER: the engine will be shared by Kyuubi servers, and the engine will be launched by Server's user.
    See also `kyuubi.engine.share.level.subdomain` and `kyuubi.engine.doAs.enabled`. | string | 1.2.0 | -| kyuubi.engine.share.level.sub.domain | <undefined> | (deprecated) - Using kyuubi.engine.share.level.subdomain instead | string | 1.2.0 | -| kyuubi.engine.share.level.subdomain | <undefined> | Allow end-users to create a subdomain for the share level of an engine. A subdomain is a case-insensitive string values that must be a valid zookeeper subpath. For example, for the `USER` share level, an end-user can share a certain engine within a subdomain, not for all of its clients. End-users are free to create multiple engines in the `USER` share level. When disable engine pool, use 'default' if absent. | string | 1.4.0 | -| kyuubi.engine.single.spark.session | false | When set to true, this engine is running in a single session mode. All the JDBC/ODBC connections share the temporary views, function registries, SQL configuration and the current database. | boolean | 1.3.0 | -| kyuubi.engine.spark.event.loggers | SPARK | A comma-separated list of engine loggers, where engine/session/operation etc events go.
    • SPARK: the events will be written to the Spark listener bus.
    • JSON: the events will be written to the location of kyuubi.engine.event.json.log.path
    • JDBC: to be done
    • CUSTOM: to be done.
    | seq | 1.7.0 | -| kyuubi.engine.spark.initialize.sql | SHOW DATABASES | The initialize sql for Spark engine. It fallback to `kyuubi.engine.initialize.sql`. | seq | 1.8.1 | -| kyuubi.engine.spark.operation.incremental.collect | false | When true, the result will be sequentially calculated and returned to the Spark driver. Note that, kyuubi.operation.result.max.rows will be ignored on incremental collect mode. It fallback to `kyuubi.operation.incremental.collect` | boolean | 1.10.0 | -| kyuubi.engine.spark.output.mode | AUTO | The output mode of Spark engine:
    • AUTO: For PySpark, the extracted `text/plain` from python response as output.
    • NOTEBOOK: For PySpark, the original python response as output.
    | string | 1.9.0 | -| kyuubi.engine.spark.python.env.archive | <undefined> | Portable Python env archive used for Spark engine Python language mode. | string | 1.7.0 | -| kyuubi.engine.spark.python.env.archive.exec.path | bin/python | The Python exec path under the Python env archive. | string | 1.7.0 | -| kyuubi.engine.spark.python.home.archive | <undefined> | Spark archive containing $SPARK_HOME/python directory, which is used to init session Python worker for Python language mode. | string | 1.7.0 | -| kyuubi.engine.submit.timeout | PT30S | Period to tolerant Driver Pod ephemerally invisible after submitting. In some Resource Managers, e.g. K8s, the Driver Pod is not visible immediately after `spark-submit` is returned. | duration | 1.7.1 | -| kyuubi.engine.trino.connection.keystore.password | <undefined> | The keystore password used for connecting to trino cluster | string | 1.8.0 | -| kyuubi.engine.trino.connection.keystore.path | <undefined> | The keystore path used for connecting to trino cluster | string | 1.8.0 | -| kyuubi.engine.trino.connection.keystore.type | <undefined> | The keystore type used for connecting to trino cluster | string | 1.8.0 | -| kyuubi.engine.trino.connection.password | <undefined> | The password used for connecting to trino cluster | string | 1.8.0 | -| kyuubi.engine.trino.connection.truststore.password | <undefined> | The truststore password used for connecting to trino cluster | string | 1.8.0 | -| kyuubi.engine.trino.connection.truststore.path | <undefined> | The truststore path used for connecting to trino cluster | string | 1.8.0 | -| kyuubi.engine.trino.connection.truststore.type | <undefined> | The truststore type used for connecting to trino cluster | string | 1.8.0 | -| kyuubi.engine.trino.connection.user | <undefined> | The user used for connecting to trino cluster | string | 1.9.0 | -| kyuubi.engine.trino.event.loggers | JSON | A comma-separated list of engine history loggers, where engine/session/operation etc events go.
    • JSON: the events will be written to the location of kyuubi.engine.event.json.log.path
    • JDBC: to be done
    • CUSTOM: to be done.
    | seq | 1.7.0 | -| kyuubi.engine.trino.extra.classpath | <undefined> | The extra classpath for the Trino query engine, for configuring other libs which may need by the Trino engine | string | 1.6.0 | -| kyuubi.engine.trino.java.options | <undefined> | The extra Java options for the Trino query engine | string | 1.6.0 | -| kyuubi.engine.trino.memory | 1g | The heap memory for the Trino query engine | string | 1.6.0 | -| kyuubi.engine.trino.operation.incremental.collect | false | When true, the result will be sequentially calculated and returned to the trino. It fallback to `kyuubi.operation.incremental.collect` | boolean | 1.10.0 | -| kyuubi.engine.type | SPARK_SQL | Specify the detailed engine supported by Kyuubi. The engine type bindings to SESSION scope. This configuration is experimental. Currently, available configs are:
    • SPARK_SQL: specify this engine type will launch a Spark engine which can provide all the capacity of the Apache Spark. Note, it's a default engine type.
    • FLINK_SQL: specify this engine type will launch a Flink engine which can provide all the capacity of the Apache Flink.
    • TRINO: specify this engine type will launch a Trino engine which can provide all the capacity of the Trino.
    • HIVE_SQL: specify this engine type will launch a Hive engine which can provide all the capacity of the Hive Server2.
    • JDBC: specify this engine type will launch a JDBC engine which can forward queries to the database system through the certain JDBC driver, for now, it supports Doris, MySQL, Phoenix, PostgreSQL, StarRocks, Impala and ClickHouse.
    • CHAT: specify this engine type will launch a Chat engine.
    | string | 1.4.0 | -| kyuubi.engine.ui.retainedSessions | 200 | The number of SQL client sessions kept in the Kyuubi Query Engine web UI. | int | 1.4.0 | -| kyuubi.engine.ui.retainedStatements | 200 | The number of statements kept in the Kyuubi Query Engine web UI. | int | 1.4.0 | -| kyuubi.engine.ui.stop.enabled | true | When true, allows Kyuubi engine to be killed from the Spark Web UI. | boolean | 1.3.0 | -| kyuubi.engine.user.isolated.spark.session | true | When set to false, if the engine is running in a group or server share level, all the JDBC/ODBC connections will be isolated against the user. Including the temporary views, function registries, SQL configuration, and the current database. Note that, it does not affect if the share level is connection or user. | boolean | 1.6.0 | -| kyuubi.engine.user.isolated.spark.session.idle.interval | PT1M | The interval to check if the user-isolated Spark session is timeout. | duration | 1.6.0 | -| kyuubi.engine.user.isolated.spark.session.idle.timeout | PT6H | If kyuubi.engine.user.isolated.spark.session is false, we will release the Spark session if its corresponding user is inactive after this configured timeout. | duration | 1.6.0 | -| kyuubi.engine.yarn.app.name | <undefined> | The YARN app name when the engine deploy mode is YARN. | string | 1.9.0 | -| kyuubi.engine.yarn.cores | 1 | kyuubi engine container core number when the engine deploy mode is YARN. | int | 1.9.0 | -| kyuubi.engine.yarn.java.options | <undefined> | The extra Java options for the AM when the engine deploy mode is YARN. | string | 1.9.0 | -| kyuubi.engine.yarn.memory | 1024 | kyuubi engine container memory in mb when the engine deploy mode is YARN. | int | 1.9.0 | -| kyuubi.engine.yarn.priority | <undefined> | kyuubi engine yarn priority when the engine deploy mode is YARN. | int | 1.9.0 | -| kyuubi.engine.yarn.queue | default | kyuubi engine yarn queue when the engine deploy mode is YARN. | string | 1.9.0 | -| kyuubi.engine.yarn.stagingDir | <undefined> | Staging directory used while submitting kyuubi engine to YARN, It should be a absolute path in HDFS. | string | 1.9.0 | -| kyuubi.engine.yarn.submit.timeout | PT30S | The engine submit timeout for YARN application. | duration | 1.7.2 | -| kyuubi.engine.yarn.tags | <undefined> | kyuubi engine yarn tags when the engine deploy mode is YARN. | seq | 1.9.0 | +| Key | Default | Meaning | Type | Since | +|-----------------------------------------------------------|---------------------------|-----------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------|----------|--------| +| kyuubi.engine.chat.ernie.http.connect.timeout | PT2M | The timeout[ms] for establishing the connection with the ernie bot server. A timeout value of zero is interpreted as an infinite timeout. | duration | 1.9.0 | +| kyuubi.engine.chat.ernie.http.proxy | <undefined> | HTTP proxy url for API calling in ernie bot engine. e.g. http://127.0.0.1:1088 | string | 1.9.0 | +| kyuubi.engine.chat.ernie.http.socket.timeout | PT2M | The timeout[ms] for waiting for data packets after ernie bot server connection is established. A timeout value of zero is interpreted as an infinite timeout. | duration | 1.9.0 | +| kyuubi.engine.chat.ernie.model | completions | ID of the model used in ernie bot. Available models are completions_pro, ernie_bot_8k, completions and eb-instant[Model overview](https://cloud.baidu.com/doc/WENXINWORKSHOP/s/6lp69is2a). | string | 1.9.0 | +| kyuubi.engine.chat.ernie.token | <undefined> | The token to access ernie bot open API, which could be got at https://cloud.baidu.com/doc/WENXINWORKSHOP/s/Ilkkrb0i5 | string | 1.9.0 | +| kyuubi.engine.chat.extra.classpath | <undefined> | The extra classpath for the Chat engine, for configuring the location of the SDK and etc. | string | 1.8.0 | +| kyuubi.engine.chat.gpt.apiKey | <undefined> | The key to access OpenAI open API, which could be got at https://platform.openai.com/account/api-keys | string | 1.8.0 | +| kyuubi.engine.chat.gpt.http.connect.timeout | PT2M | The timeout[ms] for establishing the connection with the Chat GPT server. A timeout value of zero is interpreted as an infinite timeout. | duration | 1.8.0 | +| kyuubi.engine.chat.gpt.http.proxy | <undefined> | HTTP proxy url for API calling in Chat GPT engine. e.g. http://127.0.0.1:1087 | string | 1.8.0 | +| kyuubi.engine.chat.gpt.http.socket.timeout | PT2M | The timeout[ms] for waiting for data packets after Chat GPT server connection is established. A timeout value of zero is interpreted as an infinite timeout. | duration | 1.8.0 | +| kyuubi.engine.chat.gpt.model | gpt-3.5-turbo | ID of the model used in ChatGPT. Available models refer to OpenAI's [Model overview](https://platform.openai.com/docs/models/overview). | string | 1.8.0 | +| kyuubi.engine.chat.java.options | <undefined> | The extra Java options for the Chat engine | string | 1.8.0 | +| kyuubi.engine.chat.memory | 1g | The heap memory for the Chat engine | string | 1.8.0 | +| kyuubi.engine.chat.provider | ECHO | The provider for the Chat engine. Candidates:
    • ECHO: simply replies a welcome message.
    • GPT: a.k.a ChatGPT, powered by OpenAI.
    • ERNIE: ErnieBot, powered by Baidu.
    | string | 1.8.0 | +| kyuubi.engine.connection.url.use.hostname | true | (deprecated) When true, the engine registers with hostname to zookeeper. When Spark runs on K8s with cluster mode, set to false to ensure that server can connect to engine | boolean | 1.3.0 | +| kyuubi.engine.deregister.exception.classes || A comma-separated list of exception classes. If there is any exception thrown, whose class matches the specified classes, the engine would deregister itself. | set | 1.2.0 | +| kyuubi.engine.deregister.exception.messages || A comma-separated list of exception messages. If there is any exception thrown, whose message or stacktrace matches the specified message list, the engine would deregister itself. | set | 1.2.0 | +| kyuubi.engine.deregister.exception.ttl | PT30M | Time to live(TTL) for exceptions pattern specified in kyuubi.engine.deregister.exception.classes and kyuubi.engine.deregister.exception.messages to deregister engines. Once the total error count hits the kyuubi.engine.deregister.job.max.failures within the TTL, an engine will deregister itself and wait for self-terminated. Otherwise, we suppose that the engine has recovered from temporary failures. | duration | 1.2.0 | +| kyuubi.engine.deregister.job.max.failures | 4 | Number of failures of job before deregistering the engine. | int | 1.2.0 | +| kyuubi.engine.doAs.enabled | true | Whether to enable user impersonation on launching engine. When enabled, for engines which supports user impersonation, e.g. SPARK, depends on the `kyuubi.engine.share.level`, different users will be used to launch the engine. Otherwise, Kyuubi Server's user will always be used to launch the engine. | boolean | 1.9.0 | +| kyuubi.engine.event.json.log.path | file:///tmp/kyuubi/events | The location where all the engine events go for the built-in JSON logger.
    • Local Path: start with 'file://'
    • HDFS Path: start with 'hdfs://'
    | string | 1.3.0 | +| kyuubi.engine.event.loggers | SPARK | A comma-separated list of engine history loggers, where engine/session/operation etc events go.
    • SPARK: the events will be written to the Spark listener bus.
    • JSON: the events will be written to the location of kyuubi.engine.event.json.log.path
    • JDBC: to be done
    • CUSTOM: User-defined event handlers.
    Note that: Kyuubi supports custom event handlers with the Java SPI. To register a custom event handler, the user needs to implement a subclass of `org.apache.kyuubi.events.handler.CustomEventHandlerProvider` which has a zero-arg constructor. | seq | 1.3.0 | +| kyuubi.engine.flink.application.jars | <undefined> | A comma-separated list of the local jars to be shipped with the job to the cluster. For example, SQL UDF jars. Only effective in yarn application mode. | string | 1.8.0 | +| kyuubi.engine.flink.extra.classpath | <undefined> | The extra classpath for the Flink SQL engine, for configuring the location of hadoop client jars, etc. Only effective in yarn session mode. | string | 1.6.0 | +| kyuubi.engine.flink.initialize.sql | SHOW DATABASES | The initialize sql for Flink engine. It fallback to `kyuubi.engine.initialize.sql`. | seq | 1.8.1 | +| kyuubi.engine.flink.java.options | <undefined> | The extra Java options for the Flink SQL engine. Only effective in yarn session mode. | string | 1.6.0 | +| kyuubi.engine.flink.memory | 1g | The heap memory for the Flink SQL engine. Only effective in yarn session mode. | string | 1.6.0 | +| kyuubi.engine.hive.deploy.mode | LOCAL | Configures the hive engine deploy mode, The value can be 'local', 'yarn'. In local mode, the engine operates on the same node as the KyuubiServer. In YARN mode, the engine runs within the Application Master (AM) container of YARN. | string | 1.9.0 | +| kyuubi.engine.hive.event.loggers | JSON | A comma-separated list of engine history loggers, where engine/session/operation etc events go.
    • JSON: the events will be written to the location of kyuubi.engine.event.json.log.path
    • JDBC: to be done
    • CUSTOM: to be done.
    | seq | 1.7.0 | +| kyuubi.engine.hive.extra.classpath | <undefined> | The extra classpath for the Hive query engine, for configuring location of the hadoop client jars and etc. | string | 1.6.0 | +| kyuubi.engine.hive.java.options | <undefined> | The extra Java options for the Hive query engine | string | 1.6.0 | +| kyuubi.engine.hive.memory | 1g | The heap memory for the Hive query engine | string | 1.6.0 | +| kyuubi.engine.initialize.sql | SHOW DATABASES | SemiColon-separated list of SQL statements to be initialized in the newly created engine before queries. i.e. use `SHOW DATABASES` to eagerly active HiveClient. This configuration can not be used in JDBC url due to the limitation of Beeline/JDBC driver. | seq | 1.2.0 | +| kyuubi.engine.jdbc.connection.password | <undefined> | The password is used for connecting to server | string | 1.6.0 | +| kyuubi.engine.jdbc.connection.propagateCredential | false | Whether to use the session's user and password to connect to database | boolean | 1.8.0 | +| kyuubi.engine.jdbc.connection.properties || The additional properties are used for connecting to server | seq | 1.6.0 | +| kyuubi.engine.jdbc.connection.provider | <undefined> | A JDBC connection provider plugin for the Kyuubi Server to establish a connection to the JDBC URL. The configuration value should be a subclass of `org.apache.kyuubi.engine.jdbc.connection.JdbcConnectionProvider`. Kyuubi provides the following built-in implementations:
  • doris: For establishing Doris connections.
  • mysql: For establishing MySQL connections.
  • phoenix: For establishing Phoenix connections.
  • postgresql: For establishing PostgreSQL connections.
  • starrocks: For establishing StarRocks connections.
  • impala: For establishing Impala connections.
  • clickhouse: For establishing clickhouse connections.
  • | string | 1.6.0 | +| kyuubi.engine.jdbc.connection.url | <undefined> | The server url that engine will connect to | string | 1.6.0 | +| kyuubi.engine.jdbc.connection.user | <undefined> | The user is used for connecting to server | string | 1.6.0 | +| kyuubi.engine.jdbc.deploy.mode | LOCAL | Configures the jdbc engine deploy mode, The value can be 'local', 'yarn'. In local mode, the engine operates on the same node as the KyuubiServer. In YARN mode, the engine runs within the Application Master (AM) container of YARN. | string | 1.10.0 | +| kyuubi.engine.jdbc.driver.class | <undefined> | The driver class for JDBC engine connection | string | 1.6.0 | +| kyuubi.engine.jdbc.extra.classpath | <undefined> | The extra classpath for the JDBC query engine, for configuring the location of the JDBC driver and etc. | string | 1.6.0 | +| kyuubi.engine.jdbc.fetch.size | 1000 | The fetch size of JDBC engine | int | 1.9.0 | +| kyuubi.engine.jdbc.initialize.sql | SELECT 1 | SemiColon-separated list of SQL statements to be initialized in the newly created engine before queries. i.e. use `SELECT 1` to eagerly active JDBCClient. | seq | 1.8.0 | +| kyuubi.engine.jdbc.java.options | <undefined> | The extra Java options for the JDBC query engine | string | 1.6.0 | +| kyuubi.engine.jdbc.memory | 1g | The heap memory for the JDBC query engine | string | 1.6.0 | +| kyuubi.engine.jdbc.operation.incremental.collect | false | When true, the result will be sequentially calculated and returned to the JDBC engine. It fallback to `kyuubi.operation.incremental.collect` | boolean | 1.10.0 | +| kyuubi.engine.jdbc.session.initialize.sql || SemiColon-separated list of SQL statements to be initialized in the newly created engine session before queries. | seq | 1.8.0 | +| kyuubi.engine.jdbc.type | <undefined> | The short name of JDBC type | string | 1.6.0 | +| kyuubi.engine.keytab | <undefined> | Kerberos keytab for the kyuubi engine. | string | 1.10.0 | +| kyuubi.engine.kubernetes.submit.timeout | PT30S | The engine submit timeout for Kubernetes application. | duration | 1.7.2 | +| kyuubi.engine.operation.convert.catalog.database.enabled | true | When set to true, The engine converts the JDBC methods of set/get Catalog and set/get Schema to the implementation of different engines | boolean | 1.6.0 | +| kyuubi.engine.operation.log.dir.root | engine_operation_logs | Root directory for query operation log at engine-side. | string | 1.4.0 | +| kyuubi.engine.pool.name | engine-pool | The name of the engine pool. | string | 1.5.0 | +| kyuubi.engine.pool.selectPolicy | RANDOM | The select policy of an engine from the corresponding engine pool engine for a session.
    • RANDOM - Randomly use the engine in the pool
    • POLLING - Polling use the engine in the pool
    | string | 1.7.0 | +| kyuubi.engine.pool.size | -1 | The size of the engine pool. Note that, if the size is less than 1, the engine pool will not be enabled; otherwise, the size of the engine pool will be min(this, kyuubi.engine.pool.size.threshold). | int | 1.4.0 | +| kyuubi.engine.pool.size.threshold | 9 | This parameter is introduced as a server-side parameter controlling the upper limit of the engine pool. | int | 1.4.0 | +| kyuubi.engine.principal | <undefined> | Kerberos principal for the kyuubi engine. | string | 1.10.0 | +| kyuubi.engine.session.initialize.sql || SemiColon-separated list of SQL statements to be initialized in the newly created engine session before queries. This configuration can not be used in JDBC url due to the limitation of Beeline/JDBC driver. | seq | 1.3.0 | +| kyuubi.engine.share.level | USER | Engines will be shared in different levels, available configs are:
    • CONNECTION: the engine will not be shared but only used by the current client connection, and the engine will be launched by session user.
    • USER: the engine will be shared by all sessions created by a unique username, and the engine will be launched by session user.
    • GROUP: the engine will be shared by all sessions created by all users belong to the same primary group name. The engine will be launched by the primary group name as the effective username, so here the group name is in value of special user who is able to visit the computing resources/data of the team. It follows the [Hadoop GroupsMapping](https://reurl.cc/xE61Y5) to map user to a primary group. If the primary group is not found, it fallback to the USER level.
    • SERVER: the engine will be shared by Kyuubi servers, and the engine will be launched by Server's user.
    See also `kyuubi.engine.share.level.subdomain` and `kyuubi.engine.doAs.enabled`. | string | 1.2.0 | +| kyuubi.engine.share.level.sub.domain | <undefined> | (deprecated) - Using kyuubi.engine.share.level.subdomain instead | string | 1.2.0 | +| kyuubi.engine.share.level.subdomain | <undefined> | Allow end-users to create a subdomain for the share level of an engine. A subdomain is a case-insensitive string values that must be a valid zookeeper subpath. For example, for the `USER` share level, an end-user can share a certain engine within a subdomain, not for all of its clients. End-users are free to create multiple engines in the `USER` share level. When disable engine pool, use 'default' if absent. | string | 1.4.0 | +| kyuubi.engine.single.spark.session | false | When set to true, this engine is running in a single session mode. All the JDBC/ODBC connections share the temporary views, function registries, SQL configuration and the current database. | boolean | 1.3.0 | +| kyuubi.engine.spark.connect.grpc.bind.host | <undefined> | Hostname or IP of the machine on which to run the grpc server in frontend service | string | 1.9.0 | +| kyuubi.engine.spark.connect.grpc.bind.port | 15002 | The port is used in spark connect frontendService start GrpcServer | int | 1.9.0 | +| kyuubi.engine.spark.connect.grpc.max.inbound.message.size | 134217728 | Sets the maximum inbound message in bytes size for the gRPC requests.Requests with a larger payload will fail. | int | 1.9.0 | +| kyuubi.engine.spark.event.loggers | SPARK | A comma-separated list of engine loggers, where engine/session/operation etc events go.
    • SPARK: the events will be written to the Spark listener bus.
    • JSON: the events will be written to the location of kyuubi.engine.event.json.log.path
    • JDBC: to be done
    • CUSTOM: to be done.
    | seq | 1.7.0 | +| kyuubi.engine.spark.initialize.sql | SHOW DATABASES | The initialize sql for Spark engine. It fallback to `kyuubi.engine.initialize.sql`. | seq | 1.8.1 | +| kyuubi.engine.spark.operation.incremental.collect | false | When true, the result will be sequentially calculated and returned to the Spark driver. Note that, kyuubi.operation.result.max.rows will be ignored on incremental collect mode. It fallback to `kyuubi.operation.incremental.collect` | boolean | 1.10.0 | +| kyuubi.engine.spark.output.mode | AUTO | The output mode of Spark engine:
    • AUTO: For PySpark, the extracted `text/plain` from python response as output.
    • NOTEBOOK: For PySpark, the original python response as output.
    | string | 1.9.0 | +| kyuubi.engine.spark.python.env.archive | <undefined> | Portable Python env archive used for Spark engine Python language mode. | string | 1.7.0 | +| kyuubi.engine.spark.python.env.archive.exec.path | bin/python | The Python exec path under the Python env archive. | string | 1.7.0 | +| kyuubi.engine.spark.python.home.archive | <undefined> | Spark archive containing $SPARK_HOME/python directory, which is used to init session Python worker for Python language mode. | string | 1.7.0 | +| kyuubi.engine.submit.timeout | PT30S | Period to tolerant Driver Pod ephemerally invisible after submitting. In some Resource Managers, e.g. K8s, the Driver Pod is not visible immediately after `spark-submit` is returned. | duration | 1.7.1 | +| kyuubi.engine.trino.connection.keystore.password | <undefined> | The keystore password used for connecting to trino cluster | string | 1.8.0 | +| kyuubi.engine.trino.connection.keystore.path | <undefined> | The keystore path used for connecting to trino cluster | string | 1.8.0 | +| kyuubi.engine.trino.connection.keystore.type | <undefined> | The keystore type used for connecting to trino cluster | string | 1.8.0 | +| kyuubi.engine.trino.connection.password | <undefined> | The password used for connecting to trino cluster | string | 1.8.0 | +| kyuubi.engine.trino.connection.truststore.password | <undefined> | The truststore password used for connecting to trino cluster | string | 1.8.0 | +| kyuubi.engine.trino.connection.truststore.path | <undefined> | The truststore path used for connecting to trino cluster | string | 1.8.0 | +| kyuubi.engine.trino.connection.truststore.type | <undefined> | The truststore type used for connecting to trino cluster | string | 1.8.0 | +| kyuubi.engine.trino.connection.user | <undefined> | The user used for connecting to trino cluster | string | 1.9.0 | +| kyuubi.engine.trino.event.loggers | JSON | A comma-separated list of engine history loggers, where engine/session/operation etc events go.
    • JSON: the events will be written to the location of kyuubi.engine.event.json.log.path
    • JDBC: to be done
    • CUSTOM: to be done.
    | seq | 1.7.0 | +| kyuubi.engine.trino.extra.classpath | <undefined> | The extra classpath for the Trino query engine, for configuring other libs which may need by the Trino engine | string | 1.6.0 | +| kyuubi.engine.trino.java.options | <undefined> | The extra Java options for the Trino query engine | string | 1.6.0 | +| kyuubi.engine.trino.memory | 1g | The heap memory for the Trino query engine | string | 1.6.0 | +| kyuubi.engine.trino.operation.incremental.collect | false | When true, the result will be sequentially calculated and returned to the trino. It fallback to `kyuubi.operation.incremental.collect` | boolean | 1.10.0 | +| kyuubi.engine.type | SPARK_SQL | Specify the detailed engine supported by Kyuubi. The engine type bindings to SESSION scope. This configuration is experimental. Currently, available configs are:
    • SPARK_SQL: specify this engine type will launch a Spark engine which can provide all the capacity of the Apache Spark. Note, it's a default engine type.
    • FLINK_SQL: specify this engine type will launch a Flink engine which can provide all the capacity of the Apache Flink.
    • TRINO: specify this engine type will launch a Trino engine which can provide all the capacity of the Trino.
    • HIVE_SQL: specify this engine type will launch a Hive engine which can provide all the capacity of the Hive Server2.
    • JDBC: specify this engine type will launch a JDBC engine which can forward queries to the database system through the certain JDBC driver, for now, it supports Doris, MySQL, Phoenix, PostgreSQL, StarRocks, Impala and ClickHouse.
    • CHAT: specify this engine type will launch a Chat engine.
    | string | 1.4.0 | +| kyuubi.engine.ui.retainedSessions | 200 | The number of SQL client sessions kept in the Kyuubi Query Engine web UI. | int | 1.4.0 | +| kyuubi.engine.ui.retainedStatements | 200 | The number of statements kept in the Kyuubi Query Engine web UI. | int | 1.4.0 | +| kyuubi.engine.ui.stop.enabled | true | When true, allows Kyuubi engine to be killed from the Spark Web UI. | boolean | 1.3.0 | +| kyuubi.engine.user.isolated.spark.session | true | When set to false, if the engine is running in a group or server share level, all the JDBC/ODBC connections will be isolated against the user. Including the temporary views, function registries, SQL configuration, and the current database. Note that, it does not affect if the share level is connection or user. | boolean | 1.6.0 | +| kyuubi.engine.user.isolated.spark.session.idle.interval | PT1M | The interval to check if the user-isolated Spark session is timeout. | duration | 1.6.0 | +| kyuubi.engine.user.isolated.spark.session.idle.timeout | PT6H | If kyuubi.engine.user.isolated.spark.session is false, we will release the Spark session if its corresponding user is inactive after this configured timeout. | duration | 1.6.0 | +| kyuubi.engine.yarn.app.name | <undefined> | The YARN app name when the engine deploy mode is YARN. | string | 1.9.0 | +| kyuubi.engine.yarn.cores | 1 | kyuubi engine container core number when the engine deploy mode is YARN. | int | 1.9.0 | +| kyuubi.engine.yarn.java.options | <undefined> | The extra Java options for the AM when the engine deploy mode is YARN. | string | 1.9.0 | +| kyuubi.engine.yarn.memory | 1024 | kyuubi engine container memory in mb when the engine deploy mode is YARN. | int | 1.9.0 | +| kyuubi.engine.yarn.priority | <undefined> | kyuubi engine yarn priority when the engine deploy mode is YARN. | int | 1.9.0 | +| kyuubi.engine.yarn.queue | default | kyuubi engine yarn queue when the engine deploy mode is YARN. | string | 1.9.0 | +| kyuubi.engine.yarn.stagingDir | <undefined> | Staging directory used while submitting kyuubi engine to YARN, It should be a absolute path in HDFS. | string | 1.9.0 | +| kyuubi.engine.yarn.submit.timeout | PT30S | The engine submit timeout for YARN application. | duration | 1.7.2 | +| kyuubi.engine.yarn.tags | <undefined> | kyuubi engine yarn tags when the engine deploy mode is YARN. | seq | 1.9.0 | ### Event From a1e0447951d771a537b8e40d235a6bf9cf44bfdd Mon Sep 17 00:00:00 2001 From: davidyuan Date: Tue, 28 May 2024 16:32:50 +0800 Subject: [PATCH 16/21] update --- .../org/apache/kyuubi/config/KyuubiConf.scala | 14 ++++ kyuubi-grpc-server/pom.xml | 78 +++++++++++++++++++ .../AbstractKyuubiGrpcFrontendService.scala | 42 ++++++++++ .../server/KyuubiGrpcBackendService.scala | 28 +++++++ .../kyuubi/grpc/server/KyuubiGrpcServer.scala | 32 ++++++++ .../KyuubiSparkConnectFrontendService.scala | 64 +++++++++++++++ pom.xml | 1 + 7 files changed, 259 insertions(+) create mode 100644 kyuubi-grpc-server/pom.xml create mode 100644 kyuubi-grpc-server/src/main/scala/org/apache/kyuubi/grpc/server/AbstractKyuubiGrpcFrontendService.scala create mode 100644 kyuubi-grpc-server/src/main/scala/org/apache/kyuubi/grpc/server/KyuubiGrpcBackendService.scala create mode 100644 kyuubi-grpc-server/src/main/scala/org/apache/kyuubi/grpc/server/KyuubiGrpcServer.scala create mode 100644 kyuubi-grpc-server/src/main/scala/org/apache/kyuubi/grpc/server/KyuubiSparkConnectFrontendService.scala 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 a3b2a96ba22..4cd2302f3e9 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 @@ -1064,6 +1064,20 @@ object KyuubiConf { .transformToLowerCase .createWithDefault(SaslQOP.AUTH.toString) + val GRPC_FRONTEND_BIND_HOST: OptionalConfigEntry[String] = buildConf("kyuubi.grpc.frontend.bind.host") + .doc("Hostname or IP of the machine on which to run the grpc frontend services.") + .version("1.0.0") + .serverOnly + .stringConf + .createOptional + + val GRPC_FRONTEND_SPARK_CONNECT_HOST: ConfigEntry[Option[String]] = + buildConf("kyuubi.grpc.frontend.spark.connect.host") + .doc("Hostname or IP of the machine on which to run the Spark Connect Grpc frontend service.") + .version("1.4.0") + .serverOnly + .fallbackConf(FRONTEND_BIND_HOST) + val FRONTEND_REST_BIND_HOST: ConfigEntry[Option[String]] = buildConf("kyuubi.frontend.rest.bind.host") .doc("Hostname or IP of the machine on which to run the REST frontend service.") diff --git a/kyuubi-grpc-server/pom.xml b/kyuubi-grpc-server/pom.xml new file mode 100644 index 00000000000..36052f0de53 --- /dev/null +++ b/kyuubi-grpc-server/pom.xml @@ -0,0 +1,78 @@ + + + + + kyuubi-parent + org.apache.kyuubi + 1.10.0-SNAPSHOT + ../pom.xml + + 4.0.0 + + kyuubi-grpc-server_${scala.binary.version} + jar + Kyuubi Project Grpc Server + https://kyuubi.apache.org/ + + + + org.apache.kyuubi + kyuubi-grpc_${scala.binary.version} + ${project.version} + + + + org.apache.kyuubi + kyuubi-common_${scala.binary.version} + ${project.version} + + + + org.apache.kyuubi + kyuubi-common_${scala.binary.version} + ${project.version} + test-jar + test + + + + org.apache.kyuubi + kyuubi-grpc_${scala.binary.version} + ${project.version} + test-jar + test + + + + + org.apache.spark + spark-connect_2.12 + 3.5.0 + + + io.grpc + grpc-all + + + + + + + diff --git a/kyuubi-grpc-server/src/main/scala/org/apache/kyuubi/grpc/server/AbstractKyuubiGrpcFrontendService.scala b/kyuubi-grpc-server/src/main/scala/org/apache/kyuubi/grpc/server/AbstractKyuubiGrpcFrontendService.scala new file mode 100644 index 00000000000..fc9ded961c6 --- /dev/null +++ b/kyuubi-grpc-server/src/main/scala/org/apache/kyuubi/grpc/server/AbstractKyuubiGrpcFrontendService.scala @@ -0,0 +1,42 @@ +/* + * 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.grpc.server + +import io.grpc.{Channel, Grpc, InsecureChannelCredentials, ManagedChannel} +import org.apache.kyuubi.config.KyuubiConf +import org.apache.kyuubi.grpc.service.AbstractGrpcFrontendService + +abstract class AbstractKyuubiGrpcFrontendService(name: String) + extends AbstractGrpcFrontendService(name) { + + protected var host = "" + protected var port = 0 + + def channel: ManagedChannel + + def startEngine(): (String, Int) + + override def initialize(conf: KyuubiConf): Unit = { + val serverInfo = startEngine() + host = serverInfo._1 + port = serverInfo._2 + super.initialize(conf) + } + + +} diff --git a/kyuubi-grpc-server/src/main/scala/org/apache/kyuubi/grpc/server/KyuubiGrpcBackendService.scala b/kyuubi-grpc-server/src/main/scala/org/apache/kyuubi/grpc/server/KyuubiGrpcBackendService.scala new file mode 100644 index 00000000000..add049324fd --- /dev/null +++ b/kyuubi-grpc-server/src/main/scala/org/apache/kyuubi/grpc/server/KyuubiGrpcBackendService.scala @@ -0,0 +1,28 @@ +/* + * 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.grpc.server + +import org.apache.kyuubi.grpc.service.AbstractGrpcBackendService +import org.apache.kyuubi.grpc.session.{GrpcSession, GrpcSessionManager} + +class KyuubiGrpcBackendService(name: String) extends AbstractGrpcBackendService(name) { + + def this() = this(classOf[KyuubiGrpcBackendService].getSimpleName) + + override def grpcSessionManager: GrpcSessionManager[_ <: GrpcSession] = _ +} diff --git a/kyuubi-grpc-server/src/main/scala/org/apache/kyuubi/grpc/server/KyuubiGrpcServer.scala b/kyuubi-grpc-server/src/main/scala/org/apache/kyuubi/grpc/server/KyuubiGrpcServer.scala new file mode 100644 index 00000000000..a0f9c40181b --- /dev/null +++ b/kyuubi-grpc-server/src/main/scala/org/apache/kyuubi/grpc/server/KyuubiGrpcServer.scala @@ -0,0 +1,32 @@ +/* + * 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.grpc.server + +import org.apache.kyuubi.Logging +import org.apache.kyuubi.grpc.service.GrpcSeverable + +class KyuubiGrpcServer(name: String) extends GrpcSeverable(name) { + override val backendService: Nothing = _ + override val frontendServices: Seq[Nothing] = _ + + override protected def stopServer(): Unit = ??? +} + +object KyuubiGrpcServer extends Logging{ + +} diff --git a/kyuubi-grpc-server/src/main/scala/org/apache/kyuubi/grpc/server/KyuubiSparkConnectFrontendService.scala b/kyuubi-grpc-server/src/main/scala/org/apache/kyuubi/grpc/server/KyuubiSparkConnectFrontendService.scala new file mode 100644 index 00000000000..05ec99362cf --- /dev/null +++ b/kyuubi-grpc-server/src/main/scala/org/apache/kyuubi/grpc/server/KyuubiSparkConnectFrontendService.scala @@ -0,0 +1,64 @@ +/* + * 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.grpc.server +import scala.jdk.CollectionConverters.collectionAsScalaIterableConverter + +import com.google.protobuf.MessageLite +import io.grpc._ +import org.apache.spark.connect.proto.{ConfigRequest, ConfigResponse, SparkConnectServiceGrpc} +import org.sparkproject.connect.grpc.stub.StreamObserver + +import org.apache.kyuubi.grpc.service.{GrpcBackendService, GrpcFrontendService, GrpcSeverable} +import org.apache.kyuubi.service.Service + +class KyuubiSparkConnectFrontendService(name: String) + extends AbstractKyuubiGrpcFrontendService(name) with SparkConnectServiceGrpc.AsyncService { + + lazy val stub = SparkConnectServiceGrpc.newStub(channel.asInstanceOf[Channel]) + override def channel: ManagedChannel = { + Grpc.newChannelBuilderForAddress( + host, + port, + InsecureChannelCredentials.create()).build() + } + + override def startEngine(): (String, Int) = {} + + override protected def serverHost: Option[String] = Some("localhost") + + override def bindService(): ServerServiceDefinition = { + val serviceDef = SparkConnectServiceGrpc.bindService(this) + val builder = ServerServiceDefinition.builder(serviceDef.getServiceDescriptor.getName) + serviceDef.getMethods.asScala + .asInstanceOf[Iterable[ServerMethodDefinition[MessageLite, MessageLite]]] + .foreach(method => + builder.addMethod( + methodWithCustomMarshallers(method.getMethodDescriptor), + method.getServerCallHandler)) + builder.build() + } + + override val serverable: GrpcSeverable[_ <: GrpcBackendService, _ <: GrpcFrontendService] = null + override val discoveryService: Option[Service] = None + + override def config( + request: ConfigRequest, + responseObserver: StreamObserver[ConfigResponse]): Unit = { + super.config(request, responseObserver) + } +} diff --git a/pom.xml b/pom.xml index dd0266aa8b9..1ed753eb653 100644 --- a/pom.xml +++ b/pom.xml @@ -84,6 +84,7 @@ kyuubi-util-scala kyuubi-zookeeper kyuubi-grpc + kyuubi-grpc-server From 5bf18145ba5efef7b39ca65c5594c313838f32ac Mon Sep 17 00:00:00 2001 From: davidyuan Date: Tue, 4 Jun 2024 18:50:36 +0800 Subject: [PATCH 17/21] update --- .../org/apache/kyuubi/config/KyuubiConf.scala | 13 ++-- kyuubi-grpc-server/pom.xml | 68 +++++++++++++++++-- .../AbstractKyuubiGrpcFrontendService.scala | 12 ++-- .../server/KyuubiGrpcBackendService.scala | 17 ++++- .../kyuubi/grpc/server/KyuubiGrpcServer.scala | 17 ++--- .../KyuubiSparkConnectFrontendService.scala | 20 +++--- .../test/resources/protobuf/test_case.proto | 57 ---------------- .../kyuubi/grpc/client/SimpleRpcClient.scala | 49 ------------- .../event/SimpleOperationEventsManager.scala | 37 ---------- .../event/SimpleSessionEventsManager.scala | 37 ---------- .../operation/SimpleAddOperationImpl.scala | 46 ------------- .../operation/SimpleGrpcOperationImpl.scala | 63 ----------------- .../SimpleGrpcOperationManager.scala | 46 ------------- .../SimpleOpenSessionOperationImpl.scala | 41 ----------- .../grpc/server/GrpcSeverableSuite.scala | 59 ---------------- .../kyuubi/grpc/server/SimpleGrpcServer.scala | 22 ------ .../service/SimpleGrpcBackendService.scala | 51 -------------- .../service/SimpleGrpcFrontendService.scala | 65 ------------------ .../grpc/service/SimpleGrpcSeverable.scala | 31 --------- .../grpc/session/SimpleGrpcSessionImpl.scala | 68 ------------------- .../session/SimpleGrpcSessionManager.scala | 33 --------- 21 files changed, 108 insertions(+), 744 deletions(-) delete mode 100644 kyuubi-grpc/src/test/resources/protobuf/test_case.proto delete mode 100644 kyuubi-grpc/src/test/scala/org/apache/kyuubi/grpc/client/SimpleRpcClient.scala delete mode 100644 kyuubi-grpc/src/test/scala/org/apache/kyuubi/grpc/event/SimpleOperationEventsManager.scala delete mode 100644 kyuubi-grpc/src/test/scala/org/apache/kyuubi/grpc/event/SimpleSessionEventsManager.scala delete mode 100644 kyuubi-grpc/src/test/scala/org/apache/kyuubi/grpc/operation/SimpleAddOperationImpl.scala delete mode 100644 kyuubi-grpc/src/test/scala/org/apache/kyuubi/grpc/operation/SimpleGrpcOperationImpl.scala delete mode 100644 kyuubi-grpc/src/test/scala/org/apache/kyuubi/grpc/operation/SimpleGrpcOperationManager.scala delete mode 100644 kyuubi-grpc/src/test/scala/org/apache/kyuubi/grpc/operation/SimpleOpenSessionOperationImpl.scala delete mode 100644 kyuubi-grpc/src/test/scala/org/apache/kyuubi/grpc/server/GrpcSeverableSuite.scala delete mode 100644 kyuubi-grpc/src/test/scala/org/apache/kyuubi/grpc/server/SimpleGrpcServer.scala delete mode 100644 kyuubi-grpc/src/test/scala/org/apache/kyuubi/grpc/service/SimpleGrpcBackendService.scala delete mode 100644 kyuubi-grpc/src/test/scala/org/apache/kyuubi/grpc/service/SimpleGrpcFrontendService.scala delete mode 100644 kyuubi-grpc/src/test/scala/org/apache/kyuubi/grpc/service/SimpleGrpcSeverable.scala delete mode 100644 kyuubi-grpc/src/test/scala/org/apache/kyuubi/grpc/session/SimpleGrpcSessionImpl.scala delete mode 100644 kyuubi-grpc/src/test/scala/org/apache/kyuubi/grpc/session/SimpleGrpcSessionManager.scala 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 4cd2302f3e9..b01f5d44a45 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 @@ -1064,12 +1064,13 @@ object KyuubiConf { .transformToLowerCase .createWithDefault(SaslQOP.AUTH.toString) - val GRPC_FRONTEND_BIND_HOST: OptionalConfigEntry[String] = buildConf("kyuubi.grpc.frontend.bind.host") - .doc("Hostname or IP of the machine on which to run the grpc frontend services.") - .version("1.0.0") - .serverOnly - .stringConf - .createOptional + val GRPC_FRONTEND_BIND_HOST: OptionalConfigEntry[String] = + buildConf("kyuubi.grpc.frontend.bind.host") + .doc("Hostname or IP of the machine on which to run the grpc frontend services.") + .version("1.0.0") + .serverOnly + .stringConf + .createOptional val GRPC_FRONTEND_SPARK_CONNECT_HOST: ConfigEntry[Option[String]] = buildConf("kyuubi.grpc.frontend.spark.connect.host") diff --git a/kyuubi-grpc-server/pom.xml b/kyuubi-grpc-server/pom.xml index 36052f0de53..c14f832e6fb 100644 --- a/kyuubi-grpc-server/pom.xml +++ b/kyuubi-grpc-server/pom.xml @@ -60,19 +60,73 @@ test + + org.apache.spark + spark-sql_${scala.binary.version} + + + + + + + org.apache.spark + spark-connect-common_2.12 + 3.5.1 + + org.apache.spark spark-connect_2.12 - 3.5.0 - - - io.grpc - grpc-all - - + 3.5.1 + + io.grpc + grpc-core + + + io.grpc + grpc-stub + + + + javax.servlet + javax.servlet-api + + + + + + + org.apache.maven.plugins + maven-shade-plugin + 3.2.4 + + + package + + shade + + + + + org.apache.spark:spark-connect-common_2.12 + + + + + org.apache.spark.connect.proto + shade.org.apache.spark.connect.proto + + + + + + + + +
    diff --git a/kyuubi-grpc-server/src/main/scala/org/apache/kyuubi/grpc/server/AbstractKyuubiGrpcFrontendService.scala b/kyuubi-grpc-server/src/main/scala/org/apache/kyuubi/grpc/server/AbstractKyuubiGrpcFrontendService.scala index fc9ded961c6..47452c4e2d6 100644 --- a/kyuubi-grpc-server/src/main/scala/org/apache/kyuubi/grpc/server/AbstractKyuubiGrpcFrontendService.scala +++ b/kyuubi-grpc-server/src/main/scala/org/apache/kyuubi/grpc/server/AbstractKyuubiGrpcFrontendService.scala @@ -17,15 +17,16 @@ package org.apache.kyuubi.grpc.server -import io.grpc.{Channel, Grpc, InsecureChannelCredentials, ManagedChannel} +import io.grpc.ManagedChannel + import org.apache.kyuubi.config.KyuubiConf import org.apache.kyuubi.grpc.service.AbstractGrpcFrontendService -abstract class AbstractKyuubiGrpcFrontendService(name: String) +abstract class AbstractKyuubiGrpcFrontendService(grpcSeverable: KyuubiGrpcSeverable, name: String) extends AbstractGrpcFrontendService(name) { - protected var host = "" - protected var port = 0 + var host = "" + var port = 0 def channel: ManagedChannel @@ -38,5 +39,8 @@ abstract class AbstractKyuubiGrpcFrontendService(name: String) super.initialize(conf) } + override val serverable: KyuubiGrpcSeverable = grpcSeverable + + def grpcBe: KyuubiGrpcBackendService = be.asInstanceOf[KyuubiGrpcBackendService] } diff --git a/kyuubi-grpc-server/src/main/scala/org/apache/kyuubi/grpc/server/KyuubiGrpcBackendService.scala b/kyuubi-grpc-server/src/main/scala/org/apache/kyuubi/grpc/server/KyuubiGrpcBackendService.scala index add049324fd..2af91b3caaa 100644 --- a/kyuubi-grpc-server/src/main/scala/org/apache/kyuubi/grpc/server/KyuubiGrpcBackendService.scala +++ b/kyuubi-grpc-server/src/main/scala/org/apache/kyuubi/grpc/server/KyuubiGrpcBackendService.scala @@ -17,12 +17,25 @@ package org.apache.kyuubi.grpc.server +import io.grpc.ManagedChannel +import io.grpc.stub.StreamObserver +import org.apache.spark.connect.proto._ import org.apache.kyuubi.grpc.service.AbstractGrpcBackendService -import org.apache.kyuubi.grpc.session.{GrpcSession, GrpcSessionManager} +import org.apache.kyuubi.grpc.session.{GrpcSessionManager, KyuubiGrpcSession, KyuubiGrpcSessionManager, SessionKey} class KyuubiGrpcBackendService(name: String) extends AbstractGrpcBackendService(name) { + def config( + sessionKey: SessionKey, + request: ConfigRequest, + responseObserver: StreamObserver[ConfigResponse], + channel: ManagedChannel): Unit = { + grpcSessionManager.openSession(sessionKey) + .config(request, responseObserver, channel) + } + def this() = this(classOf[KyuubiGrpcBackendService].getSimpleName) - override def grpcSessionManager: GrpcSessionManager[_ <: GrpcSession] = _ + override def grpcSessionManager: GrpcSessionManager[KyuubiGrpcSession] = + new KyuubiGrpcSessionManager() } diff --git a/kyuubi-grpc-server/src/main/scala/org/apache/kyuubi/grpc/server/KyuubiGrpcServer.scala b/kyuubi-grpc-server/src/main/scala/org/apache/kyuubi/grpc/server/KyuubiGrpcServer.scala index a0f9c40181b..c46a8eb8c32 100644 --- a/kyuubi-grpc-server/src/main/scala/org/apache/kyuubi/grpc/server/KyuubiGrpcServer.scala +++ b/kyuubi-grpc-server/src/main/scala/org/apache/kyuubi/grpc/server/KyuubiGrpcServer.scala @@ -17,16 +17,13 @@ package org.apache.kyuubi.grpc.server -import org.apache.kyuubi.Logging -import org.apache.kyuubi.grpc.service.GrpcSeverable +class KyuubiGrpcServer(name: String) + extends KyuubiGrpcSeverable(name) { + override val backendService: KyuubiGrpcBackendService = new KyuubiGrpcBackendService() + override val frontendServices: Seq[AbstractKyuubiGrpcFrontendService] = Seq.empty -class KyuubiGrpcServer(name: String) extends GrpcSeverable(name) { - override val backendService: Nothing = _ - override val frontendServices: Seq[Nothing] = _ - - override protected def stopServer(): Unit = ??? + def frontendInfo(): (String, Int) = { + (frontendServices.head.host, frontendServices.head.port) + } } -object KyuubiGrpcServer extends Logging{ - -} diff --git a/kyuubi-grpc-server/src/main/scala/org/apache/kyuubi/grpc/server/KyuubiSparkConnectFrontendService.scala b/kyuubi-grpc-server/src/main/scala/org/apache/kyuubi/grpc/server/KyuubiSparkConnectFrontendService.scala index 05ec99362cf..5587cb3108c 100644 --- a/kyuubi-grpc-server/src/main/scala/org/apache/kyuubi/grpc/server/KyuubiSparkConnectFrontendService.scala +++ b/kyuubi-grpc-server/src/main/scala/org/apache/kyuubi/grpc/server/KyuubiSparkConnectFrontendService.scala @@ -16,20 +16,21 @@ */ package org.apache.kyuubi.grpc.server + import scala.jdk.CollectionConverters.collectionAsScalaIterableConverter import com.google.protobuf.MessageLite import io.grpc._ -import org.apache.spark.connect.proto.{ConfigRequest, ConfigResponse, SparkConnectServiceGrpc} -import org.sparkproject.connect.grpc.stub.StreamObserver +import io.grpc.stub.StreamObserver +import org.apache.spark.connect.proto._ -import org.apache.kyuubi.grpc.service.{GrpcBackendService, GrpcFrontendService, GrpcSeverable} +import org.apache.kyuubi.grpc.session.SessionKey import org.apache.kyuubi.service.Service -class KyuubiSparkConnectFrontendService(name: String) - extends AbstractKyuubiGrpcFrontendService(name) with SparkConnectServiceGrpc.AsyncService { +abstract class KyuubiSparkConnectFrontendService(grpcSeverable: KyuubiGrpcSeverable, name: String) + extends AbstractKyuubiGrpcFrontendService(grpcSeverable, name) + with SparkConnectServiceGrpc.AsyncService { - lazy val stub = SparkConnectServiceGrpc.newStub(channel.asInstanceOf[Channel]) override def channel: ManagedChannel = { Grpc.newChannelBuilderForAddress( host, @@ -37,8 +38,6 @@ class KyuubiSparkConnectFrontendService(name: String) InsecureChannelCredentials.create()).build() } - override def startEngine(): (String, Int) = {} - override protected def serverHost: Option[String] = Some("localhost") override def bindService(): ServerServiceDefinition = { @@ -53,12 +52,13 @@ class KyuubiSparkConnectFrontendService(name: String) builder.build() } - override val serverable: GrpcSeverable[_ <: GrpcBackendService, _ <: GrpcFrontendService] = null override val discoveryService: Option[Service] = None override def config( request: ConfigRequest, responseObserver: StreamObserver[ConfigResponse]): Unit = { - super.config(request, responseObserver) + val sessionKey = new SessionKey(request.getUserContext.getUserId, request.getSessionId) + grpcBe.config(sessionKey, request, responseObserver, channel) } + } diff --git a/kyuubi-grpc/src/test/resources/protobuf/test_case.proto b/kyuubi-grpc/src/test/resources/protobuf/test_case.proto deleted file mode 100644 index 61f02925009..00000000000 --- a/kyuubi-grpc/src/test/resources/protobuf/test_case.proto +++ /dev/null @@ -1,57 +0,0 @@ -/* - * 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. - */ - -syntax = 'proto3'; - -package kyuubi.grpc; - -option java_multiple_files = true; -option java_package = "org.apache.kyuubi.grpc.proto"; -option go_package = "internal/generated"; - -message TestOpenSessionRequest { - string user_id = 1; - string session_id = 2; -} - -message TestAddRequest { - string user_id = 1; - string session_id = 2; - string operation_id = 3; - int64 first_num = 4; - int64 second_num = 5; -} - -message TestOpenSessionResponse { - string session_id = 1; - string server_side_session_id = 2; - string session_info = 3; -} - -message TestAddResponse { - string session_id = 1; - string server_side_session_id = 2; - string operation_id = 3; - int64 result = 4; -} - -service GrpcTestService { - rpc TestOpenSession(TestOpenSessionRequest) returns (TestOpenSessionResponse) {} - - rpc TestAdd(TestAddRequest) returns (TestAddResponse) {} - -} \ No newline at end of file diff --git a/kyuubi-grpc/src/test/scala/org/apache/kyuubi/grpc/client/SimpleRpcClient.scala b/kyuubi-grpc/src/test/scala/org/apache/kyuubi/grpc/client/SimpleRpcClient.scala deleted file mode 100644 index e131a608720..00000000000 --- a/kyuubi-grpc/src/test/scala/org/apache/kyuubi/grpc/client/SimpleRpcClient.scala +++ /dev/null @@ -1,49 +0,0 @@ -/* - * 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.grpc.client - -import java.util.UUID - -import io.grpc.ManagedChannel - -import org.apache.kyuubi.grpc.proto.{GrpcTestServiceGrpc, TestAddRequest, TestAddResponse, TestOpenSessionRequest, TestOpenSessionResponse} - -class SimpleRpcClient(val channel: ManagedChannel) { - private val DEFAULT_USER_ID = "kyuubi_grpc_test" - private val sessionId: String = UUID.randomUUID().toString - private val stub = GrpcTestServiceGrpc.newBlockingStub(channel) - - def openSession(): TestOpenSessionResponse = { - val request = TestOpenSessionRequest.newBuilder() - .setUserId(DEFAULT_USER_ID) - .setSessionId(sessionId) - .build() - stub.testOpenSession(request) - } - - def testAdd(firstNum: Int, secondNum: Int): TestAddResponse = { - val request = TestAddRequest.newBuilder() - .setUserId(DEFAULT_USER_ID) - .setSessionId(sessionId) - .setFirstNum(firstNum) - .setSecondNum(secondNum) - .build() - stub.testAdd(request) - } - -} diff --git a/kyuubi-grpc/src/test/scala/org/apache/kyuubi/grpc/event/SimpleOperationEventsManager.scala b/kyuubi-grpc/src/test/scala/org/apache/kyuubi/grpc/event/SimpleOperationEventsManager.scala deleted file mode 100644 index a2329ac9a31..00000000000 --- a/kyuubi-grpc/src/test/scala/org/apache/kyuubi/grpc/event/SimpleOperationEventsManager.scala +++ /dev/null @@ -1,37 +0,0 @@ -/* - * 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.grpc.event - -import org.apache.kyuubi.Logging -import org.apache.kyuubi.grpc.events.OperationEventsManager -import org.apache.kyuubi.grpc.operation.GrpcOperation -import org.apache.kyuubi.grpc.utils.Clock - -class SimpleOperationEventsManager(operation: GrpcOperation, clock: Clock) - extends OperationEventsManager(operation, clock) with Logging { - - override def postStarted(): Unit = { - super.postStarted() - info("Operation Event: post Started") - } - - override def postClosed(): Unit = { - info("Operation Event: post Closed") - super.postClosed() - } -} diff --git a/kyuubi-grpc/src/test/scala/org/apache/kyuubi/grpc/event/SimpleSessionEventsManager.scala b/kyuubi-grpc/src/test/scala/org/apache/kyuubi/grpc/event/SimpleSessionEventsManager.scala deleted file mode 100644 index a03abc2b0d3..00000000000 --- a/kyuubi-grpc/src/test/scala/org/apache/kyuubi/grpc/event/SimpleSessionEventsManager.scala +++ /dev/null @@ -1,37 +0,0 @@ -/* - * 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.grpc.event - -import org.apache.kyuubi.Logging -import org.apache.kyuubi.grpc.events.SessionEventsManager -import org.apache.kyuubi.grpc.session.SimpleGrpcSessionImpl -import org.apache.kyuubi.grpc.utils.Clock - -class SimpleSessionEventsManager(session: SimpleGrpcSessionImpl, clock: Clock) - extends SessionEventsManager(session, clock) with Logging { - - override def postStarted(): Unit = { - super.postStarted() - info("Session Event: post Started") - } - - override def postClosed(): Unit = { - info("Session Event: post Closed") - super.postClosed() - } -} diff --git a/kyuubi-grpc/src/test/scala/org/apache/kyuubi/grpc/operation/SimpleAddOperationImpl.scala b/kyuubi-grpc/src/test/scala/org/apache/kyuubi/grpc/operation/SimpleAddOperationImpl.scala deleted file mode 100644 index 28ab5cdee75..00000000000 --- a/kyuubi-grpc/src/test/scala/org/apache/kyuubi/grpc/operation/SimpleAddOperationImpl.scala +++ /dev/null @@ -1,46 +0,0 @@ -/* - * 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.grpc.operation - -import io.grpc.stub.StreamObserver - -import org.apache.kyuubi.grpc.proto.{TestAddRequest, TestAddResponse} -import org.apache.kyuubi.grpc.session.SimpleGrpcSessionImpl - -class SimpleAddOperationImpl( - grpcSession: SimpleGrpcSessionImpl, - shouldFail: Boolean, - request: TestAddRequest, - responseObserver: StreamObserver[TestAddResponse]) - extends SimpleGrpcOperationImpl(grpcSession, shouldFail) { - - override protected def key: OperationKey = OperationKey(grpcSession.sessionKey) - - override def runInternal(): Unit = { - super.runInternal() - val result = request.getFirstNum + request.getSecondNum - val builder = TestAddResponse.newBuilder() - .setOperationId(operationKey.operationId) - .setSessionId(operationKey.sessionId) - .setServerSideSessionId(grpcSession.serverSessionId) - .setResult(result) - responseObserver.onNext(builder.build()) - responseObserver.onCompleted() - } - -} diff --git a/kyuubi-grpc/src/test/scala/org/apache/kyuubi/grpc/operation/SimpleGrpcOperationImpl.scala b/kyuubi-grpc/src/test/scala/org/apache/kyuubi/grpc/operation/SimpleGrpcOperationImpl.scala deleted file mode 100644 index ae57c68cc1c..00000000000 --- a/kyuubi-grpc/src/test/scala/org/apache/kyuubi/grpc/operation/SimpleGrpcOperationImpl.scala +++ /dev/null @@ -1,63 +0,0 @@ -/* - * 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.grpc.operation - -import org.apache.kyuubi.{KyuubiSQLException, Logging} -import org.apache.kyuubi.grpc.event.SimpleOperationEventsManager -import org.apache.kyuubi.grpc.events.OperationEventsManager -import org.apache.kyuubi.grpc.session.SimpleGrpcSessionImpl -import org.apache.kyuubi.grpc.utils.SystemClock -import org.apache.kyuubi.operation.log.OperationLog - -class SimpleGrpcOperationImpl( - grpcSession: SimpleGrpcSessionImpl, - shouldFail: Boolean = false) - extends AbstractGrpcOperation[SimpleGrpcSessionImpl](grpcSession) with Logging { - - override def operationEventsManager: OperationEventsManager = - new SimpleOperationEventsManager(this, new SystemClock()) - - override def runInternal(): Unit = { - if (shouldFail) { - val exception = KyuubiSQLException("noop operation err") - setOperationException(exception) - operationEventsManager.postFailed(exception.getMessage) - } - } - - override def beforeRun(): Unit = { - operationEventsManager.postStarted() - } - - override def close(): Unit = { - operationEventsManager.postClosed() - } - - override def afterRun(): Unit = { - info("afterRun") - } - - override def interrupt(): Unit = { - info("interrupt") - } - - override def isTimedOut: Boolean = false - override def getOperationLog: Option[OperationLog] = None - - override protected def key: OperationKey = OperationKey(grpcSession.sessionKey) -} diff --git a/kyuubi-grpc/src/test/scala/org/apache/kyuubi/grpc/operation/SimpleGrpcOperationManager.scala b/kyuubi-grpc/src/test/scala/org/apache/kyuubi/grpc/operation/SimpleGrpcOperationManager.scala deleted file mode 100644 index a4ce69d3fd1..00000000000 --- a/kyuubi-grpc/src/test/scala/org/apache/kyuubi/grpc/operation/SimpleGrpcOperationManager.scala +++ /dev/null @@ -1,46 +0,0 @@ -/* - * 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.grpc.operation - -import io.grpc.stub.StreamObserver - -import org.apache.kyuubi.grpc.proto.{TestAddRequest, TestAddResponse, TestOpenSessionRequest, TestOpenSessionResponse} -import org.apache.kyuubi.grpc.session.SimpleGrpcSessionImpl - -class SimpleGrpcOperationManager - extends GrpcOperationManager("SimpleGrpcOperationManager") { - - def newSimpleOpenSessionOperation( - session: SimpleGrpcSessionImpl, - shouldFail: Boolean, - request: TestOpenSessionRequest, - responseObserver: StreamObserver[TestOpenSessionResponse]): GrpcOperation = { - val operation = - new SimpleOpenSessionOperationImpl(session, shouldFail, request, responseObserver) - addOperation(operation) - } - - def newSimpleAddOperation( - session: SimpleGrpcSessionImpl, - shouldFail: Boolean, - request: TestAddRequest, - responseObserver: StreamObserver[TestAddResponse]): GrpcOperation = { - val operation = new SimpleAddOperationImpl(session, shouldFail, request, responseObserver) - addOperation(operation) - } -} diff --git a/kyuubi-grpc/src/test/scala/org/apache/kyuubi/grpc/operation/SimpleOpenSessionOperationImpl.scala b/kyuubi-grpc/src/test/scala/org/apache/kyuubi/grpc/operation/SimpleOpenSessionOperationImpl.scala deleted file mode 100644 index b349e941645..00000000000 --- a/kyuubi-grpc/src/test/scala/org/apache/kyuubi/grpc/operation/SimpleOpenSessionOperationImpl.scala +++ /dev/null @@ -1,41 +0,0 @@ -/* - * 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.grpc.operation - -import io.grpc.stub.StreamObserver - -import org.apache.kyuubi.grpc.proto.{TestOpenSessionRequest, TestOpenSessionResponse} -import org.apache.kyuubi.grpc.session.SimpleGrpcSessionImpl - -class SimpleOpenSessionOperationImpl( - grpcSession: SimpleGrpcSessionImpl, - shouldFail: Boolean, - request: TestOpenSessionRequest, - responseObserver: StreamObserver[TestOpenSessionResponse]) - extends SimpleGrpcOperationImpl(grpcSession, shouldFail) { - - override protected def key: OperationKey = OperationKey(grpcSession.sessionKey) - - override def runInternal(): Unit = { - super.runInternal() - val builder = TestOpenSessionResponse.newBuilder() - .setSessionId(grpcSession.sessionKey.sessionId) - responseObserver.onNext(builder.build()) - responseObserver.onCompleted() - } -} diff --git a/kyuubi-grpc/src/test/scala/org/apache/kyuubi/grpc/server/GrpcSeverableSuite.scala b/kyuubi-grpc/src/test/scala/org/apache/kyuubi/grpc/server/GrpcSeverableSuite.scala deleted file mode 100644 index fc49f190066..00000000000 --- a/kyuubi-grpc/src/test/scala/org/apache/kyuubi/grpc/server/GrpcSeverableSuite.scala +++ /dev/null @@ -1,59 +0,0 @@ -/* - * 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.grpc.server - -import io.grpc.{Grpc, InsecureChannelCredentials} - -import org.apache.kyuubi.KyuubiFunSuite -import org.apache.kyuubi.config.KyuubiConf -import org.apache.kyuubi.grpc.client.SimpleRpcClient - -class GrpcSeverableSuite extends KyuubiFunSuite { - - test("test openSession") { - val conf = KyuubiConf().set(KyuubiConf.ENGINE_SPARK_CONNECT_GRPC_BINDING_PORT, 10023) - val server = new SimpleGrpcServer - server.initialize(conf) - server.start() - val channel = - Grpc.newChannelBuilderForAddress( - "127.0.0.1", - 10023, - InsecureChannelCredentials.create()).build() - val client = new SimpleRpcClient(channel) - client.openSession() - server.stop() - } - - test("test add") { - val conf = KyuubiConf().set(KyuubiConf.ENGINE_SPARK_CONNECT_GRPC_BINDING_PORT, 10023) - val server = new SimpleGrpcServer - server.initialize(conf) - server.start() - val channel = - Grpc.newChannelBuilderForAddress( - "127.0.0.1", - 10023, - InsecureChannelCredentials.create()).build() - val client = new SimpleRpcClient(channel) - val response = client.testAdd(1, 2) - assert(response.getResult == 3) - server.stop() - } - -} diff --git a/kyuubi-grpc/src/test/scala/org/apache/kyuubi/grpc/server/SimpleGrpcServer.scala b/kyuubi-grpc/src/test/scala/org/apache/kyuubi/grpc/server/SimpleGrpcServer.scala deleted file mode 100644 index 8dd569da862..00000000000 --- a/kyuubi-grpc/src/test/scala/org/apache/kyuubi/grpc/server/SimpleGrpcServer.scala +++ /dev/null @@ -1,22 +0,0 @@ -/* - * 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.grpc.server - -import org.apache.kyuubi.grpc.service.SimpleGrpcSeverable - -class SimpleGrpcServer extends SimpleGrpcSeverable {} diff --git a/kyuubi-grpc/src/test/scala/org/apache/kyuubi/grpc/service/SimpleGrpcBackendService.scala b/kyuubi-grpc/src/test/scala/org/apache/kyuubi/grpc/service/SimpleGrpcBackendService.scala deleted file mode 100644 index fedab4fd817..00000000000 --- a/kyuubi-grpc/src/test/scala/org/apache/kyuubi/grpc/service/SimpleGrpcBackendService.scala +++ /dev/null @@ -1,51 +0,0 @@ -/* - * 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.grpc.service - -import io.grpc.stub.StreamObserver - -import org.apache.kyuubi.KyuubiException -import org.apache.kyuubi.grpc.proto.{TestAddRequest, TestAddResponse, TestOpenSessionRequest, TestOpenSessionResponse} -import org.apache.kyuubi.grpc.session.{SessionKey, SimpleGrpcSessionManager} - -class SimpleGrpcBackendService extends AbstractGrpcBackendService("simpleTest") { - override def grpcSessionManager: SimpleGrpcSessionManager = new SimpleGrpcSessionManager() - - override def start(): Unit = { - if (conf.getOption("kyuubi.test.backend.should.fail").exists(_.toBoolean)) { - throw new KyuubiException("should fail backend") - } - super.start() - } - - def openSessionTesr( - key: SessionKey, - request: TestOpenSessionRequest, - responseObserver: StreamObserver[TestOpenSessionResponse]): Unit = { - grpcSessionManager.openSession(key) - .openSession(key, request, responseObserver) - } - - def add( - key: SessionKey, - request: TestAddRequest, - responseObserver: StreamObserver[TestAddResponse]): Unit = { - grpcSessionManager.openSession(key) - .add(key, request, responseObserver) - } -} diff --git a/kyuubi-grpc/src/test/scala/org/apache/kyuubi/grpc/service/SimpleGrpcFrontendService.scala b/kyuubi-grpc/src/test/scala/org/apache/kyuubi/grpc/service/SimpleGrpcFrontendService.scala deleted file mode 100644 index cc6f1e7add5..00000000000 --- a/kyuubi-grpc/src/test/scala/org/apache/kyuubi/grpc/service/SimpleGrpcFrontendService.scala +++ /dev/null @@ -1,65 +0,0 @@ -/* - * 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.grpc.service - -import scala.jdk.CollectionConverters.collectionAsScalaIterableConverter - -import com.google.protobuf.MessageLite -import io.grpc.{ServerMethodDefinition, ServerServiceDefinition} -import io.grpc.stub.StreamObserver - -import org.apache.kyuubi.grpc.proto.{GrpcTestServiceGrpc, TestAddRequest, TestAddResponse, TestOpenSessionRequest, TestOpenSessionResponse} -import org.apache.kyuubi.grpc.session.SessionKey -import org.apache.kyuubi.service.Service - -class SimpleGrpcFrontendService(grpcSeverable: SimpleGrpcSeverable) - extends AbstractGrpcFrontendService("SimpleGrpcFrontendService") - with GrpcTestServiceGrpc.AsyncService { - - private def grpcBe = be.asInstanceOf[SimpleGrpcBackendService] - override protected def serverHost: Option[String] = Some("localhost") - - override def bindService(): ServerServiceDefinition = { - val serviceDef = GrpcTestServiceGrpc.bindService(this) - val builder = ServerServiceDefinition.builder(serviceDef.getServiceDescriptor.getName) - serviceDef.getMethods.asScala - .asInstanceOf[Iterable[ServerMethodDefinition[MessageLite, MessageLite]]] - .foreach(method => - builder.addMethod( - methodWithCustomMarshallers(method.getMethodDescriptor), - method.getServerCallHandler)) - builder.build() - } - - override def testAdd( - request: TestAddRequest, - responseObserver: StreamObserver[TestAddResponse]): Unit = { - val key = new SessionKey(request.getUserId, request.getSessionId) - grpcBe.add(key, request, responseObserver) - } - - override def testOpenSession( - request: TestOpenSessionRequest, - responseObserver: StreamObserver[TestOpenSessionResponse]): Unit = { - val key = new SessionKey(request.getUserId, request.getSessionId) - grpcBe.openSessionTesr(key, request, responseObserver) - } - - override val serverable: SimpleGrpcSeverable = grpcSeverable - override val discoveryService: Option[Service] = None -} diff --git a/kyuubi-grpc/src/test/scala/org/apache/kyuubi/grpc/service/SimpleGrpcSeverable.scala b/kyuubi-grpc/src/test/scala/org/apache/kyuubi/grpc/service/SimpleGrpcSeverable.scala deleted file mode 100644 index e8157a62385..00000000000 --- a/kyuubi-grpc/src/test/scala/org/apache/kyuubi/grpc/service/SimpleGrpcSeverable.scala +++ /dev/null @@ -1,31 +0,0 @@ -/* - * 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.grpc.service - -class SimpleGrpcSeverable - extends GrpcSeverable[SimpleGrpcBackendService, SimpleGrpcFrontendService]( - "SimpleGrpcSeverable") { - - override val backendService: SimpleGrpcBackendService = new SimpleGrpcBackendService - override val frontendServices: Seq[SimpleGrpcFrontendService] = - Seq(new SimpleGrpcFrontendService(this)) - - override protected def stopServer(): Unit = { - info(s"stop") - } -} diff --git a/kyuubi-grpc/src/test/scala/org/apache/kyuubi/grpc/session/SimpleGrpcSessionImpl.scala b/kyuubi-grpc/src/test/scala/org/apache/kyuubi/grpc/session/SimpleGrpcSessionImpl.scala deleted file mode 100644 index 13a9f6816b6..00000000000 --- a/kyuubi-grpc/src/test/scala/org/apache/kyuubi/grpc/session/SimpleGrpcSessionImpl.scala +++ /dev/null @@ -1,68 +0,0 @@ -/* - * 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.grpc.session - -import scala.util.Random - -import io.grpc.stub.StreamObserver - -import org.apache.kyuubi.grpc.event.SimpleSessionEventsManager -import org.apache.kyuubi.grpc.events.SessionEventsManager -import org.apache.kyuubi.grpc.operation.{GrpcOperation, OperationKey} -import org.apache.kyuubi.grpc.proto.{TestAddRequest, TestAddResponse, TestOpenSessionRequest, TestOpenSessionResponse} -import org.apache.kyuubi.grpc.utils.SystemClock - -class SimpleGrpcSessionImpl( - userId: String, - sessionManager: SimpleGrpcSessionManager) - extends AbstractGrpcSession(userId) { - override def name: Option[String] = Some("SimpleGrpcSessionImpl") - - override def serverSessionId: String = Random.nextString(10) - - override def sessionEventsManager: SessionEventsManager = - new SimpleSessionEventsManager(this, new SystemClock) - - def openSession( - key: SessionKey, - request: TestOpenSessionRequest, - responseObserver: StreamObserver[TestOpenSessionResponse]): Unit = { - val operation = sessionManager.grpcOperationManager - .newSimpleOpenSessionOperation(this, false, request, responseObserver) - runGrpcOperation(operation) - } - - def add( - key: SessionKey, - request: TestAddRequest, - responseObserver: StreamObserver[TestAddResponse]): Unit = { - val operation = sessionManager.grpcOperationManager - .newSimpleAddOperation(this, false, request, responseObserver) - runGrpcOperation(operation) - } - - override def getOperation(operationKey: OperationKey): GrpcOperation = - sessionManager.grpcOperationManager.getOperation(operationKey) - - override def sessionManager: GrpcSessionManager[SimpleGrpcSessionImpl] = sessionManager - - override def close(): Unit = { - sessionEventsManager.postClosed() - super.close() - } -} diff --git a/kyuubi-grpc/src/test/scala/org/apache/kyuubi/grpc/session/SimpleGrpcSessionManager.scala b/kyuubi-grpc/src/test/scala/org/apache/kyuubi/grpc/session/SimpleGrpcSessionManager.scala deleted file mode 100644 index 72de94e991f..00000000000 --- a/kyuubi-grpc/src/test/scala/org/apache/kyuubi/grpc/session/SimpleGrpcSessionManager.scala +++ /dev/null @@ -1,33 +0,0 @@ -/* - * 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.grpc.session - -import org.apache.kyuubi.grpc.operation.SimpleGrpcOperationManager - -class SimpleGrpcSessionManager - extends GrpcSessionManager[SimpleGrpcSessionImpl]("simpleTest") { - override protected def isServer: Boolean = true - - override def grpcOperationManager: SimpleGrpcOperationManager = new SimpleGrpcOperationManager() - - override def getOrCreateSession(key: SessionKey): SimpleGrpcSessionImpl = { - val session = new SimpleGrpcSessionImpl(key.userId, this) - session - } - -} From e031c8c9e7704948a6b83fa42653640be658e7fc Mon Sep 17 00:00:00 2001 From: davidyuan Date: Tue, 4 Jun 2024 18:51:31 +0800 Subject: [PATCH 18/21] update --- .../KyuubiGrpcOperationEventsManager.scala | 37 +++++++++++++ .../KyuubiGrpcSessionEventsManager.scala | 37 +++++++++++++ .../grpc/operation/KyuubiGrpcOperation.scala | 54 ++++++++++++++++++ .../KyuubiGrpcOperationManager.scala | 39 +++++++++++++ .../kyuubi/grpc/operation/spark/Config.scala | 35 ++++++++++++ .../spark/KyuubiSparkGrpcOperation.scala | 31 +++++++++++ .../grpc/server/KyuubiGrpcSeverable.scala | 28 ++++++++++ .../grpc/session/KyuubiGrpcSession.scala | 55 +++++++++++++++++++ .../session/KyuubiGrpcSessionManager.scala | 33 +++++++++++ .../spark/sql/connect/SparkConnectSuite.scala | 46 ++++++++++++++++ .../spark/sql/connect/SparkConnectUtil.scala | 39 +++++++++++++ .../sql/connect/TestFrontendService.scala | 29 ++++++++++ .../apache/spark/sql/connect/TestServer.scala | 27 +++++++++ 13 files changed, 490 insertions(+) create mode 100644 kyuubi-grpc-server/src/main/scala/org/apache/kyuubi/grpc/event/KyuubiGrpcOperationEventsManager.scala create mode 100644 kyuubi-grpc-server/src/main/scala/org/apache/kyuubi/grpc/event/KyuubiGrpcSessionEventsManager.scala create mode 100644 kyuubi-grpc-server/src/main/scala/org/apache/kyuubi/grpc/operation/KyuubiGrpcOperation.scala create mode 100644 kyuubi-grpc-server/src/main/scala/org/apache/kyuubi/grpc/operation/KyuubiGrpcOperationManager.scala create mode 100644 kyuubi-grpc-server/src/main/scala/org/apache/kyuubi/grpc/operation/spark/Config.scala create mode 100644 kyuubi-grpc-server/src/main/scala/org/apache/kyuubi/grpc/operation/spark/KyuubiSparkGrpcOperation.scala create mode 100644 kyuubi-grpc-server/src/main/scala/org/apache/kyuubi/grpc/server/KyuubiGrpcSeverable.scala create mode 100644 kyuubi-grpc-server/src/main/scala/org/apache/kyuubi/grpc/session/KyuubiGrpcSession.scala create mode 100644 kyuubi-grpc-server/src/main/scala/org/apache/kyuubi/grpc/session/KyuubiGrpcSessionManager.scala create mode 100644 kyuubi-grpc-server/src/test/scala/org/apache/spark/sql/connect/SparkConnectSuite.scala create mode 100644 kyuubi-grpc-server/src/test/scala/org/apache/spark/sql/connect/SparkConnectUtil.scala create mode 100644 kyuubi-grpc-server/src/test/scala/org/apache/spark/sql/connect/TestFrontendService.scala create mode 100644 kyuubi-grpc-server/src/test/scala/org/apache/spark/sql/connect/TestServer.scala diff --git a/kyuubi-grpc-server/src/main/scala/org/apache/kyuubi/grpc/event/KyuubiGrpcOperationEventsManager.scala b/kyuubi-grpc-server/src/main/scala/org/apache/kyuubi/grpc/event/KyuubiGrpcOperationEventsManager.scala new file mode 100644 index 00000000000..6b9bb05de1b --- /dev/null +++ b/kyuubi-grpc-server/src/main/scala/org/apache/kyuubi/grpc/event/KyuubiGrpcOperationEventsManager.scala @@ -0,0 +1,37 @@ +/* + * 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.grpc.event + +import org.apache.kyuubi.Logging +import org.apache.kyuubi.grpc.events.OperationEventsManager +import org.apache.kyuubi.grpc.operation.GrpcOperation +import org.apache.kyuubi.grpc.utils.Clock + +class KyuubiGrpcOperationEventsManager (operation: GrpcOperation, clock: Clock) + extends OperationEventsManager(operation, clock) with Logging { + + override def postStarted(): Unit = { + super.postStarted() + info("Operation Event: post Started") + } + + override def postClosed(): Unit = { + info("Operation Event: post Closed") + super.postClosed() + } +} diff --git a/kyuubi-grpc-server/src/main/scala/org/apache/kyuubi/grpc/event/KyuubiGrpcSessionEventsManager.scala b/kyuubi-grpc-server/src/main/scala/org/apache/kyuubi/grpc/event/KyuubiGrpcSessionEventsManager.scala new file mode 100644 index 00000000000..6be29372794 --- /dev/null +++ b/kyuubi-grpc-server/src/main/scala/org/apache/kyuubi/grpc/event/KyuubiGrpcSessionEventsManager.scala @@ -0,0 +1,37 @@ +/* + * 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.grpc.event + +import org.apache.kyuubi.Logging +import org.apache.kyuubi.grpc.events.SessionEventsManager +import org.apache.kyuubi.grpc.session.KyuubiGrpcSession +import org.apache.kyuubi.grpc.utils.Clock + +class KyuubiGrpcSessionEventsManager (session: KyuubiGrpcSession, clock: Clock) + extends SessionEventsManager(session, clock) with Logging { + + override def postStarted(): Unit = { + super.postStarted() + info("Session Event: post Started") + } + + override def postClosed(): Unit = { + info("Session Event: post Closed") + super.postClosed() + } +} diff --git a/kyuubi-grpc-server/src/main/scala/org/apache/kyuubi/grpc/operation/KyuubiGrpcOperation.scala b/kyuubi-grpc-server/src/main/scala/org/apache/kyuubi/grpc/operation/KyuubiGrpcOperation.scala new file mode 100644 index 00000000000..e30da905a27 --- /dev/null +++ b/kyuubi-grpc-server/src/main/scala/org/apache/kyuubi/grpc/operation/KyuubiGrpcOperation.scala @@ -0,0 +1,54 @@ +/* + * 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.grpc.operation + +import org.apache.kyuubi.grpc.events.OperationEventsManager +import org.apache.kyuubi.grpc.session.KyuubiGrpcSession +import org.apache.kyuubi.operation.log.OperationLog + +abstract class KyuubiGrpcOperation(session: KyuubiGrpcSession) + extends AbstractGrpcOperation[KyuubiGrpcSession](session) { + + override def key: OperationKey = OperationKey(session.sessionKey) + + override def beforeRun(): Unit = { + info("beforeRun, currently empty") + } + + override def afterRun(): Unit = { + info("afterRun, currently empty") + } + + override def close(): Unit = { + info("close, currently empty") + } + + override def operationEventsManager: OperationEventsManager = { + null + } + + override def interrupt(): Unit = { + info("interrupt, currently empty") + } + + override def isTimedOut: Boolean = false + + override def getOperationLog: Option[OperationLog] = { + Some(null) + } +} diff --git a/kyuubi-grpc-server/src/main/scala/org/apache/kyuubi/grpc/operation/KyuubiGrpcOperationManager.scala b/kyuubi-grpc-server/src/main/scala/org/apache/kyuubi/grpc/operation/KyuubiGrpcOperationManager.scala new file mode 100644 index 00000000000..1351b2eca7b --- /dev/null +++ b/kyuubi-grpc-server/src/main/scala/org/apache/kyuubi/grpc/operation/KyuubiGrpcOperationManager.scala @@ -0,0 +1,39 @@ +/* + * 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.grpc.operation + +import io.grpc.ManagedChannel +import io.grpc.stub.StreamObserver +import org.apache.spark.connect.proto.{ConfigRequest, ConfigResponse} + +import org.apache.kyuubi.grpc.operation.spark.Config +import org.apache.kyuubi.grpc.session.KyuubiGrpcSession + +class KyuubiGrpcOperationManager(name: String) extends GrpcOperationManager(name) { + def this() = this(classOf[KyuubiGrpcOperationManager].getSimpleName) + + def newSparkConfigOperation( + channel: ManagedChannel, + session: KyuubiGrpcSession, + request: ConfigRequest, + responseObserver: StreamObserver[ConfigResponse]): GrpcOperation = { + val operation = + new Config(channel, session, request, responseObserver) + addOperation(operation) + } +} diff --git a/kyuubi-grpc-server/src/main/scala/org/apache/kyuubi/grpc/operation/spark/Config.scala b/kyuubi-grpc-server/src/main/scala/org/apache/kyuubi/grpc/operation/spark/Config.scala new file mode 100644 index 00000000000..5301b50a87d --- /dev/null +++ b/kyuubi-grpc-server/src/main/scala/org/apache/kyuubi/grpc/operation/spark/Config.scala @@ -0,0 +1,35 @@ +/* + * 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.grpc.operation.spark + +import io.grpc.ManagedChannel +import io.grpc.stub.StreamObserver +import org.apache.spark.connect.proto.{ConfigRequest, ConfigResponse} + +import org.apache.kyuubi.grpc.session.KyuubiGrpcSession + +class Config( + channel: ManagedChannel, + session: KyuubiGrpcSession, + request: ConfigRequest, + responseObserver: StreamObserver[ConfigResponse]) + extends KyuubiSparkGrpcOperation(channel, session) { + override protected def runInternal(): Unit = { + stub.config(request, responseObserver) + } +} diff --git a/kyuubi-grpc-server/src/main/scala/org/apache/kyuubi/grpc/operation/spark/KyuubiSparkGrpcOperation.scala b/kyuubi-grpc-server/src/main/scala/org/apache/kyuubi/grpc/operation/spark/KyuubiSparkGrpcOperation.scala new file mode 100644 index 00000000000..9b7edba04f2 --- /dev/null +++ b/kyuubi-grpc-server/src/main/scala/org/apache/kyuubi/grpc/operation/spark/KyuubiSparkGrpcOperation.scala @@ -0,0 +1,31 @@ +/* + * 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.grpc.operation.spark + +import io.grpc.ManagedChannel +import org.apache.spark.connect.proto.SparkConnectServiceGrpc +import org.apache.spark.connect.proto.SparkConnectServiceGrpc.SparkConnectServiceStub + +import org.apache.kyuubi.grpc.operation.KyuubiGrpcOperation +import org.apache.kyuubi.grpc.session.KyuubiGrpcSession + +abstract class KyuubiSparkGrpcOperation(channel: ManagedChannel, session: KyuubiGrpcSession) + extends KyuubiGrpcOperation(session) { + + def stub: SparkConnectServiceStub = SparkConnectServiceGrpc.newStub(channel) +} diff --git a/kyuubi-grpc-server/src/main/scala/org/apache/kyuubi/grpc/server/KyuubiGrpcSeverable.scala b/kyuubi-grpc-server/src/main/scala/org/apache/kyuubi/grpc/server/KyuubiGrpcSeverable.scala new file mode 100644 index 00000000000..a23b6f92427 --- /dev/null +++ b/kyuubi-grpc-server/src/main/scala/org/apache/kyuubi/grpc/server/KyuubiGrpcSeverable.scala @@ -0,0 +1,28 @@ +/* + * 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.grpc.server + +import org.apache.kyuubi.grpc.service.GrpcSeverable + +abstract class KyuubiGrpcSeverable(name: String) + extends GrpcSeverable[KyuubiGrpcBackendService, AbstractKyuubiGrpcFrontendService](name) { + + override protected def stopServer(): Unit = { + info("stop") + } +} diff --git a/kyuubi-grpc-server/src/main/scala/org/apache/kyuubi/grpc/session/KyuubiGrpcSession.scala b/kyuubi-grpc-server/src/main/scala/org/apache/kyuubi/grpc/session/KyuubiGrpcSession.scala new file mode 100644 index 00000000000..63772dd03d1 --- /dev/null +++ b/kyuubi-grpc-server/src/main/scala/org/apache/kyuubi/grpc/session/KyuubiGrpcSession.scala @@ -0,0 +1,55 @@ +/* + * 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.grpc.session +import scala.util.Random + +import io.grpc.ManagedChannel +import io.grpc.stub.StreamObserver +import org.apache.spark.connect.proto._ + +import org.apache.kyuubi.grpc.event.KyuubiGrpcSessionEventsManager +import org.apache.kyuubi.grpc.events.SessionEventsManager +import org.apache.kyuubi.grpc.operation.{GrpcOperation, OperationKey} +import org.apache.kyuubi.grpc.utils.SystemClock + +class KyuubiGrpcSession( + userId: String, + sessionManager: KyuubiGrpcSessionManager) + extends AbstractGrpcSession(userId) { + + def config( + request: ConfigRequest, + responseObserver: StreamObserver[ConfigResponse], + channel: ManagedChannel): Unit = { + val operation = sessionManager.grpcOperationManager + .newSparkConfigOperation(channel, this, request, responseObserver) + runGrpcOperation(operation) + } + + override def name: Option[String] = Some("KyuubiGrpcSessionImpl") + + override def serverSessionId: String = Random.nextString(10) + + override def sessionManager: GrpcSessionManager[KyuubiGrpcSession] = sessionManager + + override def sessionEventsManager: SessionEventsManager = + new KyuubiGrpcSessionEventsManager(this, new SystemClock()) + + override def getOperation(operationKey: OperationKey): GrpcOperation = + sessionManager.grpcOperationManager.getOperation(operationKey) +} diff --git a/kyuubi-grpc-server/src/main/scala/org/apache/kyuubi/grpc/session/KyuubiGrpcSessionManager.scala b/kyuubi-grpc-server/src/main/scala/org/apache/kyuubi/grpc/session/KyuubiGrpcSessionManager.scala new file mode 100644 index 00000000000..7dab172566a --- /dev/null +++ b/kyuubi-grpc-server/src/main/scala/org/apache/kyuubi/grpc/session/KyuubiGrpcSessionManager.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.grpc.session + +import org.apache.kyuubi.grpc.operation.KyuubiGrpcOperationManager + +class KyuubiGrpcSessionManager + extends GrpcSessionManager[KyuubiGrpcSession] ("KyuubiGrpcSessionManager") { + + override def isServer: Boolean = true + + override def grpcOperationManager: KyuubiGrpcOperationManager = new KyuubiGrpcOperationManager() + + override def getOrCreateSession(key: SessionKey): KyuubiGrpcSession = { + val session = new KyuubiGrpcSession(key.userId, this) + session + } +} diff --git a/kyuubi-grpc-server/src/test/scala/org/apache/spark/sql/connect/SparkConnectSuite.scala b/kyuubi-grpc-server/src/test/scala/org/apache/spark/sql/connect/SparkConnectSuite.scala new file mode 100644 index 00000000000..42c6517276e --- /dev/null +++ b/kyuubi-grpc-server/src/test/scala/org/apache/spark/sql/connect/SparkConnectSuite.scala @@ -0,0 +1,46 @@ +/* + * 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.spark.sql.connect + +import org.apache.spark.connect.proto.ConfigRequest +import org.apache.spark.sql.connect.client.SparkConnectClient + +import org.apache.kyuubi.KyuubiFunSuite +import org.apache.kyuubi.config.KyuubiConf + +class SparkConnectSuite extends KyuubiFunSuite { + val server: TestServer = new TestServer("testServer") + var client: SparkConnectClient = _ + override def beforeAll(): Unit = { + val conf = KyuubiConf().set(KyuubiConf.ENGINE_SPARK_CONNECT_GRPC_BINDING_PORT, 10023) + server.initialize(conf) + server.start() + client = SparkConnectClient.builder().port(server.frontendInfo()._2).build() + super.beforeAll() + + } + + test("test config") { + val request = ConfigRequest.newBuilder() + .setSessionId("abc123") + .build() + + val response = client.config(request.getOperation) + assert(response.getSessionId === "abc123") + } +} diff --git a/kyuubi-grpc-server/src/test/scala/org/apache/spark/sql/connect/SparkConnectUtil.scala b/kyuubi-grpc-server/src/test/scala/org/apache/spark/sql/connect/SparkConnectUtil.scala new file mode 100644 index 00000000000..543590b8026 --- /dev/null +++ b/kyuubi-grpc-server/src/test/scala/org/apache/spark/sql/connect/SparkConnectUtil.scala @@ -0,0 +1,39 @@ +/* + * 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.spark.sql.connect + +import java.net.InetSocketAddress +import java.time.Instant + +import org.apache.spark.SparkConf +import org.apache.spark.sql.SparkSession +import org.apache.spark.sql.connect.service.SparkConnectService + +object SparkConnectUtil { + def startSparkConnectServer(): (String, Int) = { + val _sparkConf = new SparkConf() + _sparkConf.setIfMissing("spark.sql.binaryOutputStyle", "UTF8") + _sparkConf.setIfMissing("spark.master", "local") + val appName = s"kyuubi_test_spark_${Instant.now}" + _sparkConf.setIfMissing("spark.app.name", appName) + val session = SparkSession.builder().config(_sparkConf).getOrCreate() + var isa: InetSocketAddress = null + SparkConnectService.start(session.sparkContext) + ("localhost", 15002) + } +} diff --git a/kyuubi-grpc-server/src/test/scala/org/apache/spark/sql/connect/TestFrontendService.scala b/kyuubi-grpc-server/src/test/scala/org/apache/spark/sql/connect/TestFrontendService.scala new file mode 100644 index 00000000000..03561b817f6 --- /dev/null +++ b/kyuubi-grpc-server/src/test/scala/org/apache/spark/sql/connect/TestFrontendService.scala @@ -0,0 +1,29 @@ +/* + * 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.spark.sql.connect + +import org.apache.kyuubi.grpc.server.{KyuubiGrpcSeverable, KyuubiSparkConnectFrontendService} +import org.apache.kyuubi.grpc.utils.SparkConnectUtil + +class TestFrontendService(grpcSeverable: KyuubiGrpcSeverable, name: String) + extends KyuubiSparkConnectFrontendService(grpcSeverable, name) { + override def startEngine(): (String, Int) = { + SparkConnectUtil.startSparkConnectServer() + } +} diff --git a/kyuubi-grpc-server/src/test/scala/org/apache/spark/sql/connect/TestServer.scala b/kyuubi-grpc-server/src/test/scala/org/apache/spark/sql/connect/TestServer.scala new file mode 100644 index 00000000000..734b516f402 --- /dev/null +++ b/kyuubi-grpc-server/src/test/scala/org/apache/spark/sql/connect/TestServer.scala @@ -0,0 +1,27 @@ +/* + * 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.spark.sql.connect + +import org.apache.kyuubi.grpc.server.{AbstractKyuubiGrpcFrontendService, KyuubiGrpcServer} + +class TestServer(name: String) extends KyuubiGrpcServer(name) { + + override val frontendServices: Seq[AbstractKyuubiGrpcFrontendService] = Seq( + new TestFrontendService(this, "TestFrontendService") + ) +} From 55cbae3f30bb66dcf0bd972d2bce1fd49cde14b7 Mon Sep 17 00:00:00 2001 From: davidyuan Date: Wed, 10 Jul 2024 19:17:08 +0800 Subject: [PATCH 19/21] update --- .idea/vcs.xml | 29 +++------ kyuubi-grpc-server/pom.xml | 62 +++++-------------- .../KyuubiGrpcOperationEventsManager.scala | 2 +- .../KyuubiGrpcSessionEventsManager.scala | 2 +- .../KyuubiGrpcOperationManager.scala | 2 +- .../kyuubi/grpc/operation/spark/Config.scala | 2 +- .../spark/KyuubiSparkGrpcOperation.scala | 4 +- .../server/KyuubiGrpcBackendService.scala | 3 +- .../kyuubi/grpc/server/KyuubiGrpcServer.scala | 1 - .../KyuubiSparkConnectFrontendService.scala | 2 +- .../grpc/session/KyuubiGrpcSession.scala | 2 +- .../session/KyuubiGrpcSessionManager.scala | 2 +- .../spark/sql/connect/SparkConnectUtil.scala | 2 +- .../sql/connect/TestFrontendService.scala | 2 - .../apache/spark/sql/connect/TestServer.scala | 3 +- pom.xml | 3 + 16 files changed, 38 insertions(+), 85 deletions(-) diff --git a/.idea/vcs.xml b/.idea/vcs.xml index 9c45aa8a469..b500a7ae045 100644 --- a/.idea/vcs.xml +++ b/.idea/vcs.xml @@ -1,30 +1,17 @@ - + + + + \ No newline at end of file diff --git a/kyuubi-grpc-server/pom.xml b/kyuubi-grpc-server/pom.xml index c14f832e6fb..8eb39c40011 100644 --- a/kyuubi-grpc-server/pom.xml +++ b/kyuubi-grpc-server/pom.xml @@ -15,16 +15,15 @@ ~ See the License for the specific language governing permissions and ~ limitations under the License. --> - + 4.0.0 - kyuubi-parent org.apache.kyuubi + kyuubi-parent 1.10.0-SNAPSHOT ../pom.xml - 4.0.0 kyuubi-grpc-server_${scala.binary.version} jar @@ -32,6 +31,14 @@ https://kyuubi.apache.org/ + + + + org.apache.spark + spark-connect_2.13 + 3.5.1 + + org.apache.kyuubi kyuubi-grpc_${scala.binary.version} @@ -65,20 +72,11 @@ spark-sql_${scala.binary.version} - - - org.apache.spark - spark-connect-common_2.12 - 3.5.1 - - - - - org.apache.spark - spark-connect_2.12 - 3.5.1 + org.apache.kyuubi + kyuubi-grpc-shade + ${project.version} @@ -95,38 +93,6 @@ javax.servlet-api - - - - - org.apache.maven.plugins - maven-shade-plugin - 3.2.4 - - - package - - shade - - - - - org.apache.spark:spark-connect-common_2.12 - - - - - org.apache.spark.connect.proto - shade.org.apache.spark.connect.proto - - - - - - - - - diff --git a/kyuubi-grpc-server/src/main/scala/org/apache/kyuubi/grpc/event/KyuubiGrpcOperationEventsManager.scala b/kyuubi-grpc-server/src/main/scala/org/apache/kyuubi/grpc/event/KyuubiGrpcOperationEventsManager.scala index 6b9bb05de1b..f8f88ccf434 100644 --- a/kyuubi-grpc-server/src/main/scala/org/apache/kyuubi/grpc/event/KyuubiGrpcOperationEventsManager.scala +++ b/kyuubi-grpc-server/src/main/scala/org/apache/kyuubi/grpc/event/KyuubiGrpcOperationEventsManager.scala @@ -22,7 +22,7 @@ import org.apache.kyuubi.grpc.events.OperationEventsManager import org.apache.kyuubi.grpc.operation.GrpcOperation import org.apache.kyuubi.grpc.utils.Clock -class KyuubiGrpcOperationEventsManager (operation: GrpcOperation, clock: Clock) +class KyuubiGrpcOperationEventsManager(operation: GrpcOperation, clock: Clock) extends OperationEventsManager(operation, clock) with Logging { override def postStarted(): Unit = { diff --git a/kyuubi-grpc-server/src/main/scala/org/apache/kyuubi/grpc/event/KyuubiGrpcSessionEventsManager.scala b/kyuubi-grpc-server/src/main/scala/org/apache/kyuubi/grpc/event/KyuubiGrpcSessionEventsManager.scala index 6be29372794..a47f5035fb2 100644 --- a/kyuubi-grpc-server/src/main/scala/org/apache/kyuubi/grpc/event/KyuubiGrpcSessionEventsManager.scala +++ b/kyuubi-grpc-server/src/main/scala/org/apache/kyuubi/grpc/event/KyuubiGrpcSessionEventsManager.scala @@ -22,7 +22,7 @@ import org.apache.kyuubi.grpc.events.SessionEventsManager import org.apache.kyuubi.grpc.session.KyuubiGrpcSession import org.apache.kyuubi.grpc.utils.Clock -class KyuubiGrpcSessionEventsManager (session: KyuubiGrpcSession, clock: Clock) +class KyuubiGrpcSessionEventsManager(session: KyuubiGrpcSession, clock: Clock) extends SessionEventsManager(session, clock) with Logging { override def postStarted(): Unit = { diff --git a/kyuubi-grpc-server/src/main/scala/org/apache/kyuubi/grpc/operation/KyuubiGrpcOperationManager.scala b/kyuubi-grpc-server/src/main/scala/org/apache/kyuubi/grpc/operation/KyuubiGrpcOperationManager.scala index 1351b2eca7b..5622ac15c32 100644 --- a/kyuubi-grpc-server/src/main/scala/org/apache/kyuubi/grpc/operation/KyuubiGrpcOperationManager.scala +++ b/kyuubi-grpc-server/src/main/scala/org/apache/kyuubi/grpc/operation/KyuubiGrpcOperationManager.scala @@ -19,10 +19,10 @@ package org.apache.kyuubi.grpc.operation import io.grpc.ManagedChannel import io.grpc.stub.StreamObserver -import org.apache.spark.connect.proto.{ConfigRequest, ConfigResponse} import org.apache.kyuubi.grpc.operation.spark.Config import org.apache.kyuubi.grpc.session.KyuubiGrpcSession +import org.apache.kyuubi.shade.org.apache.spark.connect.proto.{ConfigRequest, ConfigResponse} class KyuubiGrpcOperationManager(name: String) extends GrpcOperationManager(name) { def this() = this(classOf[KyuubiGrpcOperationManager].getSimpleName) diff --git a/kyuubi-grpc-server/src/main/scala/org/apache/kyuubi/grpc/operation/spark/Config.scala b/kyuubi-grpc-server/src/main/scala/org/apache/kyuubi/grpc/operation/spark/Config.scala index 5301b50a87d..6b6ac4a8abd 100644 --- a/kyuubi-grpc-server/src/main/scala/org/apache/kyuubi/grpc/operation/spark/Config.scala +++ b/kyuubi-grpc-server/src/main/scala/org/apache/kyuubi/grpc/operation/spark/Config.scala @@ -19,9 +19,9 @@ package org.apache.kyuubi.grpc.operation.spark import io.grpc.ManagedChannel import io.grpc.stub.StreamObserver -import org.apache.spark.connect.proto.{ConfigRequest, ConfigResponse} import org.apache.kyuubi.grpc.session.KyuubiGrpcSession +import org.apache.kyuubi.shade.org.apache.spark.connect.proto.{ConfigRequest, ConfigResponse} class Config( channel: ManagedChannel, diff --git a/kyuubi-grpc-server/src/main/scala/org/apache/kyuubi/grpc/operation/spark/KyuubiSparkGrpcOperation.scala b/kyuubi-grpc-server/src/main/scala/org/apache/kyuubi/grpc/operation/spark/KyuubiSparkGrpcOperation.scala index 9b7edba04f2..44a7fd4fc93 100644 --- a/kyuubi-grpc-server/src/main/scala/org/apache/kyuubi/grpc/operation/spark/KyuubiSparkGrpcOperation.scala +++ b/kyuubi-grpc-server/src/main/scala/org/apache/kyuubi/grpc/operation/spark/KyuubiSparkGrpcOperation.scala @@ -18,11 +18,11 @@ package org.apache.kyuubi.grpc.operation.spark import io.grpc.ManagedChannel -import org.apache.spark.connect.proto.SparkConnectServiceGrpc -import org.apache.spark.connect.proto.SparkConnectServiceGrpc.SparkConnectServiceStub import org.apache.kyuubi.grpc.operation.KyuubiGrpcOperation import org.apache.kyuubi.grpc.session.KyuubiGrpcSession +import org.apache.kyuubi.shade.org.apache.spark.connect.proto.SparkConnectServiceGrpc +import org.apache.kyuubi.shade.org.apache.spark.connect.proto.SparkConnectServiceGrpc.SparkConnectServiceStub abstract class KyuubiSparkGrpcOperation(channel: ManagedChannel, session: KyuubiGrpcSession) extends KyuubiGrpcOperation(session) { diff --git a/kyuubi-grpc-server/src/main/scala/org/apache/kyuubi/grpc/server/KyuubiGrpcBackendService.scala b/kyuubi-grpc-server/src/main/scala/org/apache/kyuubi/grpc/server/KyuubiGrpcBackendService.scala index 2af91b3caaa..68030a14a2b 100644 --- a/kyuubi-grpc-server/src/main/scala/org/apache/kyuubi/grpc/server/KyuubiGrpcBackendService.scala +++ b/kyuubi-grpc-server/src/main/scala/org/apache/kyuubi/grpc/server/KyuubiGrpcBackendService.scala @@ -19,9 +19,10 @@ package org.apache.kyuubi.grpc.server import io.grpc.ManagedChannel import io.grpc.stub.StreamObserver -import org.apache.spark.connect.proto._ + import org.apache.kyuubi.grpc.service.AbstractGrpcBackendService import org.apache.kyuubi.grpc.session.{GrpcSessionManager, KyuubiGrpcSession, KyuubiGrpcSessionManager, SessionKey} +import org.apache.kyuubi.shade.org.apache.spark.connect.proto._ class KyuubiGrpcBackendService(name: String) extends AbstractGrpcBackendService(name) { diff --git a/kyuubi-grpc-server/src/main/scala/org/apache/kyuubi/grpc/server/KyuubiGrpcServer.scala b/kyuubi-grpc-server/src/main/scala/org/apache/kyuubi/grpc/server/KyuubiGrpcServer.scala index c46a8eb8c32..f65f2fae4ec 100644 --- a/kyuubi-grpc-server/src/main/scala/org/apache/kyuubi/grpc/server/KyuubiGrpcServer.scala +++ b/kyuubi-grpc-server/src/main/scala/org/apache/kyuubi/grpc/server/KyuubiGrpcServer.scala @@ -26,4 +26,3 @@ class KyuubiGrpcServer(name: String) (frontendServices.head.host, frontendServices.head.port) } } - diff --git a/kyuubi-grpc-server/src/main/scala/org/apache/kyuubi/grpc/server/KyuubiSparkConnectFrontendService.scala b/kyuubi-grpc-server/src/main/scala/org/apache/kyuubi/grpc/server/KyuubiSparkConnectFrontendService.scala index 5587cb3108c..0585219b4fd 100644 --- a/kyuubi-grpc-server/src/main/scala/org/apache/kyuubi/grpc/server/KyuubiSparkConnectFrontendService.scala +++ b/kyuubi-grpc-server/src/main/scala/org/apache/kyuubi/grpc/server/KyuubiSparkConnectFrontendService.scala @@ -22,10 +22,10 @@ import scala.jdk.CollectionConverters.collectionAsScalaIterableConverter import com.google.protobuf.MessageLite import io.grpc._ import io.grpc.stub.StreamObserver -import org.apache.spark.connect.proto._ import org.apache.kyuubi.grpc.session.SessionKey import org.apache.kyuubi.service.Service +import org.apache.kyuubi.shade.org.apache.spark.connect.proto.{ConfigRequest, ConfigResponse, SparkConnectServiceGrpc} abstract class KyuubiSparkConnectFrontendService(grpcSeverable: KyuubiGrpcSeverable, name: String) extends AbstractKyuubiGrpcFrontendService(grpcSeverable, name) diff --git a/kyuubi-grpc-server/src/main/scala/org/apache/kyuubi/grpc/session/KyuubiGrpcSession.scala b/kyuubi-grpc-server/src/main/scala/org/apache/kyuubi/grpc/session/KyuubiGrpcSession.scala index 63772dd03d1..e2e475d88e9 100644 --- a/kyuubi-grpc-server/src/main/scala/org/apache/kyuubi/grpc/session/KyuubiGrpcSession.scala +++ b/kyuubi-grpc-server/src/main/scala/org/apache/kyuubi/grpc/session/KyuubiGrpcSession.scala @@ -20,12 +20,12 @@ import scala.util.Random import io.grpc.ManagedChannel import io.grpc.stub.StreamObserver -import org.apache.spark.connect.proto._ import org.apache.kyuubi.grpc.event.KyuubiGrpcSessionEventsManager import org.apache.kyuubi.grpc.events.SessionEventsManager import org.apache.kyuubi.grpc.operation.{GrpcOperation, OperationKey} import org.apache.kyuubi.grpc.utils.SystemClock +import org.apache.kyuubi.shade.org.apache.spark.connect.proto._ class KyuubiGrpcSession( userId: String, diff --git a/kyuubi-grpc-server/src/main/scala/org/apache/kyuubi/grpc/session/KyuubiGrpcSessionManager.scala b/kyuubi-grpc-server/src/main/scala/org/apache/kyuubi/grpc/session/KyuubiGrpcSessionManager.scala index 7dab172566a..2d06d1329bb 100644 --- a/kyuubi-grpc-server/src/main/scala/org/apache/kyuubi/grpc/session/KyuubiGrpcSessionManager.scala +++ b/kyuubi-grpc-server/src/main/scala/org/apache/kyuubi/grpc/session/KyuubiGrpcSessionManager.scala @@ -20,7 +20,7 @@ package org.apache.kyuubi.grpc.session import org.apache.kyuubi.grpc.operation.KyuubiGrpcOperationManager class KyuubiGrpcSessionManager - extends GrpcSessionManager[KyuubiGrpcSession] ("KyuubiGrpcSessionManager") { + extends GrpcSessionManager[KyuubiGrpcSession]("KyuubiGrpcSessionManager") { override def isServer: Boolean = true diff --git a/kyuubi-grpc-server/src/test/scala/org/apache/spark/sql/connect/SparkConnectUtil.scala b/kyuubi-grpc-server/src/test/scala/org/apache/spark/sql/connect/SparkConnectUtil.scala index 543590b8026..740707156b7 100644 --- a/kyuubi-grpc-server/src/test/scala/org/apache/spark/sql/connect/SparkConnectUtil.scala +++ b/kyuubi-grpc-server/src/test/scala/org/apache/spark/sql/connect/SparkConnectUtil.scala @@ -22,7 +22,7 @@ import java.time.Instant import org.apache.spark.SparkConf import org.apache.spark.sql.SparkSession -import org.apache.spark.sql.connect.service.SparkConnectService +import org.apache.spark.sql.connect.service._ object SparkConnectUtil { def startSparkConnectServer(): (String, Int) = { diff --git a/kyuubi-grpc-server/src/test/scala/org/apache/spark/sql/connect/TestFrontendService.scala b/kyuubi-grpc-server/src/test/scala/org/apache/spark/sql/connect/TestFrontendService.scala index 03561b817f6..2020f8945eb 100644 --- a/kyuubi-grpc-server/src/test/scala/org/apache/spark/sql/connect/TestFrontendService.scala +++ b/kyuubi-grpc-server/src/test/scala/org/apache/spark/sql/connect/TestFrontendService.scala @@ -15,11 +15,9 @@ * limitations under the License. */ - package org.apache.spark.sql.connect import org.apache.kyuubi.grpc.server.{KyuubiGrpcSeverable, KyuubiSparkConnectFrontendService} -import org.apache.kyuubi.grpc.utils.SparkConnectUtil class TestFrontendService(grpcSeverable: KyuubiGrpcSeverable, name: String) extends KyuubiSparkConnectFrontendService(grpcSeverable, name) { diff --git a/kyuubi-grpc-server/src/test/scala/org/apache/spark/sql/connect/TestServer.scala b/kyuubi-grpc-server/src/test/scala/org/apache/spark/sql/connect/TestServer.scala index 734b516f402..4c1031771a8 100644 --- a/kyuubi-grpc-server/src/test/scala/org/apache/spark/sql/connect/TestServer.scala +++ b/kyuubi-grpc-server/src/test/scala/org/apache/spark/sql/connect/TestServer.scala @@ -22,6 +22,5 @@ import org.apache.kyuubi.grpc.server.{AbstractKyuubiGrpcFrontendService, KyuubiG class TestServer(name: String) extends KyuubiGrpcServer(name) { override val frontendServices: Seq[AbstractKyuubiGrpcFrontendService] = Seq( - new TestFrontendService(this, "TestFrontendService") - ) + new TestFrontendService(this, "TestFrontendService")) } diff --git a/pom.xml b/pom.xml index 1ed753eb653..681c3e1a5e2 100644 --- a/pom.xml +++ b/pom.xml @@ -85,6 +85,7 @@ kyuubi-zookeeper kyuubi-grpc kyuubi-grpc-server + kyuubi-grpc-shade @@ -1281,6 +1282,7 @@ true + always false @@ -1296,6 +1298,7 @@ false + always central Maven Repository From 0c17121f9c7fa9607364e0c22a480cf5cab073d7 Mon Sep 17 00:00:00 2001 From: davidyuan Date: Mon, 15 Jul 2024 18:56:48 +0800 Subject: [PATCH 20/21] update --- kyuubi-grpc-shade/pom.xml | 123 ++++++++++ .../src/main/resources/META-INF/LICENSE | 226 ++++++++++++++++++ .../src/main/resources/META-INF/NOTICE | 105 ++++++++ 3 files changed, 454 insertions(+) create mode 100644 kyuubi-grpc-shade/pom.xml create mode 100644 kyuubi-grpc-shade/src/main/resources/META-INF/LICENSE create mode 100644 kyuubi-grpc-shade/src/main/resources/META-INF/NOTICE diff --git a/kyuubi-grpc-shade/pom.xml b/kyuubi-grpc-shade/pom.xml new file mode 100644 index 00000000000..6ad6dca3918 --- /dev/null +++ b/kyuubi-grpc-shade/pom.xml @@ -0,0 +1,123 @@ + + + + 4.0.0 + + org.apache.kyuubi + kyuubi-parent + 1.10.0-SNAPSHOT + + + kyuubi-grpc-shade + jar + Kyuubi Project Grpc Shade(Only for test before merge) + https://kyuubi.apache.org/ + + + + org.apache.spark + spark-connect-common_2.13 + 3.5.1 + + + + + + + org.apache.maven.plugins + maven-shade-plugin + + false + true + true + false + + + org.apache.spark:spark-connect-common_2.12 + + + + + *:* + + META-INF/*.SF + META-INF/*.DSA + META-INF/*.RSA + META-INF/DEPENDENCIES + META-INF/LICENSE.txt + META-INF/NOTICE.txt + META-INF/maven/** + META-INF/native-image/** + LICENSE.txt + NOTICE.txt + mozilla/** + arrow-git.properties + + + + + + org.apache.spark.connect + ${kyuubi.shade.packageName}.org.apache.spark.connect + + + + + + + + shade + + package + + + + + + + + + + + org.codehaus.mojo + build-helper-maven-plugin + + + compile + + attach-artifact + + package + + + + ${basedir}/target/${project.artifactId}-${project.version}.jar + jar + optional + + + + + + + + target/scala-${scala.binary.version}/classes + target/scala-${scala.binary.version}/test-classes + + diff --git a/kyuubi-grpc-shade/src/main/resources/META-INF/LICENSE b/kyuubi-grpc-shade/src/main/resources/META-INF/LICENSE new file mode 100644 index 00000000000..9e1d666a8f2 --- /dev/null +++ b/kyuubi-grpc-shade/src/main/resources/META-INF/LICENSE @@ -0,0 +1,226 @@ + + Apache License + Version 2.0, January 2004 + http://www.apache.org/licenses/ + + TERMS AND CONDITIONS FOR USE, REPRODUCTION, AND DISTRIBUTION + + 1. Definitions. + + "License" shall mean the terms and conditions for use, reproduction, + and distribution as defined by Sections 1 through 9 of this document. + + "Licensor" shall mean the copyright owner or entity authorized by + the copyright owner that is granting the License. + + "Legal Entity" shall mean the union of the acting entity and all + other entities that control, are controlled by, or are under common + control with that entity. For the purposes of this definition, + "control" means (i) the power, direct or indirect, to cause the + direction or management of such entity, whether by contract or + otherwise, or (ii) ownership of fifty percent (50%) or more of the + outstanding shares, or (iii) beneficial ownership of such entity. + + "You" (or "Your") shall mean an individual or Legal Entity + exercising permissions granted by this License. + + "Source" form shall mean the preferred form for making modifications, + including but not limited to software source code, documentation + source, and configuration files. + + "Object" form shall mean any form resulting from mechanical + transformation or translation of a Source form, including but + not limited to compiled object code, generated documentation, + and conversions to other media types. + + "Work" shall mean the work of authorship, whether in Source or + Object form, made available under the License, as indicated by a + copyright notice that is included in or attached to the work + (an example is provided in the Appendix below). + + "Derivative Works" shall mean any work, whether in Source or Object + form, that is based on (or derived from) the Work and for which the + editorial revisions, annotations, elaborations, or other modifications + represent, as a whole, an original work of authorship. For the purposes + of this License, Derivative Works shall not include works that remain + separable from, or merely link (or bind by name) to the interfaces of, + the Work and Derivative Works thereof. + + "Contribution" shall mean any work of authorship, including + the original version of the Work and any modifications or additions + to that Work or Derivative Works thereof, that is intentionally + submitted to Licensor for inclusion in the Work by the copyright owner + or by an individual or Legal Entity authorized to submit on behalf of + the copyright owner. For the purposes of this definition, "submitted" + means any form of electronic, verbal, or written communication sent + to the Licensor or its representatives, including but not limited to + communication on electronic mailing lists, source code control systems, + and issue tracking systems that are managed by, or on behalf of, the + Licensor for the purpose of discussing and improving the Work, but + excluding communication that is conspicuously marked or otherwise + designated in writing by the copyright owner as "Not a Contribution." + + "Contributor" shall mean Licensor and any individual or Legal Entity + on behalf of whom a Contribution has been received by Licensor and + subsequently incorporated within the Work. + + 2. Grant of Copyright License. Subject to the terms and conditions of + this License, each Contributor hereby grants to You a perpetual, + worldwide, non-exclusive, no-charge, royalty-free, irrevocable + copyright license to reproduce, prepare Derivative Works of, + publicly display, publicly perform, sublicense, and distribute the + Work and such Derivative Works in Source or Object form. + + 3. Grant of Patent License. Subject to the terms and conditions of + this License, each Contributor hereby grants to You a perpetual, + worldwide, non-exclusive, no-charge, royalty-free, irrevocable + (except as stated in this section) patent license to make, have made, + use, offer to sell, sell, import, and otherwise transfer the Work, + where such license applies only to those patent claims licensable + by such Contributor that are necessarily infringed by their + Contribution(s) alone or by combination of their Contribution(s) + with the Work to which such Contribution(s) was submitted. If You + institute patent litigation against any entity (including a + cross-claim or counterclaim in a lawsuit) alleging that the Work + or a Contribution incorporated within the Work constitutes direct + or contributory patent infringement, then any patent licenses + granted to You under this License for that Work shall terminate + as of the date such litigation is filed. + + 4. Redistribution. You may reproduce and distribute copies of the + Work or Derivative Works thereof in any medium, with or without + modifications, and in Source or Object form, provided that You + meet the following conditions: + + (a) You must give any other recipients of the Work or + Derivative Works a copy of this License; and + + (b) You must cause any modified files to carry prominent notices + stating that You changed the files; and + + (c) You must retain, in the Source form of any Derivative Works + that You distribute, all copyright, patent, trademark, and + attribution notices from the Source form of the Work, + excluding those notices that do not pertain to any part of + the Derivative Works; and + + (d) If the Work includes a "NOTICE" text file as part of its + distribution, then any Derivative Works that You distribute must + include a readable copy of the attribution notices contained + within such NOTICE file, excluding those notices that do not + pertain to any part of the Derivative Works, in at least one + of the following places: within a NOTICE text file distributed + as part of the Derivative Works; within the Source form or + documentation, if provided along with the Derivative Works; or, + within a display generated by the Derivative Works, if and + wherever such third-party notices normally appear. The contents + of the NOTICE file are for informational purposes only and + do not modify the License. You may add Your own attribution + notices within Derivative Works that You distribute, alongside + or as an addendum to the NOTICE text from the Work, provided + that such additional attribution notices cannot be construed + as modifying the License. + + You may add Your own copyright statement to Your modifications and + may provide additional or different license terms and conditions + for use, reproduction, or distribution of Your modifications, or + for any such Derivative Works as a whole, provided Your use, + reproduction, and distribution of the Work otherwise complies with + the conditions stated in this License. + + 5. Submission of Contributions. Unless You explicitly state otherwise, + any Contribution intentionally submitted for inclusion in the Work + by You to the Licensor shall be under the terms and conditions of + this License, without any additional terms or conditions. + Notwithstanding the above, nothing herein shall supersede or modify + the terms of any separate license agreement you may have executed + with Licensor regarding such Contributions. + + 6. Trademarks. This License does not grant permission to use the trade + names, trademarks, service marks, or product names of the Licensor, + except as required for reasonable and customary use in describing the + origin of the Work and reproducing the content of the NOTICE file. + + 7. Disclaimer of Warranty. Unless required by applicable law or + agreed to in writing, Licensor provides the Work (and each + Contributor provides its Contributions) on an "AS IS" BASIS, + WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or + implied, including, without limitation, any warranties or conditions + of TITLE, NON-INFRINGEMENT, MERCHANTABILITY, or FITNESS FOR A + PARTICULAR PURPOSE. You are solely responsible for determining the + appropriateness of using or redistributing the Work and assume any + risks associated with Your exercise of permissions under this License. + + 8. Limitation of Liability. In no event and under no legal theory, + whether in tort (including negligence), contract, or otherwise, + unless required by applicable law (such as deliberate and grossly + negligent acts) or agreed to in writing, shall any Contributor be + liable to You for damages, including any direct, indirect, special, + incidental, or consequential damages of any character arising as a + result of this License or out of the use or inability to use the + Work (including but not limited to damages for loss of goodwill, + work stoppage, computer failure or malfunction, or any and all + other commercial damages or losses), even if such Contributor + has been advised of the possibility of such damages. + + 9. Accepting Warranty or Additional Liability. While redistributing + the Work or Derivative Works thereof, You may choose to offer, + and charge a fee for, acceptance of support, warranty, indemnity, + or other liability obligations and/or rights consistent with this + License. However, in accepting such obligations, You may act only + on Your own behalf and on Your sole responsibility, not on behalf + of any other Contributor, and only if You agree to indemnify, + defend, and hold each Contributor harmless for any liability + incurred by, or claims asserted against, such Contributor by reason + of your accepting any such warranty or additional liability. + + END OF TERMS AND CONDITIONS + + APPENDIX: How to apply the Apache License to your work. + + To apply the Apache License to your work, attach the following + boilerplate notice, with the fields enclosed by brackets "[]" + replaced with your own identifying information. (Don't include + the brackets!) The text should be enclosed in the appropriate + comment syntax for the file format. We also recommend that a + file or class name and description of purpose be included on the + same "printed page" as the copyright notice for easier + identification within third-party archives. + + Copyright [yyyy] [name of copyright owner] + + Licensed 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. + +------------------------------------------------------------------------------------ + +This project bundles some components that are licensed under the + +Apache License Version 2.0 +-------------------------- +org.apache.hive:hive-service-rpc +com.google.guava:failureaccess +com.google.guava:guava +commons-codec:commons-codec +org.apache.commons:commons-lang3 +org.apache.curator:curator-framework +org.apache.curator:curator-client +org.apache.httpcomponents:httpclient +org.apache.httpcomponents:httpcore +org.apache.thrift:fb303 +org.apache.thrift:libthrift +org.apache.zookeeper:zookeeper + +MIT License +----------- +org.slf4j:slf4j-api +org.slf4j:jcl-over-slf4j diff --git a/kyuubi-grpc-shade/src/main/resources/META-INF/NOTICE b/kyuubi-grpc-shade/src/main/resources/META-INF/NOTICE new file mode 100644 index 00000000000..cf2047d5ee2 --- /dev/null +++ b/kyuubi-grpc-shade/src/main/resources/META-INF/NOTICE @@ -0,0 +1,105 @@ +Apache Kyuubi +Copyright 2021-2022 The Apache Software Foundation. + +This product includes software developed at +The Apache Software Foundation (https://www.apache.org/). + +Apache Iceberg +Copyright 2017-2022 The Apache Software Foundation + +Apache Parquet MR +Copyright 2014-2024 The Apache Software Foundation + +This project includes code from Kite, developed at Cloudera, Inc. with +the following copyright notice: + +| Copyright 2013 Cloudera Inc. +| +| Licensed 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. + +Apache Spark +Copyright 2014 and onwards The Apache Software Foundation. + +-------------------------------------------------------------------------------- + +This binary artifact contains Apache Commons Codec + +Apache Commons Codec +Copyright 2002-2020 The Apache Software Foundation + +| src/test/org/apache/commons/codec/language/DoubleMetaphoneTest.java +| contains test data from http://aspell.net/test/orig/batch0.tab. +| Copyright (C) 2002 Kevin Atkinson (kevina@gnu.org) +| +| =============================================================================== +| +| The content of package org.apache.commons.codec.language.bm has been translated +| from the original php source code available at http://stevemorse.org/phoneticinfo.htm +| with permission from the original authors. +| Original source copyright: +| Copyright (c) 2008 Alexander Beider & Stephen P. Morse. + +-------------------------------------------------------------------------------- + +This binary artifact contains + +Guava +License: Apache License, 2.0 +Copyright (C) 2009 The Guava Authors + +Apache Commons Lang +Copyright 2001-2020 The Apache Software Foundation + +Curator Client +Copyright 2011-2017 The Apache Software Foundation + +Curator Framework +Copyright 2011-2017 The Apache Software Foundation + +Hive Service RPC +Copyright 2022 The Apache Software Foundation + +Apache HttpClient +Copyright 1999-2020 The Apache Software Foundation + +Apache HttpCore +Copyright 2005-2020 The Apache Software Foundation + +Apache Thrift +Copyright 2006-2010 The Apache Software Foundation. + +Apache Zookeeper +Copyright 2009-2019 The Apache Software Foundation. + +Apache Arrow +Copyright 2016-2019 The Apache Software Foundation + +-------------------------------------------------------------------------------- + +This binary artifact contains Jackson JSON processor + +Jackson is a high-performance, Free/Open Source JSON processing library. +It was originally written by Tatu Saloranta (tatu.saloranta@iki.fi), and has +been in development since 2007. +It is currently developed by a community of developers. + +## Licensing + +Jackson 2.x core and extension components are licensed under Apache License 2.0 +To find the details that apply to this artifact see the accompanying LICENSE file. + +## Credits + +A list of contributors may be found from CREDITS(-2.x) file, which is included +in some artifacts (usually source distributions); but is always available +from the source code management (SCM) system project uses. From 2b959f9ab18bb28af65af994550c5a3e05e2ad7f Mon Sep 17 00:00:00 2001 From: davidyuan Date: Wed, 24 Jul 2024 00:13:25 +0800 Subject: [PATCH 21/21] add new module kyuubi-grpc --- kyuubi-grpc-server/pom.xml | 177 ++++++++++++++++-- .../src/main/resources/log4j2.xml | 29 +++ .../KyuubiSparkConnectFrontendService.scala | 2 +- .../grpc/session/KyuubiGrpcSession.scala | 1 + kyuubi-grpc-shade/pom.xml | 2 +- kyuubi-grpc/pom.xml | 1 - pom.xml | 2 +- 7 files changed, 194 insertions(+), 20 deletions(-) create mode 100644 kyuubi-grpc-server/src/main/resources/log4j2.xml diff --git a/kyuubi-grpc-server/pom.xml b/kyuubi-grpc-server/pom.xml index 8eb39c40011..03e84b8e444 100644 --- a/kyuubi-grpc-server/pom.xml +++ b/kyuubi-grpc-server/pom.xml @@ -31,23 +31,36 @@ https://kyuubi.apache.org/ - org.apache.spark spark-connect_2.13 3.5.1 + + + com.google.common + guava + + + - org.apache.kyuubi - kyuubi-grpc_${scala.binary.version} - ${project.version} + com.google.guava + failureaccess + 1.0.1 + + + + + org.apache.spark + spark-network-common_2.13 + 3.5.1 org.apache.kyuubi - kyuubi-common_${scala.binary.version} + kyuubi-grpc_${scala.binary.version} ${project.version} @@ -55,16 +68,74 @@ org.apache.kyuubi kyuubi-common_${scala.binary.version} ${project.version} - test-jar - test + + + org.slf4j + slf4j-api + + + org.slf4j + jcl-over-slf4j + + + org.slf4j + jul-to-slf4j + + + org.apache.logging.log4j + log4j-slf4j-impl + + + org.apache.logging.log4j + log4j-api + + + org.apache.logging.log4j + log4j-core + + + org.apache.logging.log4j + log4j-1.2-api + + org.apache.kyuubi - kyuubi-grpc_${scala.binary.version} + kyuubi-common_${scala.binary.version} ${project.version} test-jar test + + + org.slf4j + slf4j-api + + + org.slf4j + jcl-over-slf4j + + + org.slf4j + jul-to-slf4j + + + org.apache.logging.log4j + log4j-slf4j-impl + + + org.apache.logging.log4j + log4j-api + + + org.apache.logging.log4j + log4j-core + + + org.apache.logging.log4j + log4j-1.2-api + + @@ -72,13 +143,6 @@ spark-sql_${scala.binary.version} - - - org.apache.kyuubi - kyuubi-grpc-shade - ${project.version} - - io.grpc grpc-core @@ -93,6 +157,87 @@ javax.servlet-api - + + org.apache.kyuubi + kyuubi-grpc-shade + ${project.version} + + + + + + org.apache.maven.plugins + maven-patch-plugin + 1.2 + + + mylittle.patch + + + + + patch + process-sources + + apply + + + + + + + org.codehaus.mojo + build-helper-maven-plugin + + + add-scala-sources + + add-source + + generate-sources + + + src/main/scala-${scala.binary.version} + + + + + add-scala-test-sources + + add-test-source + + generate-test-sources + + + src/test/scala-${scala.binary.version} + + + + + + + org.apache.maven.plugins + maven-jar-plugin + + + prepare-test-jar + + test-jar + + test-compile + + + + + target/scala-${scala.binary.version}/classes + target/scala-${scala.binary.version}/test-classes + + + kr.motd.maven + os-maven-plugin + 1.6.2 + + + diff --git a/kyuubi-grpc-server/src/main/resources/log4j2.xml b/kyuubi-grpc-server/src/main/resources/log4j2.xml new file mode 100644 index 00000000000..6fa1ea32f63 --- /dev/null +++ b/kyuubi-grpc-server/src/main/resources/log4j2.xml @@ -0,0 +1,29 @@ + + + + + + + + + + + + + + diff --git a/kyuubi-grpc-server/src/main/scala/org/apache/kyuubi/grpc/server/KyuubiSparkConnectFrontendService.scala b/kyuubi-grpc-server/src/main/scala/org/apache/kyuubi/grpc/server/KyuubiSparkConnectFrontendService.scala index 0585219b4fd..8cbf30177a0 100644 --- a/kyuubi-grpc-server/src/main/scala/org/apache/kyuubi/grpc/server/KyuubiSparkConnectFrontendService.scala +++ b/kyuubi-grpc-server/src/main/scala/org/apache/kyuubi/grpc/server/KyuubiSparkConnectFrontendService.scala @@ -17,7 +17,7 @@ package org.apache.kyuubi.grpc.server -import scala.jdk.CollectionConverters.collectionAsScalaIterableConverter +import scala.jdk.CollectionConverters._ import com.google.protobuf.MessageLite import io.grpc._ diff --git a/kyuubi-grpc-server/src/main/scala/org/apache/kyuubi/grpc/session/KyuubiGrpcSession.scala b/kyuubi-grpc-server/src/main/scala/org/apache/kyuubi/grpc/session/KyuubiGrpcSession.scala index e2e475d88e9..93baa33b39e 100644 --- a/kyuubi-grpc-server/src/main/scala/org/apache/kyuubi/grpc/session/KyuubiGrpcSession.scala +++ b/kyuubi-grpc-server/src/main/scala/org/apache/kyuubi/grpc/session/KyuubiGrpcSession.scala @@ -16,6 +16,7 @@ */ package org.apache.kyuubi.grpc.session + import scala.util.Random import io.grpc.ManagedChannel diff --git a/kyuubi-grpc-shade/pom.xml b/kyuubi-grpc-shade/pom.xml index 6ad6dca3918..dc6c03c2601 100644 --- a/kyuubi-grpc-shade/pom.xml +++ b/kyuubi-grpc-shade/pom.xml @@ -49,7 +49,7 @@ false - org.apache.spark:spark-connect-common_2.12 + org.apache.spark:spark-connect-common_2.13 diff --git a/kyuubi-grpc/pom.xml b/kyuubi-grpc/pom.xml index 63b220695ff..0e8d1764696 100644 --- a/kyuubi-grpc/pom.xml +++ b/kyuubi-grpc/pom.xml @@ -22,7 +22,6 @@ org.apache.kyuubi kyuubi-parent 1.10.0-SNAPSHOT - ../pom.xml kyuubi-grpc_${scala.binary.version} diff --git a/pom.xml b/pom.xml index 681c3e1a5e2..cea4b623d47 100644 --- a/pom.xml +++ b/pom.xml @@ -84,8 +84,8 @@ kyuubi-util-scala kyuubi-zookeeper kyuubi-grpc - kyuubi-grpc-server kyuubi-grpc-shade + kyuubi-grpc-server