报告编号: INC-2026012901
报告日期: 2026年1月30日 故障时间: 2026年1月29日 19:53 - 20:xx
影响范围: 生产环境 HiveServer2 服务完全不可用 严重级别: P0 - 重大生产事故
目录
1. 执行摘要
1.1 问题概述
2026年1月29日 19:53,生产环境 HiveServer2 (HS2) 节点出现大量网络连接异常,随后服务端口无法响应。日志显示系统抛出 Too many open files 错误,导致 HS2 无法与 Metastore、HDFS 及 Redis 建立新连接。
1.2 根本原因
自定义 UDF
com.sohu.adp.redis.AbstractRedisUDF存在严重的资源管理缺陷:
- 非单例连接池设计:
redisClients被声明为实例变量而非静态变量- 编译期频繁实例化: Hive 优化器在常量折叠/传播阶段会创建大量临时 UDF 实例
- 连接池配置: 每个实例创建连接池时设置
MinIdle=5,立即建立 5 个长连接- 资源无法回收: UDF 临时实例被 GC 回收后,底层 TCP 连接未被正确关闭
1.3 影响量化
| 指标 | 数值 |
|---|---|
| 服务中断时长 | ~1 小时 |
| 泄漏连接池数量 | 590+ 个(基于 JMX MBean 命名推断) |
| 每池连接数 | 5 个(MinIdle 配置) |
| 总泄漏 FD 数 | 30,000+ |
| 系统 FD 上限 | 65,535 |
2. 故障时间线

3. Hive UDF 工作机制深度解析
3.1 UDF 执行位置:为什么是 HiveServer2 而非 MR/Tez NodeManager?
这是理解本次故障的核心问题。

关键结论
Redis 连接发生在 HiveServer2 而非 NodeManager 的原因:
| 阶段 | 执行位置 | 是否创建连接 | 说明 |
|---|---|---|---|
| SQL 编译优化 | HiveServer2 | ✅ 是 | 常量求值时实例化 UDF 并调用 evaluate() |
| 任务执行 | NodeManager | ✅ 是 | 正常的 UDF 执行流程 |
问题在于:编译阶段的 UDF 实例是临时的,执行完常量求值后即被丢弃,但其创建的连接池和 TCP 连接并未释放。
3.2 常量折叠 (Constant Folding) 机制

常量折叠是编译器优化技术,在编译期计算常量表达式的值。
触发条件:
- 表达式的所有操作数都是编译期可确定的常量
- 表达式本身是确定性的(Deterministic)
Hive 中的实现:
- 由
ConstantPropagateProcFactory类处理 - 在
SemanticAnalyzer阶段执行
3.3 常量传播 (Constant Propagation) 机制

常量传播将已知常量值传播到后续使用该变量的位置。
Hive 配置控制:
SET hive.optimize.constant.propagation = true; -- 默认开启 3.4 UDF 在优化阶段被实例化的时机

3.5 当前用法的核心问题
根据日志分析,每次 SQL 编译都会触发 UDF 实例化:
2026-01-29T19:48:05 - Compiling command: SELECT udf_redis_delivery(...)
2026-01-29T19:50:42 - Executing command: SELECT udf_redis_delivery(...)
问题链条:

4. 代码缺陷分析
4.1 反编译代码审查
以下是 AbstractRedisUDF.class 反编译后的关键代码结构:
public abstract class AbstractRedisUDF extends GenericUDF {
// ⚠️ 问题1:实例变量而非静态变量
protected transient StringObjectInspector[] stringOIs;
private transient Map<String, JedisCluster> redisClients; // ← 关键问题
@Override
public ObjectInspector initialize(ObjectInspector[] arguments)
throws UDFArgumentException {
// ... 参数校验 ...
// ⚠️ 问题2:每次 initialize 都创建新的 HashMap
this.redisClients = new HashMap<>();
return getOutputObjectInspector();
}
// ⚠️ 问题3:synchronized 仅锁当前实例,不同实例间无同步
protected synchronized JedisCluster getRedisClient(String apiUrl, String password) {
JedisCluster client = this.redisClients.get(apiUrl);
if (client == null) {
// 每次都 new 一个新的连接池
client = new CustomJedisCluster(
apiUrl,
2000, // connectionTimeout
2000, // soTimeout
5, // maxAttempts
password,
getPoolConfig()
);
this.redisClients.put(apiUrl, client);
}
return client;
}
protected JedisPoolConfig getPoolConfig() {
JedisPoolConfig poolConfig = new JedisPoolConfig();
poolConfig.setMinIdle(5); // ⚠️ 立即创建 5 个连接
poolConfig.setMaxIdle(8);
poolConfig.setMaxTotal(20);
poolConfig.setTestWhileIdle(true);
poolConfig.setTestOnBorrow(false);
poolConfig.setTestOnReturn(false);
poolConfig.setTestOnCreate(false);
poolConfig.setNumTestsPerEvictionRun(-1);
poolConfig.setTimeBetweenEvictionRunsMillis(300000); // 5分钟
poolConfig.setMinEvictableIdleTimeMillis(600000); // 10分钟
return poolConfig;
}
@Override
public void close() throws IOException {
// 尝试关闭连接池
for (JedisCluster client : this.redisClients.values()) {
client.close();
}
this.redisClients = null;
this.stringOIs = null;
}
}4.2 缺陷清单
| 编号 | 缺陷描述 | 严重程度 | 影响 | |
|---|---|---|---|---|
| D1 | redisClients 为实例变量而非静态变量 | 严重 | 每个 UDF 实例持有独立连接池 | |
| D2 | initialize() 每次创建新的 HashMap | 严重 | 无法复用现有连接 | |
| D3 | synchronized 仅锁实例级别 | 严重 | 多实例并发时无法防止重复创建 | |
| D4 | MinIdle=5 设置过高 | 中等 | 加速 FD 耗尽 | |
| D5 | close() 依赖 GC 调用,不可靠 | 严重 | 连接无法确保释放 | |
| D6 | 无连接池复用/共享机制 | 严重 | 每次查询创建新池 |
4.3 问题可视化

