在存算分离的场景下,会出现数据先出现在存储介质上,然后再在hive中创建表进行关联,这种情况下如果是分区表,hms里面实际是没有分区信息的,导致查询的时候,是无法找到对应的数据的。
一般解决办法就两种,一种是手动针对每一个分区执行add partition操作,另一种就是执行一下msck一次进行全表修复,关于msck可以参考:https://baifachuan.com/posts/52be3ba3.html

因为msck是全表修复,所以有的人会选择在调度器里面配置一个定时作业,周期性的执行一下msck,确保进来的数据都可以被查询到,这种思路本身没有问题,但是最近碰到一个问题,某套集群在执行msck的时候,速度非常慢。

一共就1000来个分区,跑了3小时才在hms后的db里面新增了300条记录,后面整个msck作业还失败了。

看日志的话,在hivemetastore.log中能看到这样的日志:

2023-05-25T14:16:51,699 INFO  [pool-8-thread-7]: HiveMetaStore.audit (HiveMetaStore.java:logAuditEvent(327)) - ugi=hive	ip=10.20.20.80	cmd=source:10.20.20.80 get_config_value: name=metastore.batch.retrieve.max defaultValue=50	
2023-05-25T14:17:04,390 WARN [PartitionDiscoveryTask-4]: utils.RetryUtilities$ExponentiallyDecayingBatchWork (RetryUtilities.java:run(93)) - Exception thrown while processing using a batch size 3000
org.apache.hadoop.hive.metastore.api.MetastoreException: org.apache.thrift.transport.TTransportException: java.net.SocketTimeoutException: Read timed out
at org.apache.hadoop.hive.metastore.Msck$1.execute(Msck.java:393) ~[hive-exec-3.1.2.jar:3.1.2]
at org.apache.hadoop.hive.metastore.Msck$1.execute(Msck.java:358) ~[hive-exec-3.1.2.jar:3.1.2]
at org.apache.hadoop.hive.metastore.utils.RetryUtilities$ExponentiallyDecayingBatchWork.run(RetryUtilities.java:91) [hive-exec-3.1.2.jar:3.1.2]
at org.apache.hadoop.hive.metastore.Msck.createPartitionsInBatches(Msck.java:396) [hive-exec-3.1.2.jar:3.1.2]
at org.apache.hadoop.hive.metastore.Msck.repair(Msck.java:207) [hive-exec-3.1.2.jar:3.1.2]
at org.apache.hadoop.hive.metastore.PartitionManagementTask$MsckThread.run(PartitionManagementTask.java:216) [hive-exec-3.1.2.jar:3.1.2]
at java.util.concurrent.Executors$RunnableAdapter.call(Executors.java:511) [?:1.8.0_352]
at java.util.concurrent.FutureTask.run(FutureTask.java:266) [?:1.8.0_352]
at java.util.concurrent.ThreadPoolExecutor.runWorker(ThreadPoolExecutor.java:1149) [?:1.8.0_352]
at java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:624) [?:1.8.0_352]
at java.lang.Thread.run(Thread.java:750) [?:1.8.0_352]
Caused by: org.apache.thrift.transport.TTransportException: java.net.SocketTimeoutException: Read timed out
at org.apache.thrift.transport.TIOStreamTransport.read(TIOStreamTransport.java:129) ~[hive-exec-3.1.2.jar:3.1.2]
at org.apache.thrift.transport.TTransport.readAll(TTransport.java:86) ~[hive-exec-3.1.2.jar:3.1.2]
at org.apache.thrift.protocol.TBinaryProtocol.readAll(TBinaryProtocol.java:429) ~[hive-exec-3.1.2.jar:3.1.2]
at org.apache.thrift.protocol.TBinaryProtocol.readI32(TBinaryProtocol.java:318) ~[hive-exec-3.1.2.jar:3.1.2]
at org.apache.thrift.protocol.TBinaryProtocol.readMessageBegin(TBinaryProtocol.java:219) ~[hive-exec-3.1.2.jar:3.1.2]
at org.apache.thrift.TServiceClient.receiveBase(TServiceClient.java:77) ~[hive-exec-3.1.2.jar:3.1.2]
at org.apache.hadoop.hive.metastore.api.ThriftHiveMetastore$Client.recv_add_partitions_req(ThriftHiveMetastore.java:2488) ~[hive-exec-3.1.2.jar:3.1.2]
at org.apache.hadoop.hive.metastore.api.ThriftHiveMetastore$Client.add_partitions_req(ThriftHiveMetastore.java:2475) ~[hive-exec-3.1.2.jar:3.1.2]
at org.apache.hadoop.hive.metastore.HiveMetaStoreClient.add_partitions(HiveMetaStoreClient.java:730) ~[hive-exec-3.1.2.jar:3.1.2]
at org.apache.hadoop.hive.metastore.Msck$1.execute(Msck.java:386) ~[hive-exec-3.1.2.jar:3.1.2]
... 10 more
Caused by: java.net.SocketTimeoutException: Read timed out
at java.net.SocketInputStream.socketRead0(Native Method) ~[?:1.8.0_352]
at java.net.SocketInputStream.socketRead(SocketInputStream.java:116) ~[?:1.8.0_352]
at java.net.SocketInputStream.read(SocketInputStream.java:171) ~[?:1.8.0_352]
at java.net.SocketInputStream.read(SocketInputStream.java:141) ~[?:1.8.0_352]
at java.io.BufferedInputStream.fill(BufferedInputStream.java:246) ~[?:1.8.0_352]
at java.io.BufferedInputStream.read1(BufferedInputStream.java:286) ~[?:1.8.0_352]
at java.io.BufferedInputStream.read(BufferedInputStream.java:345) ~[?:1.8.0_352]
at org.apache.thrift.transport.TIOStreamTransport.read(TIOStreamTransport.java:127) ~[hive-exec-3.1.2.jar:3.1.2]
at org.apache.thrift.transport.TTransport.readAll(TTransport.java:86) ~[hive-exec-3.1.2.jar:3.1.2]
at org.apache.thrift.protocol.TBinaryProtocol.readAll(TBinaryProtocol.java:429) ~[hive-exec-3.1.2.jar:3.1.2]
at org.apache.thrift.protocol.TBinaryProtocol.readI32(TBinaryProtocol.java:318) ~[hive-exec-3.1.2.jar:3.1.2]
at org.apache.thrift.protocol.TBinaryProtocol.readMessageBegin(TBinaryProtocol.java:219) ~[hive-exec-3.1.2.jar:3.1.2]
at org.apache.thrift.TServiceClient.receiveBase(TServiceClient.java:77) ~[hive-exec-3.1.2.jar:3.1.2]
at org.apache.hadoop.hive.metastore.api.ThriftHiveMetastore$Client.recv_add_partitions_req(ThriftHiveMetastore.java:2488) ~[hive-exec-3.1.2.jar:3.1.2]
at org.apache.hadoop.hive.metastore.api.ThriftHiveMetastore$Client.add_partitions_req(ThriftHiveMetastore.java:2475) ~[hive-exec-3.1.2.jar:3.1.2]
at org.apache.hadoop.hive.metastore.HiveMetaStoreClient.add_partitions(HiveMetaStoreClient.java:730) ~[hive-exec-3.1.2.jar:3.1.2]
at org.apache.hadoop.hive.metastore.Msck$1.execute(Msck.java:386) ~[hive-exec-3.1.2.jar:3.1.2]
... 10 more
2023-05-25T14:17:06,986 INFO [pool-6-thread-7]: txn.AcidOpenTxnsCounterService (AcidOpenTxnsCounterService.java:run(51)) - AcidOpenTxnsCounterService ran for 0 seconds. isAliveCounter = 2134
2023-05-25T14:17:32,987 INFO [pool-6-thread-2]: txn.TxnHandler (TxnHandler.java:performWriteSetGC(1564)) - Deleted 0 obsolete rows from WRTIE_SET

