2626import org .apache .hadoop .hive .metastore .conf .MetastoreConf .ConfVars ;
2727import org .apache .hadoop .hive .metastore .leader .LeaderElection ;
2828import org .apache .hadoop .hive .metastore .leader .LeaderElectionContext ;
29+ import org .apache .hadoop .hive .metastore .leader .LeaderElectionFactory ;
2930import org .apache .hadoop .hive .metastore .leader .LeaseLeaderElection ;
3031import org .apache .hadoop .hive .metastore .security .HadoopThriftAuthBridge ;
3132import org .apache .hadoop .hive .ql .stats .StatsUpdaterThread ;
4344import java .util .Map ;
4445import java .util .Set ;
4546import java .util .concurrent .CountDownLatch ;
47+ import java .util .concurrent .ScheduledExecutorService ;
4648import java .util .concurrent .TimeUnit ;
4749
4850/**
@@ -68,7 +70,7 @@ void setup(final String leaderHostName, Configuration configuration) throws Exce
6870 MetaStoreTestUtils .setConfForStandloneMode (conf );
6971 MetastoreConf .setVar (conf , ConfVars .THRIFT_BIND_HOST , "localhost" );
7072 MetastoreConf .setVar (conf , ConfVars .METASTORE_HOUSEKEEPING_LEADER_ELECTION ,
71- leaderHostName != null ? "host" : "lock" );
73+ leaderHostName != null ? LeaderElectionFactory . Method . HOST . name () : LeaderElectionFactory . Method . LOCK . name () );
7274 if (leaderHostName != null ) {
7375 MetastoreConf .setVar (conf , ConfVars .METASTORE_HOUSEKEEPING_LEADER_HOSTNAME , leaderHostName );
7476 }
@@ -286,6 +288,13 @@ public void setName(String name) {
286288
287289 @ Override
288290 protected void notifyListener () {
291+ ScheduledExecutorService service = null ;
292+ if (!isLeader ) {
293+ try {
294+ service = ThreadPool .getPool ();
295+ } catch (Exception ignored ) {
296+ }
297+ }
289298 super .notifyListener ();
290299 if (isLeader ) {
291300 if (!needRenewLease ) {
@@ -298,11 +307,17 @@ protected void notifyListener() {
298307 heartbeater .startWatch ();
299308 }
300309 } else {
301- try {
302- // This is the last one get notified, sleep some time to make sure all other
303- // services have been stopped before return
304- Thread .sleep (12000 );
305- } catch (InterruptedException ignore ) {
310+ if (service != null ) {
311+ // If the housekeeping task is running behind
312+ Assert .assertTrue (service .isShutdown ());
313+ // Interrupt all sleeping tasks
314+ service .shutdownNow ();
315+ try {
316+ // This is the last one get notified, sleep some time to make sure all other
317+ // services have been stopped before return
318+ Thread .sleep (12000 );
319+ } catch (InterruptedException ignore ) {
320+ }
306321 }
307322 }
308323 if (latch != null ) {
0 commit comments