[
https://issues.apache.org/jira/browse/HIVE-16886?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel
]
Sergio Peña updated HIVE-16886:
-------------------------------
Attachment: datastore-identity-holes.diff
[~anishek] I attached a patch that probes we can have holes when using
datastore identity IF delays happen during the commit() transaction call. I
made some modifications on the ObjectStore to allow seeing this behavior:
getNextNotification() reads the NL_ID and replaces EVENT_ID with NL_ID so that
I can read what the actual NL_ID was persisted.
addNotificationEvent() has some count down latches that signal the persist and
commit order.
The problem we saw is that NL_ID is created on the {{makePersistent}} call (sql
execution), but it is not seen by getNextNotification() until is committed.
This commit() may be delayed due to issues on network, GC or anything else. We
run into this problem in Sentry when we were running several stress tests on a
cluster with Sentry HA.
Apply the patch on master, and run the test case:
{noformat}
$ mvn test -Dtest=TestObjectStore#testDatanucleusHoles
{noformat}
The test case have assertions to probe the hole, but to look at the actual
holes, you can take a look at the log:
{noformat}
$ cat
metastore/target/surefire-reports/org.apache.hadoop.hive.metastore.TestObjectStore-output.txt
2017-08-22T14:36:06,090 INFO [main] metastore.TestObjectStore: 1st
notification request will get only one event with ID = 2
2017-08-22T14:36:06,090 INFO [main] metastore.TestObjectStore: ID = 2, TYPE =
null
2017-08-22T14:36:06,092 INFO [main] metastore.TestObjectStore: 2nd
notification request should get last event ID = 2 and a new event with ID = 1
2017-08-22T14:36:06,092 INFO [main] metastore.TestObjectStore: ID = 1, TYPE =
null
2017-08-22T14:36:06,092 INFO [main] metastore.TestObjectStore: ID = 2, TYPE =
null
{noformat}
> 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
> Attachments: datastore-identity-holes.diff, HIVE-16886.1.patch
>
>
> 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)