[
https://issues.apache.org/jira/browse/HIVE-16886?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=16128384#comment-16128384
]
anishek commented on HIVE-16886:
--------------------------------
Yeh we can do that though we have to explicitly parse and typecast the
data-store identity in metastore code. Additionally sql query from datastore
has to be used for
{code}public NotificationEventResponse
getNextNotification(NotificationEventRequest rqst){code} in object store.
I have the code in place which will address the current issue with the use of
{{NL_ID}} as event id and remove the use of
* {{MNotificationNextId}}
* {{EVENT_ID}} from {{MNotificationLog}} such that without modifying the
metastore db schema, we just populate a default of value "0" for this column in
db.
though the problem is how do we manage deployments who are using repl v1 who
are dependent on {{EVENT_ID}} and with the new release suddenly will move to
{{NL_ID}}
* one way is we map both {{NL_ID}} and {{EVENT_ID}} in {{MNotificationLog}}
and the external tool based on the value of {{EVENT_ID=0}} switches to using
id's from {{NL_ID}}
* other way is to completely redo the whole replication deployment with repl v2
rather than repl v1.
> HMS log notifications may have duplicated event IDs if multiple HMS are
> running concurrently
> --------------------------------------------------------------------------------------------
>
> Key: HIVE-16886
> URL: https://issues.apache.org/jira/browse/HIVE-16886
> Project: Hive
> Issue Type: Bug
> Components: Hive, Metastore
> Reporter: Sergio Peña
> Assignee: anishek
>
> When running multiple Hive Metastore servers and DB notifications are
> enabled, I could see that notifications can be persisted with a duplicated
> event ID.
> This does not happen when running multiple threads in a single HMS node due
> to the locking acquired on the DbNotificationsLog class, but multiple HMS
> could cause conflicts.
> The issue is in the ObjectStore#addNotificationEvent() method. The event ID
> fetched from the datastore is used for the new notification, incremented in
> the server itself, then persisted or updated back to the datastore. If 2
> servers read the same ID, then these 2 servers write a new notification with
> the same ID.
> The event ID is not unique nor a primary key.
> Here's a test case using the TestObjectStore class that confirms this issue:
> {noformat}
> @Test
> public void testConcurrentAddNotifications() throws ExecutionException,
> InterruptedException {
> final int NUM_THREADS = 2;
> CountDownLatch countIn = new CountDownLatch(NUM_THREADS);
> CountDownLatch countOut = new CountDownLatch(1);
> HiveConf conf = new HiveConf();
> conf.setVar(HiveConf.ConfVars.METASTORE_EXPRESSION_PROXY_CLASS,
> MockPartitionExpressionProxy.class.getName());
> ExecutorService executorService =
> Executors.newFixedThreadPool(NUM_THREADS);
> FutureTask<Void> tasks[] = new FutureTask[NUM_THREADS];
> for (int i=0; i<NUM_THREADS; i++) {
> final int n = i;
> tasks[i] = new FutureTask<Void>(new Callable<Void>() {
> @Override
> public Void call() throws Exception {
> ObjectStore store = new ObjectStore();
> store.setConf(conf);
> NotificationEvent dbEvent =
> new NotificationEvent(0, 0,
> EventMessage.EventType.CREATE_DATABASE.toString(), "CREATE DATABASE DB" + n);
> System.out.println("ADDING NOTIFICATION");
> countIn.countDown();
> countOut.await();
> store.addNotificationEvent(dbEvent);
> System.out.println("FINISH NOTIFICATION");
> return null;
> }
> });
> executorService.execute(tasks[i]);
> }
> countIn.await();
> countOut.countDown();
> for (int i = 0; i < NUM_THREADS; ++i) {
> tasks[i].get();
> }
> NotificationEventResponse eventResponse =
> objectStore.getNextNotification(new NotificationEventRequest());
> Assert.assertEquals(2, eventResponse.getEventsSize());
> Assert.assertEquals(1, eventResponse.getEvents().get(0).getEventId());
> // This fails because the next notification has an event ID = 1
> Assert.assertEquals(2, eventResponse.getEvents().get(1).getEventId());
> }
> {noformat}
> The last assertion fails expecting an event ID 1 instead of 2.
--
This message was sent by Atlassian JIRA
(v6.4.14#64029)