hive-issues mailing list archives

Site index · List index
Message view « Date » · « Thread »
Top « Date » · « Thread »
From "Eugene Koifman (JIRA)" <j...@apache.org>
Subject [jira] [Commented] (HIVE-11388) there should only be 1 Initiator for compactions per Hive installation
Date Wed, 16 Dec 2015 22:18:46 GMT

    [ https://issues.apache.org/jira/browse/HIVE-11388?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=15061006#comment-15061006
] 

Eugene Koifman commented on HIVE-11388:
---------------------------------------

here is one general purpose mechanism:
create table MUTEX_TABLE(keyname varchar(512) PRIMARY KEY)

Then any process that requires a mutex needs to insert a row into this table (as long as everyone
agrees on the key) and then do a "Select for update" on this row.  If the process dies, "select
for update" lock is automatically released.

For example, if 2 Initiator instances want to schedule a compaction, each could
1. select * from MUTEX_TABLE where keyname="initiator" for update.
If the "initiator" row is already there, only 1 will succeed.  The other one, once it unblocks,
will already see "this" compaction scheduled.
2. if select in 1 misses, then Initiator can insert "initiator" row and then goto 1.  Because
of PK only 1 will succeed.

Since the keyname is arbitrary, it can be "db/table/partition" to coordinate Workers if necessary.

A little primitive, but workable and avoids ZooKeeper and allows all parts of Compaction/HouseKeeping
to run on multiple MS nodes.

> there should only be 1 Initiator for compactions per Hive installation
> ----------------------------------------------------------------------
>
>                 Key: HIVE-11388
>                 URL: https://issues.apache.org/jira/browse/HIVE-11388
>             Project: Hive
>          Issue Type: Bug
>          Components: Transactions
>    Affects Versions: 1.0.0
>            Reporter: Eugene Koifman
>            Assignee: Eugene Koifman
>            Priority: Critical
>
> org.apache.hadoop.hive.ql.txn.compactor.Initiator is a thread that runs inside the metastore
service to manage compactions of ACID tables.  There should be exactly 1 instance of this
thread (even with multiple Thrift services).
> This is documented in https://cwiki.apache.org/confluence/display/Hive/Hive+Transactions#HiveTransactions-Configuration
but not enforced.
> Should add enforcement, since more than 1 Initiator could cause concurrent attempts to
compact the same table/partition - which will not work.



--
This message was sent by Atlassian JIRA
(v6.3.4#6332)

Mime
View raw message