这个日志让我非常困惑,因为在hiveserver.log中也几乎存在一样的日志:

2023-05-25T15:15:11,805 INFO  [04ecba5d-d30c-49f3-8976-c3bbaed2b240 HiveServer2-Handler-Pool: Thread-132]: conf.HiveConf (HiveConf.java:getLogIdVar(5049)) - Using the default value passed in for log id: 04ecba5d-d30c-49f3-8976-c3bbaed2b240
2023-05-25T15:15:11,805 INFO [04ecba5d-d30c-49f3-8976-c3bbaed2b240 HiveServer2-Handler-Pool: Thread-132]: session.SessionState (:()) - Resetting thread name to HiveServer2-Handler-Pool: Thread-132
2023-05-25T15:15:20,095 WARN [HiveServer2-Background-Pool: Thread-138]: utils.RetryUtilities$ExponentiallyDecayingBatchWork (:()) - Exception thrown while processing using a batch size 750
org.apache.hadoop.hive.metastore.api.MetastoreException: org.apache.thrift.transport.TTransportException: java.net.SocketTimeoutException: Read timed out
at org.apache.hadoop.hive.metastore.Msck$1.execute(Msck.java:393) ~[hive-exec-3.1.2.jar:3.1.2]
at org.apache.hadoop.hive.metastore.Msck$1.execute(Msck.java:358) ~[hive-exec-3.1.2.jar:3.1.2]
at org.apache.hadoop.hive.metastore.utils.RetryUtilities$ExponentiallyDecayingBatchWork.run(RetryUtilities.java:91) ~[hive-exec-3.1.2.jar:3.1.2]
at org.apache.hadoop.hive.metastore.Msck.createPartitionsInBatches(Msck.java:396) ~[hive-exec-3.1.2.jar:3.1.2]
at org.apache.hadoop.hive.metastore.Msck.repair(Msck.java:207) ~[hive-exec-3.1.2.jar:3.1.2]
at org.apache.hadoop.hive.ql.exec.DDLTask.msck(DDLTask.java:2128) ~[hive-exec-3.1.2.jar:3.1.2]
at org.apache.hadoop.hive.ql.exec.DDLTask.execute(DDLTask.java:446) ~[hive-exec-3.1.2.jar:3.1.2]
at org.apache.hadoop.hive.ql.exec.Task.executeTask(Task.java:205) ~[hive-exec-3.1.2.jar:3.1.2]
at org.apache.hadoop.hive.ql.exec.TaskRunner.runSequential(TaskRunner.java:97) ~[hive-exec-3.1.2.jar:3.1.2]
at org.apache.hadoop.hive.ql.Driver.launchTask(Driver.java:2664) ~[hive-exec-3.1.2.jar:3.1.2]
at org.apache.hadoop.hive.ql.Driver.execute(Driver.java:2335) ~[hive-exec-3.1.2.jar:3.1.2]
at org.apache.hadoop.hive.ql.Driver.runInternal(Driver.java:2011) ~[hive-exec-3.1.2.jar:3.1.2]
at org.apache.hadoop.hive.ql.Driver.run(Driver.java:1709) ~[hive-exec-3.1.2.jar:3.1.2]
at org.apache.hadoop.hive.ql.Driver.run(Driver.java:1703) ~[hive-exec-3.1.2.jar:3.1.2]
at org.apache.hadoop.hive.ql.reexec.ReExecDriver.run(ReExecDriver.java:157) ~[hive-exec-3.1.2.jar:3.1.2]
at org.apache.hive.service.cli.operation.SQLOperation.runQuery(SQLOperation.java:224) ~[hive-service-3.1.2.jar:3.1.2]
at org.apache.hive.service.cli.operation.SQLOperation.access$700(SQLOperation.java:87) ~[hive-service-3.1.2.jar:3.1.2]
at org.apache.hive.service.cli.operation.SQLOperation$BackgroundWork$1.run(SQLOperation.java:316) ~[hive-service-3.1.2.jar:3.1.2]
at java.security.AccessController.doPrivileged(Native Method) ~[?:1.8.0_352]
at javax.security.auth.Subject.doAs(Subject.java:422) ~[?:1.8.0_352]
at org.apache.hadoop.security.UserGroupInformation.doAs(UserGroupInformation.java:1878) ~[hadoop-common-3.3.4.jar:?]
at org.apache.hive.service.cli.operation.SQLOperation$BackgroundWork.run(SQLOperation.java:329) ~[hive-service-3.1.2.jar:3.1.2]
at java.util.concurrent.Executors$RunnableAdapter.call(Executors.java:511) ~[?:1.8.0_352]
at java.util.concurrent.FutureTask.run(FutureTask.java:266) ~[?:1.8.0_352]
at java.util.concurrent.ThreadPoolExecutor.runWorker(ThreadPoolExecutor.java:1149) ~[?:1.8.0_352]
at java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:624) ~[?:1.8.0_352]
at java.lang.Thread.run(Thread.java:750) [?:1.8.0_352]
Caused by: org.apache.thrift.transport.TTransportException: java.net.SocketTimeoutException: Read timed out
at org.apache.thrift.transport.TIOStreamTransport.read(TIOStreamTransport.java:129) ~[hive-exec-3.1.2.jar:3.1.2]
at org.apache.thrift.transport.TTransport.readAll(TTransport.java:86) ~[hive-exec-3.1.2.jar:3.1.2]
at org.apache.thrift.protocol.TBinaryProtocol.readAll(TBinaryProtocol.java:429) ~[hive-exec-3.1.2.jar:3.1.2]
at org.apache.thrift.protocol.TBinaryProtocol.readI32(TBinaryProtocol.java:318) ~[hive-exec-3.1.2.jar:3.1.2]
at org.apache.thrift.protocol.TBinaryProtocol.readMessageBegin(TBinaryProtocol.java:219) ~[hive-exec-3.1.2.jar:3.1.2]
at org.apache.thrift.TServiceClient.receiveBase(TServiceClient.java:77) ~[hive-exec-3.1.2.jar:3.1.2]
at org.apache.hadoop.hive.metastore.api.ThriftHiveMetastore$Client.recv_add_partitions_req(ThriftHiveMetastore.java:2488) ~[hive-exec-3.1.2.jar:3.1.2]
at org.apache.hadoop.hive.metastore.api.ThriftHiveMetastore$Client.add_partitions_req(ThriftHiveMetastore.java:2475) ~[hive-exec-3.1.2.jar:3.1.2]
at org.apache.hadoop.hive.metastore.HiveMetaStoreClient.add_partitions(HiveMetaStoreClient.java:730) ~[hive-exec-3.1.2.jar:3.1.2]
at org.apache.hadoop.hive.metastore.Msck$1.execute(Msck.java:386) ~[hive-exec-3.1.2.jar:3.1.2]
... 26 more
Caused by: java.net.SocketTimeoutException: Read timed out
at java.net.SocketInputStream.socketRead0(Native Method) ~[?:1.8.0_352]
at java.net.SocketInputStream.socketRead(SocketInputStream.java:116) ~[?:1.8.0_352]
at java.net.SocketInputStream.read(SocketInputStream.java:171) ~[?:1.8.0_352]
at java.net.SocketInputStream.read(SocketInputStream.java:141) ~[?:1.8.0_352]
at java.io.BufferedInputStream.fill(BufferedInputStream.java:246) ~[?:1.8.0_352]
at java.io.BufferedInputStream.read1(BufferedInputStream.java:286) ~[?:1.8.0_352]
at java.io.BufferedInputStream.read(BufferedInputStream.java:345) ~[?:1.8.0_352]
at org.apache.thrift.transport.TIOStreamTransport.read(TIOStreamTransport.java:127) ~[hive-exec-3.1.2.jar:3.1.2]
at org.apache.thrift.transport.TTransport.readAll(TTransport.java:86) ~[hive-exec-3.1.2.jar:3.1.2]
at org.apache.thrift.protocol.TBinaryProtocol.readAll(TBinaryProtocol.java:429) ~[hive-exec-3.1.2.jar:3.1.2]
at org.apache.thrift.protocol.TBinaryProtocol.readI32(TBinaryProtocol.java:318) ~[hive-exec-3.1.2.jar:3.1.2]
at org.apache.thrift.protocol.TBinaryProtocol.readMessageBegin(TBinaryProtocol.java:219) ~[hive-exec-3.1.2.jar:3.1.2]
at org.apache.thrift.TServiceClient.receiveBase(TServiceClient.java:77) ~[hive-exec-3.1.2.jar:3.1.2]
at org.apache.hadoop.hive.metastore.api.ThriftHiveMetastore$Client.recv_add_partitions_req(ThriftHiveMetastore.java:2488) ~[hive-exec-3.1.2.jar:3.1.2]
at org.apache.hadoop.hive.metastore.api.ThriftHiveMetastore$Client.add_partitions_req(ThriftHiveMetastore.java:2475) ~[hive-exec-3.1.2.jar:3.1.2]
at org.apache.hadoop.hive.metastore.HiveMetaStoreClient.add_partitions(HiveMetaStoreClient.java:730) ~[hive-exec-3.1.2.jar:3.1.2]
at org.apache.hadoop.hive.metastore.Msck$1.execute(Msck.java:386) ~[hive-exec-3.1.2.jar:3.1.2]
... 26 more
2023-05-25T15:15:46,807 INFO [HiveServer2-Handler-Pool: Thread-132]: conf.HiveConf (HiveConf.java:getLogIdVar(5049)) - Using the default value passed in for log id: 04ecba5d-d30c-49f3-8976-c3bbaed2b240

