[
https://issues.apache.org/jira/browse/HIVE-16886?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=16135433#comment-16135433
]
anishek commented on HIVE-16886:
--------------------------------
I am just going to provide some more detail information so that we all get to
understand what is happening with this bug.
As of now the plan is to hopefully have a backward compatibility with
replication v1, though primary focus is going to be fixing the issue of
duplicate event ids with multiple HMS, which is specifically detrimental for
replication v2, which is now going to replication point-in-time state of the
database rather than the latest state as was in repl v1.
As for the fix, there are few tests i am going to fix and have a patch for all
of you, hopefully today/tomorrow, to review.
* mapping for {{NOTIFICATION_SEQUENCE}} has been removed.
* there are effectively two class mappings for {{NOTIFICATION_LOG}} , one
representing the current state namely using {{EVENT_ID}} as event id and new
implementation that will use {{NL_ID}}, with the new mapping putting a default
value of 0 for {{EVENT_ID}}.
* Backward compatibility in terms of replication v1 not being broken, should be
possible if the following assumption holds in the metastore rdmbs : ??value of
NL_ID is greater than the EVENT_ID for same rows, including only events that
are not yet replicated to replica warehouse.??
* Code to switch mappings in the metastore from {{EVENT_ID}} to {{ND_ID}} is in
place, depending on what is required, For ex for existing repl v1, it will
first provide all the events using the {{EVENT_ID}} and post that start
providing events with {{NL_ID}} since {{NL_ID > EVENT_ID}} it would allow
existing setup to continue working.
> 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)