This is an automated email from the ASF dual-hosted git repository.

wzhou pushed a commit to branch master
in repository https://gitbox.apache.org/repos/asf/impala.git

commit 68f8a6a1df0d2da91baa87b8b6699ddbc495b88e
Author: Sai Hemanth Gantasala <[email protected]>
AuthorDate: Thu Feb 1 12:34:00 2024 -0800

    IMPALA-12607: Bump the GBN and fetch events specific to the db/table
    from the metastore
    
    Bump the GBN to 49623641 to leverage HIVE-27499, so that Impala can
    directly fetch the latest events specific to the db/table from the
    metastore, instead of fetching the events from metastore and then
    filtering in the cache matching the DbName/TableName.
    
    Implementation Details:
    Currently when a DDL/DML is performed in Impala, we fetch all the
    events from metastore based on current eventId and then filter them in
    Impala which can be a bottleneck if the events count is huge. This can
    be optimized by including db name and/or table name in the notification
    event request object and then filter by event type in impala. This can
    provide performance boost on tables that generate a lot of events.
    
    Note:
    Also included ShowUtils class in hive-minimal-exec jar as it is
    required in the current build version
    
    Testing:
    1) Did some tests in local cluster
    2) Added a test case in MetaStoreEventsProcessorTest
    
    Change-Id: I6aecd5108b31c24e6e2c6f9fba6d4d44a3b00729
    Reviewed-on: http://gerrit.cloudera.org:8080/20979
    Reviewed-by: Impala Public Jenkins <[email protected]>
    Tested-by: Impala Public Jenkins <[email protected]>
---
 bin/impala-config.sh                               |  22 ++--
 .../org/apache/impala/compat/MetastoreShim.java    |   6 ++
 .../org/apache/impala/compat/MetastoreShim.java    |  20 ++++
 .../catalog/events/MetastoreEventsProcessor.java   | 118 +++++++++++++++++----
 .../events/MetastoreEventsProcessorTest.java       |  76 +++++++++++++
 .../CatalogHmsSyncToLatestEventIdTest.java         |   2 +-
 java/shaded-deps/hive-exec/pom.xml                 |   2 +
 7 files changed, 213 insertions(+), 33 deletions(-)

diff --git a/bin/impala-config.sh b/bin/impala-config.sh
index af391a3d7..4bcb364d3 100755
--- a/bin/impala-config.sh
+++ b/bin/impala-config.sh
@@ -255,19 +255,19 @@ fi
 : ${IMPALA_TOOLCHAIN_HOST:=native-toolchain.s3.amazonaws.com}
 export IMPALA_TOOLCHAIN_HOST
 
-export CDP_BUILD_NUMBER=45689292
+export CDP_BUILD_NUMBER=49623641
 export CDP_MAVEN_REPOSITORY=\
 
"https://${IMPALA_TOOLCHAIN_HOST}/build/cdp_components/${CDP_BUILD_NUMBER}/maven";
-export CDP_AVRO_JAVA_VERSION=1.8.2.7.2.18.0-369
-export CDP_HADOOP_VERSION=3.1.1.7.2.18.0-369
-export CDP_HBASE_VERSION=2.4.17.7.2.18.0-369
-export CDP_HIVE_VERSION=3.1.3000.7.2.18.0-369
-export CDP_ICEBERG_VERSION=1.3.1.7.2.18.0-369
-export CDP_KNOX_VERSION=1.3.0.7.2.18.0-369
-export CDP_OZONE_VERSION=1.3.0.7.2.18.0-369
-export CDP_PARQUET_VERSION=1.10.99.7.2.18.0-369
-export CDP_RANGER_VERSION=2.4.0.7.2.18.0-369
-export CDP_TEZ_VERSION=0.9.1.7.2.18.0-369
+export CDP_AVRO_JAVA_VERSION=1.8.2.7.3.0.0-128
+export CDP_HADOOP_VERSION=3.1.1.7.3.0.0-128
+export CDP_HBASE_VERSION=2.4.17.7.3.0.0-128
+export CDP_HIVE_VERSION=3.1.3000.7.3.0.0-128
+export CDP_ICEBERG_VERSION=1.3.1.7.3.0.0-128
+export CDP_KNOX_VERSION=2.0.0.7.3.0.0-128
+export CDP_OZONE_VERSION=1.3.0.7.3.0.0-128
+export CDP_PARQUET_VERSION=1.12.3.7.3.0.0-128
+export CDP_RANGER_VERSION=2.4.0.7.3.0.0-128
+export CDP_TEZ_VERSION=0.9.1.7.3.0.0-128
 
 # Ref: https://infra.apache.org/release-download-pages.html#closer
 : ${APACHE_MIRROR:="https://www.apache.org/dyn/closer.cgi"}
