Details
-
Bug
-
Status: Closed
-
Major
-
Resolution: Fixed
-
2.3.2, 2.3.3, 3.0.0
-
Reviewed
Description
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:
@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()); }
The last assertion fails expecting an event ID 1 instead of 2.
Attachments
Attachments
Issue Links
- Blocked
-
SENTRY-1885 Remove unused NOTIFICATION_ID from the SENTRY_PATH_CHANGE table
- Open
- duplicates
-
HIVE-16738 Notification ID generation in DBNotification might not be unique across HS2 instances.
- Closed
- is related to
-
SENTRY-1895 Sentry should handle the case of multiple notifications with the same ID
- Resolved
-
HIVE-18940 Hive notifications serialize all write DDL operations
- Open
-
HIVE-18768 Use Datanucleus to serialize notification updates
- Resolved
-
HIVE-16738 Notification ID generation in DBNotification might not be unique across HS2 instances.
- Closed
-
HIVE-17735 ObjectStore.addNotificationEvent is leaking queries
- Closed
-
HIVE-17758 NOTIFICATION_SEQUENCE_LOCK_RETRY_SLEEP_INTERVAL.defaultLongVal is -1
- Closed
-
SENTRY-2106 If Sentry is ahead do not trigger a full snapshot
- Resolved
- relates to
-
SENTRY-1855 Improve scalability of permission delta updates
- Open
-
SENTRY-2109 Fix the logic of identifying HMS out of Sync and handle gaps and out-of-sequence notifications.
- Patch Available
-
HIVE-18300 Use LAST_INSERT_ID for Sequence Tables for MySQL
- Open
- links to
1.
|
Backport HIVE-16886 to Hive 2 | Resolved | Alex Kolbasov |