我一度怀疑是hive的日志配置问题,导致日志相互串位了,正常来说hms自身是直接去操作背后的db,不应该出现再去连接hms且发生timeout的情况,于是挨个去检查了一下所有的log,确保日志配置没有问题,再仔细分析了一下堆栈。

确定这个日志确实是hivemetastore输出的,于是只能从代码层面去分析了,从堆栈来看,hms的timeout是从PartitionManagementTask出来的,而msck的命令是通过msck执行的,其次在hiveserver2中确实看到了tiimeout,那么就需要仔细分析清楚,这个timeout到底是怎么来的。

src/main/java/org/apache/hadoop/hive/metastore/PartitionManagementTask.java这个代码中的128行,有这样一个注释:

// TODO: Msck creates MetastoreClient (MSC) on its own. MSC creation is expensive. Sharing MSC also

也就是说PartitionManagementTask会创建出一个MSC对象,这样的话,它确实会去访问HMS,且通过HMS的thrift接口去访问,那么就能理解为什么会有thrift的调用了,接下来继续分析这个Task主要是干什么,继续跟源码,可以看到它的逻辑是这样的:

  • 定义一个candidateTables的list。
  • 遍历所有表,把table的properties中discover.partitions=true的筛选出来,添加到 candidateTables中。
  • 对每一张表执行msck指令。

