You are viewing a plain text version of this content. The canonical link for it is here.
Posted to issues@hive.apache.org by "anishek (JIRA)" <ji...@apache.org> on 2018/02/01 10:31:00 UTC

[jira] [Commented] (HIVE-16886) HMS log notifications may have duplicated event IDs if multiple HMS are running concurrently

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

anishek commented on HIVE-16886:
--------------------------------

[~akolb] i would rather not have difference between the two branches, if you could make sure that the branch 3 is also updated with the fix very soon, in near future that would be great, assuming you are doing branch-2 fix first. 


> 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
>    Affects Versions: 3.0.0, 2.3.2, 2.3.3
>            Reporter: Sergio Peña
>            Assignee: anishek
>            Priority: Major
>              Labels: TODOC3.0
>             Fix For: 3.0.0
>
>         Attachments: HIVE-16886.1.patch, HIVE-16886.2.patch, HIVE-16886.3.patch, HIVE-16886.4.patch, HIVE-16886.5.patch, HIVE-16886.6.patch, HIVE-16886.7.patch, HIVE-16886.8.patch, datastore-identity-holes.diff
>
>
> 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
(v7.6.3#76005)