Uploaded image for project: 'Hive'
  1. Hive
  2. HIVE-16738

Notification ID generation in DBNotification might not be unique across HS2 instances.

    XMLWordPrintableJSON

Details

    • Bug
    • Status: Closed
    • Major
    • Resolution: Duplicate
    • 3.0.0
    • 3.0.0
    • HiveServer2
    • None

    Description

      Going to explain the problem in scope of "replication" feature for hive 2 that is being built, as it is easier to explain:

      To allow replication to work we need to set "hive.metastore.transactional.event.listeners" to DBNotificationListener. For use cases where there are multiple HiveServer2 Instances running

       private void process(NotificationEvent event, ListenerEvent listenerEvent) throws MetaException {
          event.setMessageFormat(msgFactory.getMessageFormat());
          synchronized (NOTIFICATION_TBL_LOCK) {
            LOG.debug("DbNotificationListener: Processing : {}:{}", event.getEventId(),
                event.getMessage());
            HMSHandler.getMSForConf(hiveConf).addNotificationEvent(event);
          }
      
            // Set the DB_NOTIFICATION_EVENT_ID for future reference by other listeners.
            if (event.isSetEventId()) {
              listenerEvent.putParameter(
                  MetaStoreEventListenerConstants.DB_NOTIFICATION_EVENT_ID_KEY_NAME,
                  Long.toString(event.getEventId()));
            }
        }
      

      the above code in DBNotificationListner having the object lock wont be guarantee enough to make sure that all events get a unique id. The transaction isolation level at the db "read-comitted" or "repeatable-read" would also not guarantee the same, unless a lock is at the db level preferably on table NOTIFICATION_SEQUENCE which only has one row.

      Attachments

        Issue Links

          Activity

            People

              anishek Anishek Agarwal
              anishek Anishek Agarwal
              Votes:
              0 Vote for this issue
              Watchers:
              6 Start watching this issue

              Dates

                Created:
                Updated:
                Resolved: