Repository: hive
Updated Branches:
  refs/heads/branch-2.3 a4e3e3592 -> 62d9b1981


HIVE-15761: ObjectStore.getNextNotification could return an empty 
NotificationEventResponse causing TProtocolException (Sergio Pena, reviewed by 
Aihua Xu)


Project: http://git-wip-us.apache.org/repos/asf/hive/repo
Commit: http://git-wip-us.apache.org/repos/asf/hive/commit/145ed20b
Tree: http://git-wip-us.apache.org/repos/asf/hive/tree/145ed20b
Diff: http://git-wip-us.apache.org/repos/asf/hive/diff/145ed20b

Branch: refs/heads/branch-2.3
Commit: 145ed20b04e4bf6bcb833e94efe71bcbe0851500
Parents: ecf9704
Author: Sergio Pena <sergio.p...@cloudera.com>
Authored: Fri Apr 21 15:23:52 2017 -0500
Committer: Sahil Takiar <stak...@cloudera.com>
Committed: Tue Nov 7 08:15:47 2017 -0800

----------------------------------------------------------------------
 .../java/org/apache/hadoop/hive/metastore/ObjectStore.java    | 7 ++++---
 .../org/apache/hadoop/hive/metastore/TestObjectStore.java     | 5 +++++
 2 files changed, 9 insertions(+), 3 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hive/blob/145ed20b/metastore/src/java/org/apache/hadoop/hive/metastore/ObjectStore.java
----------------------------------------------------------------------
diff --git 
a/metastore/src/java/org/apache/hadoop/hive/metastore/ObjectStore.java 
b/metastore/src/java/org/apache/hadoop/hive/metastore/ObjectStore.java
index a63519a..358cf17 100644
--- a/metastore/src/java/org/apache/hadoop/hive/metastore/ObjectStore.java
+++ b/metastore/src/java/org/apache/hadoop/hive/metastore/ObjectStore.java
@@ -8260,6 +8260,9 @@ public class ObjectStore implements RawStore, 
Configurable {
   public NotificationEventResponse 
getNextNotification(NotificationEventRequest rqst) {
     boolean commited = false;
     Query query = null;
+
+    NotificationEventResponse result = new NotificationEventResponse();
+    result.setEvents(new ArrayList<NotificationEvent>());
     try {
       openTransaction();
       long lastEvent = rqst.getLastEvent();
@@ -8269,11 +8272,9 @@ public class ObjectStore implements RawStore, 
Configurable {
       Collection<MNotificationLog> events = (Collection) 
query.execute(lastEvent);
       commited = commitTransaction();
       if (events == null) {
-        return null;
+        return result;
       }
       Iterator<MNotificationLog> i = events.iterator();
-      NotificationEventResponse result = new NotificationEventResponse();
-      result.setEvents(new ArrayList<NotificationEvent>());
       int maxEvents = rqst.getMaxEvents() > 0 ? rqst.getMaxEvents() : 
Integer.MAX_VALUE;
       int numEvents = 0;
       while (i.hasNext() && numEvents++ < maxEvents) {

http://git-wip-us.apache.org/repos/asf/hive/blob/145ed20b/metastore/src/test/org/apache/hadoop/hive/metastore/TestObjectStore.java
----------------------------------------------------------------------
diff --git 
a/metastore/src/test/org/apache/hadoop/hive/metastore/TestObjectStore.java 
b/metastore/src/test/org/apache/hadoop/hive/metastore/TestObjectStore.java
index 9b8eaf2..6524ee7 100644
--- a/metastore/src/test/org/apache/hadoop/hive/metastore/TestObjectStore.java
+++ b/metastore/src/test/org/apache/hadoop/hive/metastore/TestObjectStore.java
@@ -153,11 +153,16 @@ public class TestObjectStore {
     Assert.assertEquals(2, eventResponse.getEventsSize());
     Assert.assertEquals(FIRST_EVENT_ID, 
eventResponse.getEvents().get(0).getEventId());
     Assert.assertEquals(SECOND_EVENT_ID, 
eventResponse.getEvents().get(1).getEventId());
+
     // Verify that getNextNotification(last) returns events after a specified 
event
     eventResponse = objectStore.getNextNotification(new 
NotificationEventRequest(FIRST_EVENT_ID));
     Assert.assertEquals(1, eventResponse.getEventsSize());
     Assert.assertEquals(SECOND_EVENT_ID, 
eventResponse.getEvents().get(0).getEventId());
 
+    // Verify that getNextNotification(last) returns zero events if there are 
no more notifications available
+    eventResponse = objectStore.getNextNotification(new 
NotificationEventRequest(SECOND_EVENT_ID));
+    Assert.assertEquals(0, eventResponse.getEventsSize());
+
     // Verify that cleanNotificationEvents() cleans up all old notifications
     Thread.sleep(1);
     objectStore.cleanNotificationEvents(1);

Reply via email to