diff --git 
a/fe/src/compat-apache-hive-3/java/org/apache/impala/compat/MetastoreShim.java 
b/fe/src/compat-apache-hive-3/java/org/apache/impala/compat/MetastoreShim.java
index c34d2d8af..98c96f1d2 100644
--- 
a/fe/src/compat-apache-hive-3/java/org/apache/impala/compat/MetastoreShim.java
+++ 
b/fe/src/compat-apache-hive-3/java/org/apache/impala/compat/MetastoreShim.java
@@ -74,6 +74,7 @@ import org.apache.impala.catalog.Hive3MetastoreShimBase;
 import org.apache.impala.catalog.MetaStoreClientPool;
 import org.apache.impala.catalog.MetaStoreClientPool.MetaStoreClient;
 import org.apache.impala.catalog.events.MetastoreEvents.MetastoreEvent;
+import 
org.apache.impala.catalog.events.MetastoreEventsProcessor.MetaDataFilter;
 import org.apache.impala.catalog.events.MetastoreNotificationException;
 import org.apache.impala.catalog.events.SelfEventContext;
 import org.apache.impala.catalog.local.MetaProvider.PartitionMetadata;
@@ -980,4 +981,9 @@ public class MetastoreShim extends Hive3MetastoreShimBase {
     // IMetaStoreClient.getDataConnector().
     return null;
   }
+
+  public static void setNotificationEventRequestWithFilter(
+      NotificationEventRequest eventRequest, MetaDataFilter metaDataFilter) {
+    // noop for non-existent fields of NotificationEventRequest
+  }
 }
diff --git 
a/fe/src/compat-hive-3/java/org/apache/impala/compat/MetastoreShim.java 
b/fe/src/compat-hive-3/java/org/apache/impala/compat/MetastoreShim.java
index f5354c0a4..03feafe22 100644
--- a/fe/src/compat-hive-3/java/org/apache/impala/compat/MetastoreShim.java
+++ b/fe/src/compat-hive-3/java/org/apache/impala/compat/MetastoreShim.java
@@ -33,6 +33,7 @@ import com.google.common.collect.Iterables;
 import java.net.InetAddress;
 import java.net.UnknownHostException;
 import java.util.ArrayList;
+import java.util.Arrays;
 import java.util.Collections;
 import java.util.Comparator;
 import java.util.HashMap;
@@ -107,6 +108,7 @@ import org.apache.impala.catalog.TableWriteId;
 import org.apache.impala.catalog.events.MetastoreEvents.MetastoreEvent;
 import org.apache.impala.catalog.events.MetastoreEvents.MetastoreEventType;
 import org.apache.impala.catalog.events.MetastoreEventsProcessor;
+import 
org.apache.impala.catalog.events.MetastoreEventsProcessor.MetaDataFilter;
 import org.apache.impala.catalog.events.MetastoreNotificationException;
 import 
org.apache.impala.catalog.events.MetastoreNotificationNeedsInvalidateException;
 import org.apache.impala.catalog.events.SelfEventContext;
@@ -1082,4 +1084,22 @@ public class MetastoreShim extends 
Hive3MetastoreShimBase {
     }
     return null;
   }
+
+  public static void setNotificationEventRequestWithFilter(
+      NotificationEventRequest eventRequest, MetaDataFilter metaDataFilter) {
+    if (metaDataFilter != null) {
+      if (metaDataFilter.getCatName() != null &&
+          !metaDataFilter.getCatName().isEmpty()) {
+        eventRequest.setCatName(metaDataFilter.getCatName());
+      }
+      if (metaDataFilter.getDbName() != null &&
+          !metaDataFilter.getDbName().isEmpty()) {
+        eventRequest.setDbName(metaDataFilter.getDbName());
+      }
+      if (metaDataFilter.getTableName() != null &&
+          !metaDataFilter.getTableName().isEmpty()) {
+        
eventRequest.setTableNames(Arrays.asList(metaDataFilter.getTableName()));
+      }
+    }
+  }
 }
