Skip to content

Commit d086525

Browse files
zhaomin1423ulysses-you
authored andcommitted
[KYUUBI #3098] Unify the event log code path
### _Why are the changes needed?_ close #3098 (comment) ### _How was this patch tested?_ - [x] Add some test cases that check the changes thoroughly including negative and positive cases if possible - [ ] Add screenshots for manual tests if appropriate - [ ] [Run test](https://kyuubi.apache.org/docs/latest/develop_tools/testing.html#running-tests) locally before make a pull request Closes #3090 from zhaomin1423/event_spi. Closes #3098 322e48d [Min] clear unnecessary variable 22b448d [Min] fix review a452321 [Min] [KYUUBI #3098] Unify the event log code path Authored-by: Min <zhaomin1423@163.com> Signed-off-by: ulysses-you <ulyssesyou@apache.org>
1 parent 6aa898e commit d086525

File tree

9 files changed

+257
-92
lines changed

9 files changed

+257
-92
lines changed

externals/kyuubi-hive-sql-engine/src/main/scala/org/apache/kyuubi/engine/hive/HiveSQLEngine.scala

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

1818
package org.apache.kyuubi.engine.hive
1919

20-
import java.net.InetAddress
2120
import java.security.PrivilegedExceptionAction
2221

2322
import scala.util.control.NonFatal
@@ -27,15 +26,14 @@ import org.apache.hadoop.security.UserGroupInformation
2726

2827
import org.apache.kyuubi.{Logging, Utils}
2928
import org.apache.kyuubi.config.{KyuubiConf, KyuubiReservedKeys}
30-
import org.apache.kyuubi.config.KyuubiConf.{ENGINE_EVENT_JSON_LOG_PATH, ENGINE_EVENT_LOGGERS}
29+
import org.apache.kyuubi.config.KyuubiConf.ENGINE_EVENT_LOGGERS
3130
import org.apache.kyuubi.engine.hive.HiveSQLEngine.currentEngine
32-
import org.apache.kyuubi.engine.hive.events.HiveEngineEvent
33-
import org.apache.kyuubi.engine.hive.events.handler.HiveJsonLoggingEventHandler
34-
import org.apache.kyuubi.events.{EventBus, EventLoggerType, KyuubiEvent}
31+
import org.apache.kyuubi.engine.hive.events.{HiveEngineEvent, HiveEventHandlerRegister}
32+
import org.apache.kyuubi.events.EventBus
3533
import org.apache.kyuubi.ha.HighAvailabilityConf.HA_ZK_CONN_RETRY_POLICY
3634
import org.apache.kyuubi.ha.client.RetryPolicies
3735
import org.apache.kyuubi.service.{AbstractBackendService, AbstractFrontendService, Serverable, ServiceState}
38-
import org.apache.kyuubi.util.{KyuubiHadoopUtils, SignalRegister}
36+
import org.apache.kyuubi.util.SignalRegister
3937

4038
class HiveSQLEngine extends Serverable("HiveSQLEngine") {
4139
override val backendService: AbstractBackendService = new HiveBackendService(this)
@@ -115,20 +113,7 @@ object HiveSQLEngine extends Logging {
115113
}
116114

117115
private def initLoggerEventHandler(conf: KyuubiConf): Unit = {
118-
val hadoopConf = KyuubiHadoopUtils.newHadoopConf(conf)
119-
conf.get(ENGINE_EVENT_LOGGERS).map(EventLoggerType.withName).foreach {
120-
case EventLoggerType.JSON =>
121-
val hostName = InetAddress.getLocalHost.getCanonicalHostName
122-
val handler = HiveJsonLoggingEventHandler(
123-
s"Hive-$hostName",
124-
ENGINE_EVENT_JSON_LOG_PATH,
125-
hadoopConf,
126-
conf)
127-
EventBus.register[KyuubiEvent](handler)
128-
case logger =>
129-
throw new IllegalArgumentException(s"Unrecognized event logger: $logger")
130-
131-
}
116+
HiveEventHandlerRegister.registerEngineEventLoggers(conf)
132117
}
133118

134119
def main(args: Array[String]): Unit = {
Original file line numberDiff line numberDiff line change
@@ -0,0 +1,40 @@
1+
/*
2+
* Licensed to the Apache Software Foundation (ASF) under one or more
3+
* contributor license agreements. See the NOTICE file distributed with
4+
* this work for additional information regarding copyright ownership.
5+
* The ASF licenses this file to You under the Apache License, Version 2.0
6+
* (the "License"); you may not use this file except in compliance with
7+
* the License. You may obtain a copy of the License at
8+
*
9+
* http://www.apache.org/licenses/LICENSE-2.0
10+
*
11+
* Unless required by applicable law or agreed to in writing, software
12+
* distributed under the License is distributed on an "AS IS" BASIS,
13+
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
14+
* See the License for the specific language governing permissions and
15+
* limitations under the License.
16+
*/
17+
package org.apache.kyuubi.engine.hive.events
18+
19+
import java.net.InetAddress
20+
21+
import org.apache.kyuubi.config.KyuubiConf
22+
import org.apache.kyuubi.config.KyuubiConf.ENGINE_EVENT_JSON_LOG_PATH
23+
import org.apache.kyuubi.engine.hive.events.handler.HiveJsonLoggingEventHandler
24+
import org.apache.kyuubi.events.{EventHandlerRegister, KyuubiEvent}
25+
import org.apache.kyuubi.events.handler.EventHandler
26+
import org.apache.kyuubi.util.KyuubiHadoopUtils
27+
28+
object HiveEventHandlerRegister extends EventHandlerRegister {
29+
30+
override protected def createJsonEventHandler(
31+
kyuubiConf: KyuubiConf): EventHandler[KyuubiEvent] = {
32+
val hadoopConf = KyuubiHadoopUtils.newHadoopConf(kyuubiConf)
33+
val hostName = InetAddress.getLocalHost.getCanonicalHostName
34+
HiveJsonLoggingEventHandler(
35+
s"Hive-$hostName",
36+
ENGINE_EVENT_JSON_LOG_PATH,
37+
hadoopConf,
38+
kyuubiConf)
39+
}
40+
}

externals/kyuubi-spark-sql-engine/src/main/scala/org/apache/kyuubi/engine/spark/SparkSQLEngine.scala

Lines changed: 4 additions & 23 deletions
Original file line numberDiff line numberDiff line change
@@ -35,9 +35,8 @@ import org.apache.kyuubi.config.{KyuubiConf, KyuubiReservedKeys}
3535
import org.apache.kyuubi.config.KyuubiConf._
3636
import org.apache.kyuubi.config.KyuubiReservedKeys.KYUUBI_ENGINE_SUBMIT_TIME_KEY
3737
import org.apache.kyuubi.engine.spark.SparkSQLEngine.{countDownLatch, currentEngine}
38-
import org.apache.kyuubi.engine.spark.events.{EngineEvent, EngineEventsStore}
39-
import org.apache.kyuubi.engine.spark.events.handler.{SparkHistoryLoggingEventHandler, SparkJsonLoggingEventHandler}
40-
import org.apache.kyuubi.events.{EventBus, EventLoggerType, KyuubiEvent}
38+
import org.apache.kyuubi.engine.spark.events.{EngineEvent, EngineEventsStore, SparkEventHandlerRegister}
39+
import org.apache.kyuubi.events.EventBus
4140
import org.apache.kyuubi.ha.HighAvailabilityConf._
4241
import org.apache.kyuubi.ha.client.RetryPolicies
4342
import org.apache.kyuubi.service.Serverable
@@ -235,27 +234,9 @@ object SparkSQLEngine extends Logging {
235234
}
236235

237236
def initLoggerEventHandler(conf: KyuubiConf): Unit = {
238-
conf.get(ENGINE_EVENT_LOGGERS)
239-
.map(EventLoggerType.withName)
240-
.foreach {
241-
case EventLoggerType.SPARK =>
242-
EventBus.register[KyuubiEvent](new SparkHistoryLoggingEventHandler(spark.sparkContext))
243-
case EventLoggerType.JSON =>
244-
val handler = SparkJsonLoggingEventHandler(
245-
spark.sparkContext.applicationAttemptId
246-
.map(id => s"${spark.sparkContext.applicationId}_$id")
247-
.getOrElse(spark.sparkContext.applicationId),
248-
ENGINE_EVENT_JSON_LOG_PATH,
249-
spark.sparkContext.hadoopConfiguration,
250-
conf)
251-
252-
// register JsonLogger as a event handler for default event bus
253-
EventBus.register[KyuubiEvent](handler)
254-
case _ =>
255-
}
256-
237+
val sparkEventRegister = new SparkEventHandlerRegister(spark)
238+
sparkEventRegister.registerEngineEventLoggers(conf)
257239
}
258-
259240
}
260241

261242
def main(args: Array[String]): Unit = {
Original file line numberDiff line numberDiff line change
@@ -0,0 +1,45 @@
1+
/*
2+
* Licensed to the Apache Software Foundation (ASF) under one or more
3+
* contributor license agreements. See the NOTICE file distributed with
4+
* this work for additional information regarding copyright ownership.
5+
* The ASF licenses this file to You under the Apache License, Version 2.0
6+
* (the "License"); you may not use this file except in compliance with
7+
* the License. You may obtain a copy of the License at
8+
*
9+
* http://www.apache.org/licenses/LICENSE-2.0
10+
*
11+
* Unless required by applicable law or agreed to in writing, software
12+
* distributed under the License is distributed on an "AS IS" BASIS,
13+
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
14+
* See the License for the specific language governing permissions and
15+
* limitations under the License.
16+
*/
17+
package org.apache.kyuubi.engine.spark.events
18+
19+
import org.apache.spark.sql.SparkSession
20+
21+
import org.apache.kyuubi.config.KyuubiConf
22+
import org.apache.kyuubi.config.KyuubiConf.ENGINE_EVENT_JSON_LOG_PATH
23+
import org.apache.kyuubi.engine.spark.events.handler.{SparkHistoryLoggingEventHandler, SparkJsonLoggingEventHandler}
24+
import org.apache.kyuubi.events.{EventHandlerRegister, KyuubiEvent}
25+
import org.apache.kyuubi.events.handler.EventHandler
26+
27+
class SparkEventHandlerRegister(spark: SparkSession) extends EventHandlerRegister {
28+
29+
override protected def createSparkEventHandler(kyuubiConf: KyuubiConf)
30+
: EventHandler[KyuubiEvent] = {
31+
new SparkHistoryLoggingEventHandler(spark.sparkContext)
32+
}
33+
34+
override protected def createJsonEventHandler(kyuubiConf: KyuubiConf)
35+
: EventHandler[KyuubiEvent] = {
36+
SparkJsonLoggingEventHandler(
37+
spark.sparkContext.applicationAttemptId
38+
.map(id => s"${spark.sparkContext.applicationId}_$id")
39+
.getOrElse(spark.sparkContext.applicationId),
40+
ENGINE_EVENT_JSON_LOG_PATH,
41+
spark.sparkContext.hadoopConfiguration,
42+
kyuubiConf)
43+
}
44+
45+
}

externals/kyuubi-trino-engine/src/main/scala/org/apache/kyuubi/engine/trino/TrinoSqlEngine.scala

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

1818
package org.apache.kyuubi.engine.trino
1919

20-
import java.net.InetAddress
2120
import java.util.concurrent.CountDownLatch
2221

2322
import scala.util.control.NonFatal
2423

2524
import org.apache.kyuubi.{Logging, Utils}
26-
import org.apache.kyuubi.Utils.TRINO_ENGINE_SHUTDOWN_PRIORITY
27-
import org.apache.kyuubi.Utils.addShutdownHook
25+
import org.apache.kyuubi.Utils.{addShutdownHook, TRINO_ENGINE_SHUTDOWN_PRIORITY}
2826
import org.apache.kyuubi.config.KyuubiConf
29-
import org.apache.kyuubi.config.KyuubiConf.{ENGINE_EVENT_JSON_LOG_PATH, ENGINE_EVENT_LOGGERS}
30-
import org.apache.kyuubi.engine.trino.TrinoSqlEngine.countDownLatch
31-
import org.apache.kyuubi.engine.trino.TrinoSqlEngine.currentEngine
32-
import org.apache.kyuubi.engine.trino.event.TrinoEngineEvent
33-
import org.apache.kyuubi.engine.trino.event.handler.TrinoJsonLoggingEventHandler
34-
import org.apache.kyuubi.events.{EventBus, EventLoggerType, KyuubiEvent}
27+
import org.apache.kyuubi.config.KyuubiConf.ENGINE_EVENT_LOGGERS
28+
import org.apache.kyuubi.engine.trino.TrinoSqlEngine.{countDownLatch, currentEngine}
29+
import org.apache.kyuubi.engine.trino.event.{TrinoEngineEvent, TrinoEventHandlerRegister}
30+
import org.apache.kyuubi.events.EventBus
3531
import org.apache.kyuubi.ha.HighAvailabilityConf.HA_ZK_CONN_RETRY_POLICY
3632
import org.apache.kyuubi.ha.client.RetryPolicies
3733
import org.apache.kyuubi.service.Serverable
38-
import org.apache.kyuubi.util.{KyuubiHadoopUtils, SignalRegister}
34+
import org.apache.kyuubi.util.SignalRegister
3935

4036
case class TrinoSqlEngine()
4137
extends Serverable("TrinoSQLEngine") {
@@ -92,19 +88,7 @@ object TrinoSqlEngine extends Logging {
9288
}
9389

9490
private def initLoggerEventHandler(conf: KyuubiConf): Unit = {
95-
val hadoopConf = KyuubiHadoopUtils.newHadoopConf(conf)
96-
conf.get(ENGINE_EVENT_LOGGERS).map(EventLoggerType.withName).foreach {
97-
case EventLoggerType.JSON =>
98-
val hostName = InetAddress.getLocalHost.getCanonicalHostName
99-
val handler = TrinoJsonLoggingEventHandler(
100-
s"Trino-$hostName",
101-
ENGINE_EVENT_JSON_LOG_PATH,
102-
hadoopConf,
103-
conf)
104-
EventBus.register[KyuubiEvent](handler)
105-
case logger =>
106-
throw new IllegalArgumentException(s"Unrecognized event logger: $logger")
107-
}
91+
TrinoEventHandlerRegister.registerEngineEventLoggers(conf)
10892
}
10993

11094
def main(args: Array[String]): Unit = {
Original file line numberDiff line numberDiff line change
@@ -0,0 +1,40 @@
1+
/*
2+
* Licensed to the Apache Software Foundation (ASF) under one or more
3+
* contributor license agreements. See the NOTICE file distributed with
4+
* this work for additional information regarding copyright ownership.
5+
* The ASF licenses this file to You under the Apache License, Version 2.0
6+
* (the "License"); you may not use this file except in compliance with
7+
* the License. You may obtain a copy of the License at
8+
*
9+
* http://www.apache.org/licenses/LICENSE-2.0
10+
*
11+
* Unless required by applicable law or agreed to in writing, software
12+
* distributed under the License is distributed on an "AS IS" BASIS,
13+
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
14+
* See the License for the specific language governing permissions and
15+
* limitations under the License.
16+
*/
17+
package org.apache.kyuubi.engine.trino.event
18+
19+
import java.net.InetAddress
20+
21+
import org.apache.kyuubi.config.KyuubiConf
22+
import org.apache.kyuubi.config.KyuubiConf.ENGINE_EVENT_JSON_LOG_PATH
23+
import org.apache.kyuubi.engine.trino.event.handler.TrinoJsonLoggingEventHandler
24+
import org.apache.kyuubi.events.{EventHandlerRegister, KyuubiEvent}
25+
import org.apache.kyuubi.events.handler.EventHandler
26+
import org.apache.kyuubi.util.KyuubiHadoopUtils
27+
28+
object TrinoEventHandlerRegister extends EventHandlerRegister {
29+
30+
override protected def createJsonEventHandler(kyuubiConf: KyuubiConf)
31+
: EventHandler[KyuubiEvent] = {
32+
val hadoopConf = KyuubiHadoopUtils.newHadoopConf(kyuubiConf)
33+
val hostName = InetAddress.getLocalHost.getCanonicalHostName
34+
TrinoJsonLoggingEventHandler(
35+
s"Trino-$hostName",
36+
ENGINE_EVENT_JSON_LOG_PATH,
37+
hadoopConf,
38+
kyuubiConf)
39+
}
40+
}
Lines changed: 74 additions & 0 deletions
Original file line numberDiff line numberDiff line change
@@ -0,0 +1,74 @@
1+
/*
2+
* Licensed to the Apache Software Foundation (ASF) under one or more
3+
* contributor license agreements. See the NOTICE file distributed with
4+
* this work for additional information regarding copyright ownership.
5+
* The ASF licenses this file to You under the Apache License, Version 2.0
6+
* (the "License"); you may not use this file except in compliance with
7+
* the License. You may obtain a copy of the License at
8+
*
9+
* http://www.apache.org/licenses/LICENSE-2.0
10+
*
11+
* Unless required by applicable law or agreed to in writing, software
12+
* distributed under the License is distributed on an "AS IS" BASIS,
13+
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
14+
* See the License for the specific language governing permissions and
15+
* limitations under the License.
16+
*/
17+
package org.apache.kyuubi.events
18+
19+
import org.apache.kyuubi.{KyuubiException, Logging}
20+
import org.apache.kyuubi.config.KyuubiConf
21+
import org.apache.kyuubi.config.KyuubiConf.{ENGINE_EVENT_LOGGERS, SERVER_EVENT_LOGGERS}
22+
import org.apache.kyuubi.events.EventLoggerType.EventLoggerType
23+
import org.apache.kyuubi.events.handler.EventHandler
24+
25+
trait EventHandlerRegister extends Logging {
26+
27+
def registerEngineEventLoggers(conf: KyuubiConf): Unit = {
28+
val loggers = conf.get(ENGINE_EVENT_LOGGERS)
29+
register(loggers, conf)
30+
}
31+
32+
def registerServerEventLoggers(conf: KyuubiConf): Unit = {
33+
val loggers = conf.get(SERVER_EVENT_LOGGERS)
34+
register(loggers, conf)
35+
}
36+
37+
private def register(loggers: Seq[String], conf: KyuubiConf): Unit = {
38+
loggers
39+
.map(EventLoggerType.withName)
40+
.foreach { logger =>
41+
EventBus.register(loadEventHandler(logger, conf))
42+
}
43+
}
44+
45+
protected def createSparkEventHandler(kyuubiConf: KyuubiConf): EventHandler[KyuubiEvent] = {
46+
throw new KyuubiException(s"Unsupported spark event logger.")
47+
}
48+
49+
protected def createJsonEventHandler(kyuubiConf: KyuubiConf): EventHandler[KyuubiEvent] = {
50+
throw new KyuubiException(s"Unsupported json event logger.")
51+
}
52+
53+
protected def createJdbcEventHandler(kyuubiConf: KyuubiConf): EventHandler[KyuubiEvent] = {
54+
throw new KyuubiException(s"Unsupported jdbc event logger.")
55+
}
56+
57+
private def loadEventHandler(
58+
eventLoggerType: EventLoggerType,
59+
kyuubiConf: KyuubiConf): EventHandler[KyuubiEvent] = {
60+
eventLoggerType match {
61+
case EventLoggerType.SPARK =>
62+
createSparkEventHandler(kyuubiConf)
63+
64+
case EventLoggerType.JSON =>
65+
createJsonEventHandler(kyuubiConf)
66+
67+
case EventLoggerType.JDBC =>
68+
createJdbcEventHandler(kyuubiConf)
69+
70+
case other =>
71+
throw new KyuubiException(s"Unsupported event logger: ${other.toString}")
72+
}
73+
}
74+
}
Lines changed: 38 additions & 0 deletions
Original file line numberDiff line numberDiff line change
@@ -0,0 +1,38 @@
1+
/*
2+
* Licensed to the Apache Software Foundation (ASF) under one or more
3+
* contributor license agreements. See the NOTICE file distributed with
4+
* this work for additional information regarding copyright ownership.
5+
* The ASF licenses this file to You under the Apache License, Version 2.0
6+
* (the "License"); you may not use this file except in compliance with
7+
* the License. You may obtain a copy of the License at
8+
*
9+
* http://www.apache.org/licenses/LICENSE-2.0
10+
*
11+
* Unless required by applicable law or agreed to in writing, software
12+
* distributed under the License is distributed on an "AS IS" BASIS,
13+
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
14+
* See the License for the specific language governing permissions and
15+
* limitations under the License.
16+
*/
17+
package org.apache.kyuubi.events
18+
19+
import java.net.InetAddress
20+
21+
import org.apache.kyuubi.config.KyuubiConf
22+
import org.apache.kyuubi.config.KyuubiConf.SERVER_EVENT_JSON_LOG_PATH
23+
import org.apache.kyuubi.events.handler.{EventHandler, ServerJsonLoggingEventHandler}
24+
import org.apache.kyuubi.util.KyuubiHadoopUtils
25+
26+
object ServerEventHandlerRegister extends EventHandlerRegister {
27+
28+
override protected def createJsonEventHandler(kyuubiConf: KyuubiConf)
29+
: EventHandler[KyuubiEvent] = {
30+
val hadoopConf = KyuubiHadoopUtils.newHadoopConf(kyuubiConf)
31+
val hostName = InetAddress.getLocalHost.getCanonicalHostName
32+
ServerJsonLoggingEventHandler(
33+
s"server-$hostName",
34+
SERVER_EVENT_JSON_LOG_PATH,
35+
hadoopConf,
36+
kyuubiConf)
37+
}
38+
}

0 commit comments

Comments
 (0)