[ 
https://issues.apache.org/jira/browse/HIVE-26242?focusedWorklogId=785083&page=com.atlassian.jira.plugin.system.issuetabpanels:worklog-tabpanel#worklog-785083
 ]

ASF GitHub Bot logged work on HIVE-26242:
-----------------------------------------

                Author: ASF GitHub Bot
            Created on: 27/Jun/22 13:12
            Start Date: 27/Jun/22 13:12
    Worklog Time Spent: 10m 
      Work Description: deniskuzZ commented on code in PR #3303:
URL: https://github.com/apache/hive/pull/3303#discussion_r907373970


##########
ql/src/java/org/apache/hadoop/hive/ql/txn/compactor/Worker.java:
##########
@@ -692,18 +657,7 @@ void open(CompactionInfo ci) throws TException {
             + "}, status {" + res.getState() + "}, reason {" + 
res.getErrorMessage() + "}");
       }
       lockId = res.getLockid();
-      heartbeatExecutor = Executors.newSingleThreadScheduledExecutor(new 
ThreadFactoryBuilder()
-                .setPriority(Thread.MIN_PRIORITY)
-                .setDaemon(true)
-                .setNameFormat("CompactionTxnHeartbeater-" + txnId)
-                .build());
-      long txnTimeout = MetastoreConf.getTimeVar(conf, 
MetastoreConf.ConfVars.TXN_TIMEOUT, TimeUnit.MILLISECONDS);
-      heartbeatExecutor.scheduleAtFixedRate(
-              new CompactionHeartbeater(this, 
TxnUtils.getFullTableName(ci.dbname, ci.tableName), conf),
-              txnTimeout / 4,
-              txnTimeout / 2,
-              TimeUnit.MILLISECONDS
-      );
+      CompactionHeartbeatService.getInstance(conf).startHeartbeat(txnId, 
lockId, ci.tableName);

Review Comment:
   should we pass `TxnUtils.getFullTableName(ci.dbname, ci.tableName)`



##########
ql/src/java/org/apache/hadoop/hive/ql/txn/compactor/Worker.java:
##########
@@ -692,18 +657,7 @@ void open(CompactionInfo ci) throws TException {
             + "}, status {" + res.getState() + "}, reason {" + 
res.getErrorMessage() + "}");
       }
       lockId = res.getLockid();
-      heartbeatExecutor = Executors.newSingleThreadScheduledExecutor(new 
ThreadFactoryBuilder()
-                .setPriority(Thread.MIN_PRIORITY)
-                .setDaemon(true)
-                .setNameFormat("CompactionTxnHeartbeater-" + txnId)
-                .build());
-      long txnTimeout = MetastoreConf.getTimeVar(conf, 
MetastoreConf.ConfVars.TXN_TIMEOUT, TimeUnit.MILLISECONDS);
-      heartbeatExecutor.scheduleAtFixedRate(
-              new CompactionHeartbeater(this, 
TxnUtils.getFullTableName(ci.dbname, ci.tableName), conf),
-              txnTimeout / 4,
-              txnTimeout / 2,
-              TimeUnit.MILLISECONDS
-      );
+      CompactionHeartbeatService.getInstance(conf).startHeartbeat(txnId, 
lockId, ci.tableName);

Review Comment:
   should we pass `TxnUtils.getFullTableName(ci.dbname, ci.tableName)`?





Issue Time Tracking
-------------------

    Worklog Id:     (was: 785083)
    Time Spent: 5h 10m  (was: 5h)

> Compaction heartbeater improvements
> -----------------------------------
>
>                 Key: HIVE-26242
>                 URL: https://issues.apache.org/jira/browse/HIVE-26242
>             Project: Hive
>          Issue Type: Improvement
>            Reporter: László Végh
>            Assignee: László Végh
>            Priority: Major
>              Labels: pull-request-available
>          Time Spent: 5h 10m
>  Remaining Estimate: 0h
>
> The Compaction heartbeater should be improved the following ways:
>  * The metastore clients should be reused between heartbeats and closed only 
> at the end, when the transaction ends
>  * Instead of having a dedicated heartbeater thread for each Compaction 
> transaction, there should be shared a heartbeater executor where the 
> heartbeat tasks can be scheduled/submitted.



--
This message was sent by Atlassian Jira
(v8.20.7#820007)

Reply via email to