diff --git 
a/fe/src/main/java/org/apache/impala/catalog/events/MetastoreEventsProcessor.java
 
b/fe/src/main/java/org/apache/impala/catalog/events/MetastoreEventsProcessor.java
index 30673e835..0d0c858b7 100644
--- 
a/fe/src/main/java/org/apache/impala/catalog/events/MetastoreEventsProcessor.java
+++ 
b/fe/src/main/java/org/apache/impala/catalog/events/MetastoreEventsProcessor.java
@@ -66,6 +66,7 @@ import org.apache.impala.service.CatalogOpExecutor;
 import org.apache.impala.thrift.TEventBatchProgressInfo;
 import org.apache.impala.thrift.TEventProcessorMetrics;
 import org.apache.impala.thrift.TEventProcessorMetricsSummaryResponse;
+import org.apache.impala.util.AcidUtils;
 import org.apache.impala.util.MetaStoreUtil;
 import org.apache.impala.util.NoOpEventSequence;
 import org.apache.impala.util.ThreadNameAnnotator;
@@ -296,7 +297,9 @@ public class MetastoreEventsProcessor implements 
ExternalEventsProcessor {
         eventType.equals(notificationEvent.getEventType())
             && catName.equalsIgnoreCase(notificationEvent.getCatName())
             && dbName.equalsIgnoreCase(notificationEvent.getDbName());
-    return getNextMetastoreEventsInBatches(catalog, eventId, filter, 
eventType);
+    MetaDataFilter metaDataFilter = new MetaDataFilter(filter, catName, 
dbName);
+    return getNextMetastoreEventsWithFilterInBatches(catalog, eventId, 
metaDataFilter,
+        EVENTS_BATCH_SIZE_PER_RPC, eventType);
   }
 
   public static List<NotificationEvent> 