5. 根因分析
5.1 FD 泄漏路径

5.2 泄漏量估算
根据日志分析,从 1月3日重启到 1月29日崩溃(26天):
| 日期 | UDF 调用次数 | 预估新建连接池 | 预估 FD 增量 |
|---|---|---|---|
| 1/26 | ~8 次 | ~8 个 | ~40 |
| 1/27 | ~10 次 | ~10 个 | ~50 |
| 1/28 | ~8 次 | ~8 个 | ~40 |
| 1/29 | ~10 次 | ~10 个 | ~50 |
| 日均 | ~10 次 | ~10 个 | ~50 |
| 26天累计 | ~260 次 | ~260 个 | ~1,300 |
注意:实际泄漏量可能更高,原因:
- 常量传播优化可能多次实例化 UDF
- 日志可能未记录所有实例化
- JMX 显示 pool590+ 表明实际创建了更多实例
5.3 为何之前未爆发?

结论:
- 过去因为有周期性重启(运维操作或其他原因),FD 定期被释放
- 1月3日到29日期间没有重启,给了泄漏足够的累积时间
6. 修复方案
6.1 方案对比
| 方案 | 实现复杂度 | 侵入性 | 推荐指数 |
|---|---|---|---|
| A. 静态单例连接池 | ⭐⭐ | 中 | ⭐⭐⭐⭐⭐ |
| B. 双重检查锁单例 | ⭐⭐⭐ | 中 | ⭐⭐⭐⭐ |
| C. 静态内部类 Holder | ⭐⭐ | 低 | ⭐⭐⭐⭐⭐ |
| D. 使用 IoC 容器 | ⭐⭐⭐⭐ | 高 | ⭐⭐⭐ |
6.2 推荐方案:静态单例 + ConcurrentHashMap
public abstract class AbstractRedisUDF extends GenericUDF {
// ✅ 修复1:使用静态 ConcurrentHashMap 实现全局单例
private static final ConcurrentHashMap<String, JedisCluster> REDIS_CLIENTS =
new ConcurrentHashMap<>();
// ✅ 修复2:静态锁对象,确保全局同步
private static final Object LOCK = new Object();
protected transient StringObjectInspector[] stringOIs;
@Override
public ObjectInspector initialize(ObjectInspector[] arguments)
throws UDFArgumentException {
// 参数校验逻辑保持不变
// ...
// ✅ 修复3:移除 redisClients = new HashMap()
// 不再需要,因为使用静态变量
return getOutputObjectInspector();
}
// ✅ 修复4:使用 computeIfAbsent 确保线程安全的单例创建
protected JedisCluster getRedisClient(String apiUrl, String password) {
String cacheKey = apiUrl + "|" + password.hashCode(); // 组合键
return REDIS_CLIENTS.computeIfAbsent(cacheKey, key -> {
return new CustomJedisCluster(
apiUrl,
2000,
2000,
5,
password,
getPoolConfig()
);
});
}
protected JedisPoolConfig getPoolConfig() {
JedisPoolConfig poolConfig = new JedisPoolConfig();
// ✅ 修复5:降低 MinIdle,减少初始连接数
poolConfig.setMinIdle(1); // 从 5 降到 1
poolConfig.setMaxIdle(4); // 从 8 降到 4
poolConfig.setMaxTotal(10); // 从 20 降到 10
poolConfig.setTestWhileIdle(true);
poolConfig.setTestOnBorrow(false);
poolConfig.setTestOnReturn(false);
poolConfig.setTestOnCreate(false);
poolConfig.setNumTestsPerEvictionRun(-1);
poolConfig.setTimeBetweenEvictionRunsMillis(60000); // 1分钟
poolConfig.setMinEvictableIdleTimeMillis(300000); // 5分钟
return poolConfig;
}
@Override
public void close() throws IOException {
// ✅ 修复6:不再关闭连接池,因为是全局共享的
// 连接池生命周期与 JVM 一致
this.stringOIs = null;
// 注意:不要调用 REDIS_CLIENTS.clear() 或关闭连接
}
// ✅ 可选:添加 JVM 关闭钩子
static {
Runtime.getRuntime().addShutdownHook(new Thread(() -> {
for (JedisCluster client : REDIS_CLIENTS.values()) {
try {
client.close();
} catch (Exception e) {
// 忽略关闭异常
}
}
}));
}
}6.3 修复前后对比