具体的代码可以看:

for (TableMeta tableMeta : foundTableMetas) {
Table table = msc.getTable(tableMeta.getCatName(), tableMeta.getDbName(), tableMeta.getTableName());
if (table.getParameters() != null && table.getParameters().containsKey(DISCOVER_PARTITIONS_TBLPROPERTY) &&
table.getParameters().get(DISCOVER_PARTITIONS_TBLPROPERTY).equalsIgnoreCase("true")) {
candidateTables.add(table);
}
}

上面是创建List。

for (Table table : candidateTables) {
qualifiedTableName = Warehouse.getCatalogQualifiedTableName(table);
long retentionSeconds = getRetentionPeriodInSeconds(table);
LOG.info("Running partition discovery for table {} retentionPeriod: {}s", qualifiedTableName,
retentionSeconds);
// this always runs in 'sync' mode where partitions can be added and dropped
MsckInfo msckInfo = new MsckInfo(table.getCatName(), table.getDbName(), table.getTableName(),
null, null, true, true, true, retentionSeconds);
executorService.submit(new MsckThread(msckInfo, conf, qualifiedTableName, countDownLatch));
}

上面是执行MSCK,这样的话,可以确认在HMS里面,如果配置了discover.partitions=true,那么这个Task会自动调用HMS的thrift接口执行msck,接下来另外一些困惑又来了,这个key是哪里配置的,这个task是如何启动的?

