You are viewing a plain text version of this content. The canonical link for it is here.
Posted to dev@hive.apache.org by "Sergio Peña (JIRA)" <ji...@apache.org> on 2017/06/12 21:01:00 UTC
[jira] [Created] (HIVE-16886) HMS log notifications may have
duplicated event IDs if multiple HMS are running concurrently
Sergio Peña created HIVE-16886:
----------------------------------
Summary: 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
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)