getNextMetastoreEventsInBatchesForTable(
@@ -318,7 +321,8 @@ public class MetastoreEventsProcessor implements 
ExternalEventsProcessor {
             && catName.equalsIgnoreCase(notificationEvent.getCatName())
             && dbName.equalsIgnoreCase(notificationEvent.getDbName())
             && tblName.equalsIgnoreCase(notificationEvent.getTableName());
-    return getNextMetastoreEventsInBatches(catalog, eventId, filter,
+    MetaDataFilter metaDataFilter = new MetaDataFilter(filter, catName, 
dbName, tblName);
+    return getNextMetastoreEventsWithFilterInBatches(catalog, eventId, 
metaDataFilter,
         EVENTS_BATCH_SIZE_PER_RPC, eventType);
   }
 
@@ -329,27 +333,37 @@ public class MetastoreEventsProcessor implements 
ExternalEventsProcessor {
         EVENTS_BATCH_SIZE_PER_RPC, eventTypes);
   }
 
-  /**
-   * Gets the next list of {@link NotificationEvent} from Hive Metastore which 
are
-   * greater than the given eventId and filtered according to the provided 
filter.
-   * @param catalog The CatalogServiceCatalog used to get the metastore client
-   * @param eventId The eventId after which the events are needed.
-   * @param filter The {@link NotificationFilter} used to filter the list of 
fetched
-   *               events. Note that this is a client side filter not a server 
side
-   *               filter. Unfortunately, HMS doesn't provide a similar 
mechanism to
-   *               do server side filtering.
-   * @param eventsBatchSize the batch size for fetching the events from 
metastore.
-   * @return List of {@link NotificationEvent} which are all greater than 
eventId and
-   * satisfy the given filter.
-   * @throws MetastoreNotificationFetchException in case of RPC errors to 
metastore.
-   */
   @VisibleForTesting
   public static List<NotificationEvent> getNextMetastoreEventsInBatches(
       CatalogServiceCatalog catalog, long eventId, NotificationFilter filter,
       int eventsBatchSize, String... eventTypes)
       throws MetastoreNotificationFetchException {
+    MetaDataFilter metaDataFilter = new MetaDataFilter(filter);
+    return getNextMetastoreEventsWithFilterInBatches(catalog, eventId, 
metaDataFilter,
+        eventsBatchSize, eventTypes);
+  }
+
+      /**
+       * Gets the next list of {@link NotificationEvent} from Hive Metastore 
which are
+       * greater than the given eventId and filtered according to the provided 
filter.
+       * @param catalog The CatalogServiceCatalog used to get the metastore 
client
+       * @param eventId The eventId after which the events are needed.
+       * @param metaDataFilter The {@link MetaDataFilter} used to filter the 
list of
+       *               fetched events based on catName/dbName/tableName and 
then filter
+       *               by required event types. Note that this is a server 
side filter.
+       * @param eventsBatchSize the batch size for fetching the events from 
metastore.
+       * @return List of {@link NotificationEvent} which are all greater than 
eventId and
+       * satisfy the given filter.
+       * @throws MetastoreNotificationFetchException in case of RPC errors to 
metastore.
+       */
+  @VisibleForTesting
+  public static List<NotificationEvent> 
getNextMetastoreEventsWithFilterInBatches(
+      CatalogServiceCatalog catalog, long eventId, MetaDataFilter 
metaDataFilter,
+      int eventsBatchSize, String... eventTypes)
+      throws MetastoreNotificationFetchException {
     Preconditions.checkArgument(eventsBatchSize > 0);
     List<NotificationEvent> result = new ArrayList<>();
+    NotificationFilter filter = metaDataFilter.getNotificationFilter();
     try (MetaStoreClient msc = catalog.getMetaStoreClient()) {
       long toEventId = msc.getHiveClient().getCurrentNotificationEventId()
           .getEventId();
@@ -376,6 +390,9 @@ public class MetastoreEventsProcessor implements 
ExternalEventsProcessor {
         NotificationEventRequest eventRequest = new NotificationEventRequest();
         eventRequest.setMaxEvents(batchSize);
         eventRequest.setLastEvent(currentEventId);
+        // Need to set table/dbnames in the request according to the filter
+        MetastoreShim.setNotificationEventRequestWithFilter(eventRequest,
+            metaDataFilter);
         NotificationEventResponse notificationEventResponse =
             MetastoreShim.getNextNotification(msc.getHiveClient(), 
eventRequest,
                 eventTypeSkipList);
@@ -432,8 +449,18 @@ public class MetastoreEventsProcessor implements 
ExternalEventsProcessor {
     String annotation = String.format("sync table %s to latest HMS event id",
         tbl.getFullName());
     try(ThreadNameAnnotator tna = new ThreadNameAnnotator(annotation)) {
-      List<NotificationEvent> events = getNextMetastoreEventsInBatches(catalog,
-          lastEventId, getTableNotificationEventFilter(tbl));
+      MetaDataFilter metaDataFilter;
+      // For ACID tables, events may include commit_txn and abort_txn which 
doesn't have
+      // db_name and table_name. So it makes sense to fetch all the events and 
filter
+      // them in catalogD.
+      if 
(AcidUtils.isTransactionalTable(tbl.getMetaStoreTable().getParameters())) {
+        metaDataFilter = new 
MetaDataFilter(getTableNotificationEventFilter(tbl));
+      } else {
+        metaDataFilter = new 
MetaDataFilter(getTableNotificationEventFilter(tbl),
+            MetastoreShim.getDefaultCatalogName(), tbl.getDb().getName(), 
tbl.getName());
+      }
+      List<NotificationEvent> events = 
getNextMetastoreEventsWithFilterInBatches(catalog,
+          lastEventId, metaDataFilter, EVENTS_BATCH_SIZE_PER_RPC);
 
       if (events.isEmpty()) {
         LOG.debug("table {} synced till event id {}. No new HMS events to 
process from "
@@ -494,8 +521,11 @@ public class MetastoreEventsProcessor implements 
ExternalEventsProcessor {
 
     String annotation = String.format("sync db %s to latest HMS event id", 
db.getName());
     try(ThreadNameAnnotator tna = new ThreadNameAnnotator(annotation)) {
-      List<NotificationEvent> events = getNextMetastoreEventsInBatches(catalog,
-          lastEventId, getDbNotificationEventFilter(db));
+      MetaDataFilter metaDataFilter = new MetaDataFilter(
+          getDbNotificationEventFilter(db), 
MetastoreShim.getDefaultCatalogName(),
+          db.getName());
+      List<NotificationEvent> events = 
getNextMetastoreEventsWithFilterInBatches(catalog,
+          lastEventId, metaDataFilter, EVENTS_BATCH_SIZE_PER_RPC);
 
       if (events.isEmpty()) {
         LOG.debug("db {} already synced till event id: {}, no new hms events 
from "
@@ -550,7 +580,8 @@ public class MetastoreEventsProcessor implements 
ExternalEventsProcessor {
   This filter is used when syncing db to the latest HMS event id. The
   filter accepts all events except table related ones
    */
-  private static NotificationFilter getDbNotificationEventFilter(Db db) {
+  @VisibleForTesting
+  public static NotificationFilter getDbNotificationEventFilter(Db db) {
     NotificationFilter filter = new NotificationFilter() {
       @Override
       public boolean accept(NotificationEvent event) {
@@ -976,6 +1007,7 @@ public class MetastoreEventsProcessor implements 
ExternalEventsProcessor {
       NotificationEventRequest eventRequest = new NotificationEventRequest();
       eventRequest.setLastEvent(eventId);
       eventRequest.setMaxEvents(batchSize);
+      // Currently filter is always null. No need to set table/db in the 
request object
       NotificationEventResponse response =
           MetastoreShim.getNextNotification(msClient.getHiveClient(), 
eventRequest,
               catalog_.getDefaultSkippedHmsEventTypes());
@@ -1435,4 +1467,48 @@ public class MetastoreEventsProcessor implements 
ExternalEventsProcessor {
   public static MessageDeserializer getMessageDeserializer() {
     return MESSAGE_DESERIALIZER;
   }
+
+  public static class MetaDataFilter {
+    public NotificationFilter filter_;
+    public String catName_;
+    public String dbName_;
+    public String tableName_;
+
+    public MetaDataFilter(NotificationFilter notificationFilter) {
+      this.filter_ = notificationFilter; // if this null then don't build 
event filter
+    }
+
+    public MetaDataFilter(NotificationFilter notificationFilter, String 
catName,
+        String dbName) {
+      this(notificationFilter);
+      this.catName_ = Preconditions.checkNotNull(catName);
+      this.dbName_ = Preconditions.checkNotNull(dbName);
+    }
+
+    public MetaDataFilter(NotificationFilter notificationFilter, String 
catName,
+        String databaseName, String tblName) {
+      this(notificationFilter, catName, databaseName);
+      this.tableName_ = tblName;
+    }
+
+    public void setNotificationFilter(NotificationFilter notificationFilter) {
+      this.filter_ = notificationFilter;
+    }
+
+    public NotificationFilter getNotificationFilter() {
+      return filter_;
+    }
+
+    public String getCatName() {
+      return catName_;
+    }
+
+    public String getDbName() {
+      return dbName_;
+    }
+
+    public String getTableName() {
+      return tableName_;
+    }
+  }
 }
diff --git 
a/fe/src/test/java/org/apache/impala/catalog/events/MetastoreEventsProcessorTest.java
 
b/fe/src/test/java/org/apache/impala/catalog/events/MetastoreEventsProcessorTest.java
index 69b8a084d..157663c5f 100644
--- 
a/fe/src/test/java/org/apache/impala/catalog/events/MetastoreEventsProcessorTest.java
+++ 
b/fe/src/test/java/org/apache/impala/catalog/events/MetastoreEventsProcessorTest.java
@@ -60,6 +60,7 @@ import 
org.apache.hadoop.hive.metastore.api.InsertEventRequestData;
 import org.apache.hadoop.hive.metastore.api.NotificationEvent;
 import org.apache.hadoop.hive.metastore.api.Partition;
 import org.apache.hadoop.hive.metastore.api.PrincipalType;
+import org.apache.hadoop.hive.metastore.IMetaStoreClient.NotificationFilter;
 import org.apache.hadoop.hive.metastore.messaging.AlterTableMessage;
 import org.apache.hadoop.io.IOUtils;
 import org.apache.impala.analysis.FunctionName;
@@ -96,6 +97,7 @@ import 
org.apache.impala.catalog.events.MetastoreEvents.MetastoreEventFactory;
 import 
org.apache.impala.catalog.events.MetastoreEvents.MetastoreEventPropertyKey;
 import org.apache.impala.catalog.events.MetastoreEvents.MetastoreEventType;
 import 
org.apache.impala.catalog.events.MetastoreEventsProcessor.EventProcessorStatus;
+import 
org.apache.impala.catalog.events.MetastoreEventsProcessor.MetaDataFilter;
 import org.apache.impala.catalog.FileMetadataLoadOpts;
 import org.apache.impala.common.FileSystemUtil;
 import org.apache.impala.common.ImpalaException;
@@ -3962,6 +3964,80 @@ public class MetastoreEventsProcessorTest {
     }
   }
 
+  @Test
+  public void testNotificationEventRequest() throws Exception {
+    long currentEventId = eventsProcessor_.getCurrentEventId();
+    int EVENTS_BATCH_SIZE_PER_RPC = 1000;
+    // Generate some DB only related events
+    createDatabaseFromImpala(TEST_DB_NAME, null);
+    String testDbParamKey = "testKey", testDbParamVal = "testVal";
+    String testTable1 = "testNotifyTable1", testTable2 = "testNotifyTable2";
+    addDatabaseParameters(testDbParamKey, testDbParamVal);
+    eventsProcessor_.processEvents();
+    // Verify DB related events only.
+
+    // Generate some table events for managed table
+    testInsertIntoTransactionalTable(testTable1, true, false);
+    alterTableAddColsFromImpala(
+        TEST_DB_NAME, testTable1, "newCol", TPrimitiveType.STRING);
+    alterTableRemoveColFromImpala(TEST_DB_NAME, testTable1, "newCol");
+    eventsProcessor_.processEvents();
+
+    // Generate some table events for external table
+    createTable(testTable2, false);
+    eventsProcessor_.processEvents();
+    alterTableSetOwnerFromImpala(TEST_DB_NAME, testTable2, "testowner");
+    alterTableSetFileFormatFromImpala(
+        TEST_DB_NAME, testTable2, THdfsFileFormat.TEXT);
+    eventsProcessor_.processEvents();
+
+    // verify DB only events
+    NotificationFilter filter = event ->
+        MetastoreEvents.CreateDatabaseEvent.EVENT_TYPE
+        .equals(event.getEventType());
+    MetaDataFilter metaDataFilter = new MetaDataFilter(filter,
+        MetastoreShim.getDefaultCatalogName(), TEST_DB_NAME);
+    assertEquals(MetastoreEventsProcessor
+        .getNextMetastoreEventsWithFilterInBatches(catalog_, currentEventId,
+            metaDataFilter, EVENTS_BATCH_SIZE_PER_RPC).size(), 1);
+    Db db = catalog_.getDb(TEST_DB_NAME);
+    metaDataFilter.setNotificationFilter(
+        MetastoreEventsProcessor.getDbNotificationEventFilter(db));
+    
assertEquals(MetastoreEventsProcessor.getNextMetastoreEventsWithFilterInBatches(
+        catalog_, currentEventId, metaDataFilter, 
EVENTS_BATCH_SIZE_PER_RPC).size(), 2);
+
+    // verify events for table 1
+    filter = event -> "ALTER_TABLE".equals(event.getEventType());
+    metaDataFilter = new MetaDataFilter(filter, 
MetastoreShim.getDefaultCatalogName(),
+        TEST_DB_NAME, testTable1);
+    
assertEquals(MetastoreEventsProcessor.getNextMetastoreEventsWithFilterInBatches(
+        catalog_, currentEventId, metaDataFilter, 
EVENTS_BATCH_SIZE_PER_RPC).size(), 2);
+    metaDataFilter.setNotificationFilter(null);
+    
assertEquals(MetastoreEventsProcessor.getNextMetastoreEventsWithFilterInBatches(
+        catalog_, currentEventId, metaDataFilter, 
EVENTS_BATCH_SIZE_PER_RPC).size(), 3);
+
+    // verify events for table 2
+    filter = event -> "ALTER_TABLE".equals(event.getEventType());
+    metaDataFilter = new MetaDataFilter(filter, 
MetastoreShim.getDefaultCatalogName(),
+        TEST_DB_NAME, testTable2);
+    
assertEquals(MetastoreEventsProcessor.getNextMetastoreEventsWithFilterInBatches(
+        catalog_, currentEventId, metaDataFilter, 
EVENTS_BATCH_SIZE_PER_RPC).size(), 2);
+    metaDataFilter.setNotificationFilter(null);
+    
assertEquals(MetastoreEventsProcessor.getNextMetastoreEventsWithFilterInBatches(
+        catalog_, currentEventId, metaDataFilter, 
EVENTS_BATCH_SIZE_PER_RPC).size(), 3);
+
+    // verify all events
+    filter = event -> "ALTER_TABLE".equals(event.getEventType());
+    metaDataFilter = new MetaDataFilter(filter, 
MetastoreShim.getDefaultCatalogName(),
+        TEST_DB_NAME);
+    
assertEquals(MetastoreEventsProcessor.getNextMetastoreEventsWithFilterInBatches(
+        catalog_, currentEventId, metaDataFilter, 
EVENTS_BATCH_SIZE_PER_RPC).size(), 4);
+    metaDataFilter.setNotificationFilter(null);
+    // 2 DB + 6 table events
+    
assertEquals(MetastoreEventsProcessor.getNextMetastoreEventsWithFilterInBatches(
+        catalog_, currentEventId, metaDataFilter, 
EVENTS_BATCH_SIZE_PER_RPC).size(), 8);
+  }
+
   private void createDatabase(String catName, String dbName,
       Map<String, String> params) throws TException {
     try(MetaStoreClient msClient = catalog_.getMetaStoreClient()) {
diff --git 
a/fe/src/test/java/org/apache/impala/catalog/metastore/CatalogHmsSyncToLatestEventIdTest.java
 
b/fe/src/test/java/org/apache/impala/catalog/metastore/CatalogHmsSyncToLatestEventIdTest.java
index 762c6fe69..fa0c71e17 100644
--- 
a/fe/src/test/java/org/apache/impala/catalog/metastore/CatalogHmsSyncToLatestEventIdTest.java
+++ 
b/fe/src/test/java/org/apache/impala/catalog/metastore/CatalogHmsSyncToLatestEventIdTest.java
@@ -459,7 +459,7 @@ public class CatalogHmsSyncToLatestEventIdTest extends 
AbstractCatalogMetastoreT
     @Test
     public void testAlterTableRename() throws Exception {
         LOG.info("Executing testALterTableRename");
-        String tblName = "test_alter_table_rename_" + tableType_ + "_tbl";
+        String tblName = ("test_alter_table_rename_" + tableType_ + 
"_tbl").toLowerCase();
         String newTblName = tblName + "_new";
         try {
             createDatabaseInCatalog(TEST_DB_NAME);
diff --git a/java/shaded-deps/hive-exec/pom.xml 
b/java/shaded-deps/hive-exec/pom.xml
index f0bce94ac..d6c7535a8 100644
--- a/java/shaded-deps/hive-exec/pom.xml
+++ b/java/shaded-deps/hive-exec/pom.xml
@@ -92,6 +92,8 @@ the same dependencies
                 <include>org/apache/hive/common/util/TxnIdUtils*</include>
                 <!-- Needed to support describe formatted command compat with 
Hive -->
                 <include>org/apache/hadoop/hive/ql/metadata/**/*</include>
+                <!-- Needed to support describe formatted command since 
HIVE-24509 -->
+                
<include>org/apache/hadoop/hive/ql/ddl/ShowUtils.class</include>
                 
<include>org/apache/hadoop/hive/ql/parse/SemanticException.class</include>
                 <!-- Needed to support Hive udfs -->
                 <include>org/apache/hadoop/hive/ql/exec/*UDF*</include>

Reply via email to