继续分析代码,可以看到在src/java/org/apache/hadoop/hive/ql/plan/CreateTableDesc.java中:

 if (isExternal()) {
tbl.setProperty("EXTERNAL", "TRUE");
tbl.setTableType(TableType.EXTERNAL_TABLE);
// only add if user have not explicit set it (user explicitly disabled for example in which case don't flip it)
if (tbl.getProperty(PartitionManagementTask.DISCOVER_PARTITIONS_TBLPROPERTY) == null) {
// partition discovery is on by default if undefined
tbl.setProperty(PartitionManagementTask.DISCOVER_PARTITIONS_TBLPROPERTY, "true");
}
}

也就是说如果你创建的是外表,那么在create table的时候,一定会自动添加discover.partitions=true这个配置。

src/main/java/org/apache/hadoop/hive/metastore/HiveMetaStore.java中的startMetaStoreThreads中调用了startRemoteOnlyTasks通过runFrequency方法会走到PartitionManagementTask来。

这个频率是:

// Partition management task params
PARTITION_MANAGEMENT_TASK_FREQUENCY("metastore.partition.management.task.frequency",
"metastore.partition.management.task.frequency",
300, TimeUnit.SECONDS, "Frequency at which timer task runs to do automatic partition management for tables\n" +
"with table property 'discover.partitions'='true'. Partition management include 2 pieces. One is partition\n" +
"discovery and other is partition retention period. When 'discover.partitions'='true' is set, partition\n" +
"management will look for partitions in table location and add partitions objects for it in metastore.\n" +
"Similarly if partition object exists in metastore and partition location does not exist, partition object\n" +
"will be dropped. The second piece in partition management is retention period. When 'discover.partition'\n" +
"is set to true and if 'partition.retention.period' table property is defined, partitions that are older\n" +
"than the specified retention period will be automatically dropped from metastore along with the data."),

