[ https://issues.apache.org/jira/browse/HIVE-16886?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=16136029#comment-16136029 ]
Alexander Kolbasov commented on HIVE-16886: ------------------------------------------- >From what I saw all mechanisms supported by DataNucleus can not guarantee that >there are no holes (although tehy deal with duplicates). There may be two >kinds of holes - temporary ones (that will be filled later when transaction is >committed) and permanent ones (which will be never filled in). MySQL InnoDB has a mechanism to provide no-holes semantics (https://dev.mysql.com/doc/refman/5.7/en/innodb-auto-increment-handling.html) but ir seems that Oracle and PostgreSQL don't have similar mechanisms. Holes create troubles for consumers. E.g. when consumer reads notification ID 10 some earlier IDs may be committed later so they may be skipped or not processed properly. Also, consumers have no way of knowing whether the hole will be filled in later (when corresponding transaction commits) or never (if the transaction that allocated ID fails). There is a way to guarantee that there re no holes and duplicates: a) Make the ID primary key (which doesn't allow duplicates) b) As part of the transaction, read the value of ID, increment it by 1 and persist it. This approach guarantees that there are no holes or duplicates, but transactions can fail because of the ID conflicts, so it is important to retry such transactions. HMS doesn't provide per-transaction retries, but may be per-operation retries are Ok as well. > 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: 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)