Skip to content

Commit 53034a3

Browse files
turboFeipan3793
authored andcommitted
[KYUUBI #6866] Add metrics for SSL keystore expiration time
### Why are the changes needed? Add metrics for SSL keystore expiration, then we can add alert if the keystore will expire in 1 month. ### How was this patch tested? Integration testing. <img width="1721" alt="image" src="https://github.com/user-attachments/assets/f4ef6af6-923b-403c-a80d-06dbb80dbe1c" /> ### Was this patch authored or co-authored using generative AI tooling? No. Closes #6866 from turboFei/keystore_expire. Closes #6866 77c6db0 [Wang, Fei] Add metrics for SSL keystore expiration time #6866 Authored-by: Wang, Fei <[email protected]> Signed-off-by: Cheng Pan <[email protected]>
1 parent 031caf3 commit 53034a3

File tree

5 files changed

+98
-7
lines changed

5 files changed

+98
-7
lines changed

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

+7-3
Original file line numberDiff line numberDiff line change
@@ -53,6 +53,10 @@ abstract class TBinaryFrontendService(name: String)
5353
private var _actualPort: Int = _
5454
override protected lazy val actualPort: Int = _actualPort
5555

56+
protected var keyStorePath: Option[String] = None
57+
protected var keyStorePassword: Option[String] = None
58+
protected var keyStoreType: Option[String] = None
59+
5660
// Removed OOM hook since Kyuubi #1800 to respect the hive server2 #2383
5761

5862
override def initialize(conf: KyuubiConf): Unit = synchronized {
@@ -73,9 +77,9 @@ abstract class TBinaryFrontendService(name: String)
7377
val tServerSocket =
7478
// only enable ssl for server side
7579
if (isServer() && conf.get(FRONTEND_THRIFT_BINARY_SSL_ENABLED)) {
76-
val keyStorePath = conf.get(FRONTEND_SSL_KEYSTORE_PATH)
77-
val keyStorePassword = conf.get(FRONTEND_SSL_KEYSTORE_PASSWORD)
78-
val keyStoreType = conf.get(FRONTEND_SSL_KEYSTORE_TYPE)
80+
keyStorePath = conf.get(FRONTEND_SSL_KEYSTORE_PATH)
81+
keyStorePassword = conf.get(FRONTEND_SSL_KEYSTORE_PASSWORD)
82+
keyStoreType = conf.get(FRONTEND_SSL_KEYSTORE_TYPE)
7983
val keyStoreAlgorithm = conf.get(FRONTEND_SSL_KEYSTORE_ALGORITHM)
8084
val disallowedSslProtocols = conf.get(FRONTEND_THRIFT_BINARY_SSL_DISALLOWED_PROTOCOLS)
8185
val includeCipherSuites = conf.get(FRONTEND_THRIFT_BINARY_SSL_INCLUDE_CIPHER_SUITES)

kyuubi-metrics/src/main/scala/org/apache/kyuubi/metrics/MetricsConstants.scala

+2
Original file line numberDiff line numberDiff line change
@@ -37,6 +37,8 @@ object MetricsConstants {
3737
final private val THRIFT_BINARY_CONN = KYUUBI + "thrift.binary.connection."
3838
final private val REST_CONN = KYUUBI + "rest.connection."
3939

40+
final val THRIFT_SSL_CERT_EXPIRATION = KYUUBI + "thrift.ssl.cert.expiration"
41+
4042
final val CONN_OPEN: String = CONN + "opened"
4143
final val CONN_FAIL: String = CONN + "failed"
4244
final val CONN_TOTAL: String = CONN + "total"

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

+6
Original file line numberDiff line numberDiff line change
@@ -35,6 +35,7 @@ import org.apache.kyuubi.session.KyuubiSessionImpl
3535
import org.apache.kyuubi.shaded.hive.service.rpc.thrift._
3636
import org.apache.kyuubi.shaded.thrift.protocol.TProtocol
3737
import org.apache.kyuubi.shaded.thrift.server.ServerContext
38+
import org.apache.kyuubi.util.SSLUtils
3839

3940
final class KyuubiTBinaryFrontendService(
4041
override val serverable: Serverable)
@@ -122,4 +123,9 @@ final class KyuubiTBinaryFrontendService(
122123
resp.setStatus(notSupportTokenErrorStatus)
123124
resp
124125
}
126+
127+
override def start(): Unit = {
128+
super.start()
129+
SSLUtils.tracingThriftSSLCertExpiration(keyStorePath, keyStorePassword, keyStoreType)
130+
}
125131
}

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

+13-4
Original file line numberDiff line numberDiff line change
@@ -50,7 +50,7 @@ import org.apache.kyuubi.service.TFrontendService.{CURRENT_SERVER_CONTEXT, OK_ST
5050
import org.apache.kyuubi.session.KyuubiSessionImpl
5151
import org.apache.kyuubi.shaded.hive.service.rpc.thrift.{TCLIService, TOpenSessionReq, TOpenSessionResp}
5252
import org.apache.kyuubi.shaded.thrift.protocol.TBinaryProtocol
53-
import org.apache.kyuubi.util.NamedThreadFactory
53+
import org.apache.kyuubi.util.{NamedThreadFactory, SSLUtils}
5454

5555
/**
5656
* Apache Thrift based hive service rpc
@@ -67,6 +67,10 @@ final class KyuubiTHttpFrontendService(
6767
override protected lazy val actualPort: Int = portNum
6868
override protected lazy val serverSocket: ServerSocket = null
6969

70+
private var keyStorePath: Option[String] = None
71+
private var keyStorePassword: Option[String] = None
72+
private var keyStoreType: Option[String] = None
73+
7074
private var server: Option[Server] = None
7175
private val APPLICATION_THRIFT = "application/x-thrift"
7276

@@ -122,15 +126,15 @@ final class KyuubiTHttpFrontendService(
122126
// Change connector if SSL is used
123127
val connector =
124128
if (useSsl) {
125-
val keyStorePath = conf.get(FRONTEND_THRIFT_HTTP_SSL_KEYSTORE_PATH)
129+
keyStorePath = conf.get(FRONTEND_THRIFT_HTTP_SSL_KEYSTORE_PATH)
126130

127131
if (keyStorePath.isEmpty) {
128132
throw new IllegalArgumentException(FRONTEND_THRIFT_HTTP_SSL_KEYSTORE_PATH.key +
129133
" Not configured for SSL connection, please set the key with: " +
130134
FRONTEND_THRIFT_HTTP_SSL_KEYSTORE_PATH.doc)
131135
}
132136

133-
val keyStorePassword = conf.get(FRONTEND_THRIFT_HTTP_SSL_KEYSTORE_PASSWORD)
137+
keyStorePassword = conf.get(FRONTEND_THRIFT_HTTP_SSL_KEYSTORE_PASSWORD)
134138
if (keyStorePassword.isEmpty) {
135139
throw new IllegalArgumentException(FRONTEND_THRIFT_HTTP_SSL_KEYSTORE_PASSWORD.key +
136140
" Not configured for SSL connection. please set the key with: " +
@@ -140,7 +144,7 @@ final class KyuubiTHttpFrontendService(
140144
val sslContextFactory = new SslContextFactory.Server
141145
val excludedProtocols = conf.get(FRONTEND_THRIFT_HTTP_SSL_PROTOCOL_BLACKLIST)
142146
val excludeCipherSuites = conf.get(FRONTEND_THRIFT_HTTP_SSL_EXCLUDE_CIPHER_SUITES)
143-
val keyStoreType = conf.get(FRONTEND_SSL_KEYSTORE_TYPE)
147+
keyStoreType = conf.get(FRONTEND_SSL_KEYSTORE_TYPE)
144148
val keyStoreAlgorithm = conf.get(FRONTEND_SSL_KEYSTORE_ALGORITHM)
145149
info("Thrift HTTP Server SSL: adding excluded protocols: " +
146150
String.join(",", excludedProtocols: _*))
@@ -359,4 +363,9 @@ final class KyuubiTHttpFrontendService(
359363

360364
ret
361365
}
366+
367+
override def start(): Unit = {
368+
super.start()
369+
SSLUtils.tracingThriftSSLCertExpiration(keyStorePath, keyStorePassword, keyStoreType)
370+
}
362371
}
Original file line numberDiff line numberDiff line change
@@ -0,0 +1,70 @@
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+
18+
package org.apache.kyuubi.util
19+
import java.io.FileInputStream
20+
import java.security.KeyStore
21+
import java.security.cert.X509Certificate
22+
23+
import scala.collection.JavaConverters._
24+
25+
import org.apache.kyuubi.{Logging, Utils}
26+
import org.apache.kyuubi.metrics.{MetricsConstants, MetricsSystem}
27+
28+
object SSLUtils extends Logging {
29+
30+
/**
31+
* Get the keystore certificate latest expiration time.
32+
*/
33+
private def getKeyStoreExpirationTime(
34+
keyStorePath: String,
35+
keyStorePassword: String,
36+
keyStoreType: Option[String]): Option[Long] = {
37+
try {
38+
val keyStore = KeyStore.getInstance(keyStoreType.getOrElse(KeyStore.getDefaultType))
39+
keyStore.load(new FileInputStream(keyStorePath), keyStorePassword.toCharArray)
40+
keyStore.aliases().asScala.toSeq.map { alias =>
41+
keyStore.getCertificate(alias).asInstanceOf[X509Certificate].getNotAfter.getTime
42+
}.sorted.headOption
43+
} catch {
44+
case e: Throwable =>
45+
error("Error getting keystore expiration time.", e)
46+
None
47+
}
48+
}
49+
50+
def tracingThriftSSLCertExpiration(
51+
keyStorePath: Option[String],
52+
keyStorePassword: Option[String],
53+
keyStoreType: Option[String]): Unit = {
54+
if (keyStorePath.isDefined && keyStorePassword.isDefined) {
55+
SSLUtils.getKeyStoreExpirationTime(
56+
keyStorePath.get,
57+
keyStorePassword.get,
58+
keyStoreType).foreach { expiration =>
59+
info(s"Thrift SSL Serve KeyStore ${keyStorePath.get} will expire at:" +
60+
s" ${Utils.getDateFromTimestamp(expiration)}")
61+
MetricsSystem.tracing { ms =>
62+
ms.registerGauge(
63+
MetricsConstants.THRIFT_SSL_CERT_EXPIRATION,
64+
expiration - System.currentTimeMillis(),
65+
0L)
66+
}
67+
}
68+
}
69+
}
70+
}

0 commit comments

Comments
 (0)