6.4 部署步骤
- 代码修复
- 按照上述方案修改
AbstractRedisUDF.java - 进行单元测试和集成测试
- 按照上述方案修改
- 重新打包
mvn clean package -DskipTests # 生成新的 adp_udf-3.1.jar ``` - 灰度发布
- 先在测试环境验证
- 选择一个低流量 HS2 节点进行灰度
- 监控 FD 使用情况 72 小时
- 全量发布
- 滚动更新所有 HS2 节点
- 更新 Hive 函数注册
7. 预防措施与最佳实践
7.1 UDF 开发规范

7.2 监控告警建议
| 监控项 | 指标 | 告警阈值 | 级别 |
|---|---|---|---|
| 进程 FD 使用率 | process_open_fds / process_max_fds | > 70% | P2 |
| 进程 FD 使用率 | process_open_fds / process_max_fds | > 85% | P1 |
| TCP 连接数 | `netstat -an | grep ESTABLISHED | wc -l` |
| 连接池数量 | JMX pool* MBean 数量 | > 50 | P2 |
| Redis 连接增长率 | delta(redis_connections, 1h) | > 100/h | P2 |
7.3 运维建议
- 定期重启策略(短期缓解)
# 建议每周重启一次 HS2(错峰) 0 3 * * 0 systemctl restart hive-server2 - ulimit 调整
# /etc/security/limits.d/hive.conf hive soft nofile 200000 hive hard nofile 200000 - FD 使用监控脚本
#!/bin/bash # 每分钟检查 HS2 进程 FD 使用情况 HS2_PID=$(pgrep -f HiveServer2) FD_COUNT=$(ls /proc/$HS2_PID/fd 2>/dev/null | wc -l) FD_LIMIT=$(cat /proc/$HS2_PID/limits | grep "Max open files" | awk '{print $4}') USAGE_PCT=$((FD_COUNT * 100 / FD_LIMIT)) echo "$(date): HS2 FD Usage: $FD_COUNT / $FD_LIMIT ($USAGE_PCT%)" if [ $USAGE_PCT -gt 80 ]; then # 发送告警 curl -X POST "https://alert-api/send" \ -d "message=HS2 FD Usage Critical: $USAGE_PCT%" fi
8. 附录
8.1 关键日志摘要
2026-01-29T19:53:58 - Could not retrieve canonical hostname
2026-01-29T19:53:58 - Failed to connect to the MetaStore Server
2026-01-29T19:56:39 - java.net.SocketException: Too many open files
2026-01-29T19:57:26 - java.net.UnknownHostException: dnn014013
8.2 JMX MBean 证据
pool590: {NumIdle: 5, NumActive: 0}
pool591: {NumIdle: 5, NumActive: 0}
pool592: {NumIdle: 5, NumActive: 0}
...
8.3 反编译代码完整结构
AbstractRedisUDF.class
├── 字段
│ ├── stringOIs: StringObjectInspector[] (transient, protected)
│ └── redisClients: Map<String, JedisCluster> (transient, private) ⚠️
├── 方法
│ ├── initialize(ObjectInspector[]): 创建新 HashMap│ ├── getRedisClient(String, String): synchronized, 创建新 JedisCluster│ ├── getPoolConfig(): MinIdle=5, MaxIdle=8, MaxTotal=20
│ └── close(): 遍历关闭连接池
8.4 相关配置参数
-- 临时禁用常量传播(短期缓解)
SET hive.optimize.constant.propagation = false;
-- 查看当前设置
SET hive.optimize.constant.propagation;