默认是300s也就是5分钟,总结一下就是如果创建的表是外表,那么hms会自动启动一个周期作业,定期自动的执行msck指令,把远端的分区加载过来,这个思路理清楚了后,就需要继续深究另一个问题,那就是为什么出现了timeout?

继续看src/main/java/org/apache/hadoop/hive/metastore/Msck.java的代码,可以看到执行msck的时候是需要申请lock的,由2个控制acquireLocklockRequired,acquireLock是参数传递过来,默认是ture,lockRequired的计算逻辑是:

boolean lockRequired = totalPartsToFix > 0 &&
msckInfo.isRepairPartitions() &&
(msckInfo.isAddPartitions() || msckInfo.isDropPartitions());

当上面的条件都符合的时候,然后会去申请lock:

if (acquireLock && lockRequired && table.getParameters() != null &&
MetaStoreServerUtils.isTransactionalTable(table.getParameters())) {
// Running MSCK from beeline/cli will make DDL task acquire X lock when repair is enabled, since we are directly
// invoking msck.repair() without SQL statement, we need to do the same and acquire X lock (repair is default)
LockRequest lockRequest = createLockRequest(msckInfo.getDbName(), msckInfo.getTableName());
txnId = lockRequest.getTxnid();
try {
LockResponse res = getMsc().lock(lockRequest);
if (res.getState() != LockState.ACQUIRED) {
throw new MetastoreException("Unable to acquire lock(X) on " + qualifiedTableName);
}
lockId = res.getLockid();
} catch (TException e) {
throw new MetastoreException("Unable to acquire lock(X) on " + qualifiedTableName, e);
}
LOG.info("Acquired lock(X) on {}. LockId: {}", qualifiedTableName, lockId);
}

具体的locK实现在:

private LockRequest createLockRequest(final String dbName, final String tableName) throws TException {
UserGroupInformation loggedInUser = null;
String username;
try {
loggedInUser = UserGroupInformation.getLoginUser();
} catch (IOException e) {
LOG.warn("Unable to get logged in user via UGI. err: {}", e.getMessage());
}
if (loggedInUser == null) {
username = System.getProperty("user.name");
} else {
username = loggedInUser.getShortUserName();
}
long txnId = getMsc().openTxn(username);
String agentInfo = Thread.currentThread().getName();
LockRequestBuilder requestBuilder = new LockRequestBuilder(agentInfo);
requestBuilder.setUser(username);
requestBuilder.setTransactionId(txnId);

LockComponentBuilder lockCompBuilder = new LockComponentBuilder()
.setDbName(dbName)
.setTableName(tableName)
.setIsTransactional(true)
.setExclusive()
// WriteType is DDL_EXCLUSIVE for MSCK REPAIR so we need NO_TXN. Refer AcidUtils.makeLockComponents
.setOperationType(DataOperationType.NO_TXN);
requestBuilder.addLockComponent(lockCompBuilder.build());

LOG.info("Created lock(X) request with info - user: {} txnId: {} agentInfo: {} dbName: {} tableName: {}",
username, txnId, agentInfo, dbName, tableName);
return requestBuilder.build();
}

也就是说,这个timeout很大可能就是出现在long txnId = getMsc().openTxn(username); 这里,自此,整个链路就完全清晰了:

  • 当创建外表的时候会自动添加discovery属性,hms会定期启动一个load partition task。
  • 用户通过beeline执行msck命令,由于 DbNotificationListener 问题,速度非常慢,具体可以查:https://baifachuan.com/posts/52be3ba3.html
  • 自此HMS的task到达周期后,进行lock申请,由于beeline已经申请到lock,导致hms这边获取lock失败,出现timeout。
  • 而hms拿到lock后,执行依旧很慢,从而再导致hs2这边获取lock 失败,出现timeout,出现相互影响。

所以我们在使用msck的时候,需要总体看下,是否有必要。


扫码手机观看或分享: