You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@impala.apache.org by jo...@apache.org on 2019/04/30 22:13:51 UTC

[impala] 02/02: IMPALA-8419 : Validate event processing related configurations

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

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

commit 66fda62213472ab1a5e07e8719c73de4d4cd7ab4
Author: Bharath Krishna <bh...@cloudera.com>
AuthorDate: Tue Apr 23 18:38:51 2019 -0700

    IMPALA-8419 : Validate event processing related configurations
    
    Using the Metastore API to get the configuration values, verify that the
    configurations needed for event processing are set correctly. Also check
    that the parameters required for event processing is not filtered out by
    the Hive config METASTORE_PARAMETER_EXCLUDE_PATTERNS.
    This validation is done while creating the event processor and throws
    CatalogException if the configuration is incorrect.
    
    Testing
    - Added unit tests
    
    Change-Id: I94c2783e36287a65122003aa55d8075a806bc606
    Reviewed-on: http://gerrit.cloudera.org:8080/13019
    Reviewed-by: Impala Public Jenkins <im...@cloudera.com>
    Tested-by: Impala Public Jenkins <im...@cloudera.com>
---
 .../events/EventProcessorConfigValidator.java      | 213 ++++++++++++++++++++
 .../impala/catalog/events/MetastoreEvents.java     |  91 ++++++---
 .../catalog/events/MetastoreEventsProcessor.java   |  31 ++-
 .../apache/impala/service/CatalogOpExecutor.java   |  35 ++--
 .../java/org/apache/impala/util/MetaStoreUtil.java |   8 +
 .../events/MetastoreEventsProcessorTest.java       | 215 +++++++++++++++++++--
 .../SynchronousHMSEventProcessorForTests.java      |   4 +-
 7 files changed, 530 insertions(+), 67 deletions(-)

diff --git a/fe/src/main/java/org/apache/impala/catalog/events/EventProcessorConfigValidator.java b/fe/src/main/java/org/apache/impala/catalog/events/EventProcessorConfigValidator.java
new file mode 100644
index 0000000..81e808a
--- /dev/null
+++ b/fe/src/main/java/org/apache/impala/catalog/events/EventProcessorConfigValidator.java
@@ -0,0 +1,213 @@
+// Licensed to the Apache Software Foundation (ASF) under one
+// or more contributor license agreements.  See the NOTICE file
+// distributed with this work for additional information
+// regarding copyright ownership.  The ASF licenses this file
+// to you under the Apache License, Version 2.0 (the
+// "License"); you may not use this file except in compliance
+// with the License.  You may obtain a copy of the License at
+//
+//   http://www.apache.org/licenses/LICENSE-2.0
+//
+// Unless required by applicable law or agreed to in writing,
+// software distributed under the License is distributed on an
+// "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+// KIND, either express or implied.  See the License for the
+// specific language governing permissions and limitations
+// under the License.
+
+package org.apache.impala.catalog.events;
+
+import static java.util.regex.Pattern.compile;
+
+import com.google.common.annotations.VisibleForTesting;
+import com.google.common.base.Preconditions;
+import java.util.Arrays;
+import java.util.Optional;
+import java.util.function.Function;
+import java.util.function.Predicate;
+import java.util.stream.Collectors;
+import org.apache.hadoop.util.StringUtils;
+import org.apache.impala.catalog.events.EventProcessorConfigValidator.ValidationResult;
+import org.apache.impala.catalog.events.MetastoreEvents.MetastoreEventPropertyKey;
+import org.apache.thrift.TException;
+import java.util.List;
+
+/**
+ * This interface contains methods to validate the event processor during its
+ * initialization. For any validation on the MetastoreEventsProcessor, it should return a
+ * ValidationResult object. If the validation is successful, a ValidationResult object
+ * with valid set as true should be returned. If the validation is unsuccessful, a
+ * ValidationResult object is returned with valid = false and also the reason for the
+ * validation failure.
+ */
+public interface EventProcessorConfigValidator extends Function<MetastoreEventsProcessor,
+    ValidationResult> {
+
+  // Metastore configurations and their expected values for event processing.
+  // These configs are validated before creating an instance of the event processor.
+  enum MetastoreEventConfigsToValidate {
+    ADD_THRIFT_OBJECTS("hive.metastore.notifications.add.thrift.objects", "true"),
+    ALTER_NOTIFICATIONS_BASIC("hive.metastore.alter.notifications.basic", "false"),
+    FIRE_EVENTS_FOR_DML("hive.metastore.dml.events", "true");
+
+    private final String conf_, expectedValue_;
+
+    MetastoreEventConfigsToValidate(String conf, String expectedValue) {
+      this.conf_ = conf;
+      this.expectedValue_ = expectedValue;
+    }
+
+    /**
+     * Returns the value expected to be set for a Metastore config which is required to
+     * run event processor successfully
+     */
+    public String getExpectedValue() { return expectedValue_; }
+
+    /**
+     * Returns the configuration name
+     */
+    public String getConf() { return conf_; }
+
+    @Override
+    public String toString() {
+      return "Config : " + conf_ + ", Expected Value : " + expectedValue_;
+    }
+  }
+
+  // Hive config name that can have regular expressions to filter out parameters.
+  String METASTORE_PARAMETER_EXCLUDE_PATTERNS =
+      "hive.metastore.notification.parameters.exclude.patterns";
+
+  // Default config value (currently just an empty String) to be returned from Metastore
+  // when the given config is not set.
+  String DEFAULT_METASTORE_CONFIG_VALUE = "";
+
+  static EventProcessorConfigValidator hasValidMetastoreConfigs() {
+    return EventProcessorConfigValidator::validateMetastoreConfigs;
+  }
+
+  static EventProcessorConfigValidator verifyParametersNotFiltered() {
+    return EventProcessorConfigValidator::validateMetastoreEventParameters;
+  }
+
+  default EventProcessorConfigValidator and(EventProcessorConfigValidator other) {
+    return metastoreEventsProcessor -> {
+      final ValidationResult result = this.apply(metastoreEventsProcessor);
+      return result.isValid() ? other.apply(metastoreEventsProcessor) : result;
+    };
+  }
+
+  /**
+   * Validates the Impala parameters required for event processing. Hive has a config
+   * METASTORE_PARAMETER_EXCLUDE_PATTERNS that filters out parameters from Metastore
+   * entities, so we need to make sure that the relevant parameters for event processing
+   * are not filtered. We do this by getting the regex from this Hive config and testing
+   * against each key in MetatsoreEvents.MetastoreEventPropertyKey. If the regex does not
+   * filter any parameter, the method returns a ValidationResult indicating success. If
+   * any parameter matches the regex, it returns a ValidationResult indicating failure,
+   * the regex and the parameter filtered.
+   */
+  @VisibleForTesting
+  static ValidationResult validateMetastoreEventParameters(
+      MetastoreEventsProcessor eventsProcessor) {
+    ValidationResult result = new ValidationResult();
+    try {
+      String filterRegex = eventsProcessor.getConfigValueFromMetastore(
+          METASTORE_PARAMETER_EXCLUDE_PATTERNS, DEFAULT_METASTORE_CONFIG_VALUE);
+      List<String> excludePatterns =
+          Arrays.asList(StringUtils.getTrimmedStrings(filterRegex));
+      // Combine all Predicates to a single Predicate (as done in Hive side).
+      Predicate<String> paramsFilter =
+          compilePatternsToPredicates(excludePatterns).stream().reduce(Predicate::or)
+              .orElse(x -> false);
+      for (MetastoreEventPropertyKey param : MetastoreEventPropertyKey.values()) {
+        if (paramsFilter.test(param.getKey())) {
+          result = new ValidationResult(false, String.format("Failed config validation. "
+                  + "Required Impala parameter %s is"
+                  + " filtered out using the Hive configuration %s=%s", param.getKey(),
+              METASTORE_PARAMETER_EXCLUDE_PATTERNS, filterRegex));
+          break;
+        }
+      }
+      return result;
+    } catch (TException e) {
+      return new ValidationResult(false,
+          "Unable to get the configuration from Metastore. Check if the Metastore "
+              + "service is accessible.");
+    }
+  }
+
+  static List<Predicate<String>> compilePatternsToPredicates(List<String> patterns) {
+    return patterns.stream().map(pattern -> compile(pattern).asPredicate()).collect(
+        Collectors.toList());
+  }
+
+  /**
+   * Validates the metastore configurations required for event processing. This method
+   * verifies that each config provided in MetastoreEventConfigsToValidate has the
+   * expected value set in Metastore. If the expected value is set for all the configs,
+   * the method returns a ValidationResult indicating success. Otherwise, it returns a
+   * ValidationResult indicating invalid config and also will include reason for failure.
+   */
+  @VisibleForTesting
+  static ValidationResult validateMetastoreConfigs(MetastoreEventsProcessor processor) {
+    ValidationResult result = new ValidationResult();
+    try {
+      for (MetastoreEventConfigsToValidate metaConf : MetastoreEventConfigsToValidate
+          .values()) {
+        String configValueFromMetastore = processor
+            .getConfigValueFromMetastore(metaConf.getConf(),
+                DEFAULT_METASTORE_CONFIG_VALUE);
+        if (!configValueFromMetastore.equals(metaConf.getExpectedValue())) {
+          result = new ValidationResult(false, String.format("Incorrect configuration "
+              + "for %s. Found : %s", metaConf.toString(), configValueFromMetastore));
+          break;
+        }
+      }
+      return result;
+    } catch (TException e) {
+      return new ValidationResult(false,
+          "Unable to get the configuration from Metastore. Check if the Metastore "
+              + "service is accessible.");
+    }
+  }
+
+
+  /**
+   * This class represents the result object for EventProcessor validation. If result is
+   * success, valid should be set to true and reason is optional(ignored). If result is
+   * failure, valid should be set to false and reason of failure should be set
+   * accordingly.
+   */
+  class ValidationResult {
+
+    // Boolean indicating whether the validation was success or failure.
+    private boolean valid_;
+    // Reason indicating the failure of validation. Only relevant when
+    // the validation fails.
+    private String reason_;
+
+    ValidationResult(Boolean valid, String reason) {
+      Preconditions.checkNotNull(valid);
+      this.valid_ = valid;
+      this.reason_ = reason;
+    }
+
+    ValidationResult() {
+      this(true, null);
+    }
+
+    boolean isValid() {
+      return valid_;
+    }
+
+    /**
+     * When reason is null, the method returns an Optional.empty. Returns an Optional with
+     * present value if the reason is not null.
+     */
+    Optional<String> getReason() {
+      return Optional.ofNullable(reason_);
+    }
+  }
+
+}
\ No newline at end of file
diff --git a/fe/src/main/java/org/apache/impala/catalog/events/MetastoreEvents.java b/fe/src/main/java/org/apache/impala/catalog/events/MetastoreEvents.java
index 23ba456..c3c3280 100644
--- a/fe/src/main/java/org/apache/impala/catalog/events/MetastoreEvents.java
+++ b/fe/src/main/java/org/apache/impala/catalog/events/MetastoreEvents.java
@@ -65,15 +65,28 @@ import org.slf4j.Logger;
  */
 public class MetastoreEvents {
 
-  // key to be used for catalog version in table properties for detecting self-events
-  public static final String CATALOG_VERSION_PROP_KEY = "impala.events.catalogVersion";
-  // key to be used for catalog service id for detecting self-events
-  public static final String CATALOG_SERVICE_ID_PROP_KEY =
-      "impala.events.catalogServiceId";
-  // flag to be set in the table/database parameters to disable event based metadata sync
-  // Note the this is a user-facing property. Any changes to this key name
-  // will break backwards compatibility
-  public static final String DISABLE_EVENT_HMS_SYNC_KEY = "impala.disableHmsSync";
+  /**
+   * This enum contains keys for parameters added in Metastore entities, relevant for
+   * event processing. When eventProcessor is instantiated, we make sure during config
+   * validation that these parameters are not filtered out through the Metastore config
+   * EVENT_NOTIFICATION_PARAMETERS_EXCLUDE_PATTERNS.
+   */
+  public enum MetastoreEventPropertyKey {
+    // key to be used for catalog version in table properties for detecting self-events
+    CATALOG_VERSION("impala.events.catalogVersion"),
+    // key to be used for catalog service id for detecting self-events
+    CATALOG_SERVICE_ID("impala.events.catalogServiceId"),
+    // flag to be set in the table/database parameters to disable event based metadata
+    // sync. Note the this is a user-facing property. Any changes to this key name
+    // will break backwards compatibility
+    DISABLE_EVENT_HMS_SYNC("impala.disableHmsSync");
+
+    private String key_;
+
+    MetastoreEventPropertyKey(String key) { this.key_ = key; }
+
+    public String getKey() { return key_; }
+  }
 
   public enum MetastoreEventType {
     CREATE_TABLE("CREATE_TABLE"),
@@ -505,9 +518,9 @@ public class MetastoreEvents {
      * level property.f
      *
      * @return Boolean value of the table property with the key
-     *     <code>DISABLE_EVENT_HMS_SYNC_KEY</code>. Else, returns the database property
-     *     which is associated with this table. Returns false if neither of the properties
-     *     are set.
+     *     <code>MetastoreEventPropertyKey.DISABLE_EVENT_HMS_SYNC</code>. Else,
+     *     returns the database property which is associated with this table. Returns
+     *     false if neither of the properties are set.
      */
     @Override
     protected boolean isEventProcessingDisabled() {
@@ -515,16 +528,21 @@ public class MetastoreEvents {
       Boolean tblProperty = getHmsSyncProperty(msTbl_);
       if (tblProperty != null) {
         infoLog("Found table level flag {} is set to {} for table {}",
-            DISABLE_EVENT_HMS_SYNC_KEY, tblProperty.toString(), getFullyQualifiedTblName());
+            MetastoreEventPropertyKey.DISABLE_EVENT_HMS_SYNC.getKey(),
+            tblProperty.toString(),
+            getFullyQualifiedTblName());
         return tblProperty;
       }
       // if the tbl property is not set check at db level
-      String dbFlagVal = catalog_.getDbProperty(dbName_, DISABLE_EVENT_HMS_SYNC_KEY);
+      String dbFlagVal = catalog_.getDbProperty(dbName_,
+          MetastoreEventPropertyKey.DISABLE_EVENT_HMS_SYNC.getKey());
       if (dbFlagVal != null) {
         // no need to spew unnecessary logs. Most tables/databases are expected to not
         // have this flag set when event based HMS polling is enabled
         debugLog("Table level flag is not set. Db level flag {} is {} for "
-            + "database {}", DISABLE_EVENT_HMS_SYNC_KEY, dbFlagVal, dbName_);
+                + "database {}",
+            MetastoreEventPropertyKey.DISABLE_EVENT_HMS_SYNC.getKey(),
+            dbFlagVal, dbName_);
       }
       // flag value of null also returns false
       return Boolean.valueOf(dbFlagVal);
@@ -532,15 +550,19 @@ public class MetastoreEvents {
 
     /**
      * Gets the value of the parameter with the key
-     * <code>DISABLE_EVENT_HMS_SYNC_KEY</code> from the given table
+     * <code>MetastoreEventPropertyKey.DISABLE_EVENT_HMS_SYNC</code> from the given
+     * table
      *
      * @return the Boolean value of the property with the key
-     *     <code>DISABLE_EVENT_HMS_SYNC_KEY</code> if it is available else returns null
+     *     <code>MetastoreEventPropertyKey.DISABLE_EVENT_HMS_SYNC</code> if it is
+     *     available else returns null
      */
     public static Boolean getHmsSyncProperty(
         org.apache.hadoop.hive.metastore.api.Table tbl) {
       if (!tbl.isSetParameters()) return null;
-      String val = tbl.getParameters().get(DISABLE_EVENT_HMS_SYNC_KEY);
+      String val =
+          tbl.getParameters()
+              .get(MetastoreEventPropertyKey.DISABLE_EVENT_HMS_SYNC.getKey());
       if (val == null || val.isEmpty()) return null;
       return Boolean.valueOf(val);
     }
@@ -559,7 +581,8 @@ public class MetastoreEvents {
 
     /**
      * Even though there is a database level property
-     * <code>DISABLE_EVENT_HMS_SYNC_KEY</code> it is only used for tables within that
+     * <code>MetastoreEventPropertyKey.DISABLE_EVENT_HMS_SYNC</code> it is only used
+     * for tables within that
      * database. As such this property does not control if the database level DDLs are
      * skipped or not.
      *
@@ -817,15 +840,18 @@ public class MetastoreEvents {
       eventSyncBeforeFlag_ = getHmsSyncProperty(msTbl_);
       eventSyncAfterFlag_ = getHmsSyncProperty(tableAfter_);
       dbFlagVal =
-          Boolean.valueOf(catalog_.getDbProperty(dbName_, DISABLE_EVENT_HMS_SYNC_KEY));
+          Boolean.valueOf(catalog_.getDbProperty(dbName_,
+              MetastoreEventPropertyKey.DISABLE_EVENT_HMS_SYNC.getKey()));
     }
 
     @Override
     protected void initSelfEventIdentifiersFromEvent() {
       versionNumberFromEvent_ = Long.parseLong(
-          getStringProperty(tableAfter_.getParameters(), CATALOG_VERSION_PROP_KEY, "-1"));
+          getStringProperty(tableAfter_.getParameters(),
+              MetastoreEventPropertyKey.CATALOG_VERSION.getKey(), "-1"));
       serviceIdFromEvent_ =
-          getStringProperty(tableAfter_.getParameters(), CATALOG_SERVICE_ID_PROP_KEY, "");
+          getStringProperty(tableAfter_.getParameters(),
+              MetastoreEventPropertyKey.CATALOG_SERVICE_ID.getKey(), "");
       try {
         if (isRename_) {
           //if this is rename event then identifiers will be in tableAfter
@@ -932,7 +958,8 @@ public class MetastoreEvents {
     /**
      * In case of alter table events, it is possible that the alter event is generated
      * because user changed the value of the parameter
-     * <code>DISABLE_EVENT_HMS_SYNC_KEY</code>. If the parameter is unchanged, it doesn't
+     * <code>MetastoreEventPropertyKey.DISABLE_EVENT_HMS_SYNC</code>. If the
+     * parameter is unchanged, it doesn't
      * matter if you use the before or after table object here since the eventual action
      * is going be invalidate or rename. If however, the parameter is changed, couple of
      * things could happen. The flag changes from unset/false to true or it changes from
@@ -1125,9 +1152,11 @@ public class MetastoreEvents {
     @Override
     protected void initSelfEventIdentifiersFromEvent() {
       versionNumberFromEvent_ = Long.parseLong(getStringProperty(
-          alteredDatabase_.getParameters(), CATALOG_VERSION_PROP_KEY, "-1"));
+          alteredDatabase_.getParameters(),
+          MetastoreEventPropertyKey.CATALOG_VERSION.getKey(), "-1"));
       serviceIdFromEvent_ = getStringProperty(
-          alteredDatabase_.getParameters(), CATALOG_SERVICE_ID_PROP_KEY, "");
+          alteredDatabase_.getParameters(),
+          MetastoreEventPropertyKey.CATALOG_SERVICE_ID.getKey(), "");
       try {
         pendingVersionNumbersFromCatalog_ =
             catalog_.getInFlightVersionsForEvents(dbName_, tblName_);
@@ -1286,9 +1315,11 @@ public class MetastoreEvents {
     @Override
     protected void initSelfEventIdentifiersFromEvent() {
       versionNumberFromEvent_ = Long.parseLong(getStringProperty(
-          lastAddedPartition_.getParameters(), CATALOG_VERSION_PROP_KEY, "-1"));
+          lastAddedPartition_.getParameters(),
+          MetastoreEventPropertyKey.CATALOG_VERSION.getKey(), "-1"));
       serviceIdFromEvent_ = getStringProperty(
-          lastAddedPartition_.getParameters(), CATALOG_SERVICE_ID_PROP_KEY, "");
+          lastAddedPartition_.getParameters(),
+          MetastoreEventPropertyKey.CATALOG_SERVICE_ID.getKey(), "");
       try {
         pendingVersionNumbersFromCatalog_ =
             catalog_.getInFlightVersionsForEvents(dbName_, tblName_);
@@ -1334,9 +1365,11 @@ public class MetastoreEvents {
     @Override
     protected void initSelfEventIdentifiersFromEvent() {
       versionNumberFromEvent_ = Long.parseLong(getStringProperty(
-          partitionAfter_.getParameters(), CATALOG_VERSION_PROP_KEY, "-1"));
+          partitionAfter_.getParameters(),
+          MetastoreEventPropertyKey.CATALOG_VERSION.getKey(), "-1"));
       serviceIdFromEvent_ = getStringProperty(
-          partitionAfter_.getParameters(), CATALOG_SERVICE_ID_PROP_KEY, "");
+          partitionAfter_.getParameters(),
+          MetastoreEventPropertyKey.CATALOG_SERVICE_ID.getKey(), "");
       try {
         pendingVersionNumbersFromCatalog_ =
             catalog_.getInFlightVersionsForEvents(dbName_, tblName_);
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 679344a..5723961 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
@@ -17,6 +17,9 @@
 
 package org.apache.impala.catalog.events;
 
+import static org.apache.impala.catalog.events.EventProcessorConfigValidator.hasValidMetastoreConfigs;
+import static org.apache.impala.catalog.events.EventProcessorConfigValidator.verifyParametersNotFiltered;
+
 import com.codahale.metrics.Gauge;
 import com.codahale.metrics.Timer;
 import com.google.common.annotations.VisibleForTesting;
@@ -28,6 +31,7 @@ import java.util.concurrent.Executors;
 import java.util.concurrent.ScheduledExecutorService;
 import java.util.concurrent.TimeUnit;
 import java.util.concurrent.atomic.AtomicLong;
+import org.apache.hadoop.hive.metastore.IMetaStoreClient;
 import org.apache.hadoop.hive.metastore.api.CurrentNotificationEventId;
 import org.apache.hadoop.hive.metastore.api.NotificationEvent;
 import org.apache.hadoop.hive.metastore.api.NotificationEventResponse;
@@ -36,11 +40,13 @@ import org.apache.hadoop.hive.metastore.messaging.json.ExtendedJSONMessageFactor
 import org.apache.impala.catalog.CatalogException;
 import org.apache.impala.catalog.CatalogServiceCatalog;
 import org.apache.impala.catalog.MetaStoreClientPool.MetaStoreClient;
+import org.apache.impala.catalog.events.EventProcessorConfigValidator.ValidationResult;
 import org.apache.impala.catalog.events.MetastoreEvents.MetastoreEvent;
 import org.apache.impala.catalog.events.MetastoreEvents.MetastoreEventFactory;
 import org.apache.impala.common.Metrics;
 import org.apache.impala.thrift.TEventProcessorMetrics;
 import org.apache.impala.thrift.TEventProcessorMetricsSummaryResponse;
+import org.apache.impala.util.MetaStoreUtil;
 import org.apache.thrift.TException;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
@@ -202,7 +208,6 @@ public class MetastoreEventsProcessor implements ExternalEventsProcessor {
   // metric name for number of tables which are invalidated by event processor so far
   public static final String NUMBER_OF_TABLE_INVALIDATES = "tables-invalidated";
 
-
   // possible status of event processor
   public enum EventProcessorStatus {
     PAUSED, // event processor is paused because catalog is being reset concurrently
@@ -241,9 +246,15 @@ public class MetastoreEventsProcessor implements ExternalEventsProcessor {
 
   @VisibleForTesting
   MetastoreEventsProcessor(CatalogServiceCatalog catalog, long startSyncFromId,
-      long pollingFrequencyInSec) {
+      long pollingFrequencyInSec) throws CatalogException {
     Preconditions.checkState(pollingFrequencyInSec > 0);
     this.catalog_ = Preconditions.checkNotNull(catalog);
+    ValidationResult result =
+        hasValidMetastoreConfigs().and(verifyParametersNotFiltered()).apply(this);
+    if (!result.isValid()) {
+      throw new CatalogException(result.getReason().orElse("Event Processor "
+          + "initialization failed during validation check."));
+    }
     lastSyncedEventId_.set(startSyncFromId);
     metastoreEventFactory_ = new MetastoreEventFactory(catalog_, metrics_);
     pollingFrequencyInSec_ = pollingFrequencyInSec;
@@ -289,6 +300,19 @@ public class MetastoreEventsProcessor implements ExternalEventsProcessor {
   }
 
   /**
+   * Returns the value for a given config key from Hive Metastore.
+   * @param config Hive configuration name
+   * @param defaultVal Default value to return if config not present in Hive
+   */
+  @VisibleForTesting
+  public String getConfigValueFromMetastore(String config, String defaultVal)
+      throws TException {
+    try (MetaStoreClient metaStoreClient = catalog_.getMetaStoreClient()) {
+      IMetaStoreClient iMetaStoreClient = metaStoreClient.getHiveClient();
+      return MetaStoreUtil.getMetastoreConfigValue(iMetaStoreClient, config, defaultVal);
+    }
+  }
+  /**
    * returns the current value of LastSyncedEventId. This method is not thread-safe and
    * only to be used for testing purposes
    */
@@ -592,7 +616,8 @@ public class MetastoreEventsProcessor implements ExternalEventsProcessor {
    *     instantiated
    */
   public static synchronized ExternalEventsProcessor getInstance(
-      CatalogServiceCatalog catalog, long startSyncFromId, long eventPollingInterval) {
+      CatalogServiceCatalog catalog, long startSyncFromId, long eventPollingInterval)
+      throws CatalogException {
     if (instance != null) {
       return instance;
     }
diff --git a/fe/src/main/java/org/apache/impala/service/CatalogOpExecutor.java b/fe/src/main/java/org/apache/impala/service/CatalogOpExecutor.java
index e8491ed..a356749 100644
--- a/fe/src/main/java/org/apache/impala/service/CatalogOpExecutor.java
+++ b/fe/src/main/java/org/apache/impala/service/CatalogOpExecutor.java
@@ -90,6 +90,7 @@ import org.apache.impala.catalog.TableNotFoundException;
 import org.apache.impala.catalog.Type;
 import org.apache.impala.catalog.View;
 import org.apache.impala.catalog.events.MetastoreEvents;
+import org.apache.impala.catalog.events.MetastoreEvents.MetastoreEventPropertyKey;
 import org.apache.impala.common.FileSystemUtil;
 import org.apache.impala.common.ImpalaException;
 import org.apache.impala.common.ImpalaRuntimeException;
@@ -881,8 +882,11 @@ public class CatalogOpExecutor {
       long newCatalogVersion) {
     if (!catalog_.isExternalEventProcessingEnabled()) return;
     org.apache.hadoop.hive.metastore.api.Table msTbl = tbl.getMetaStoreTable();
-    msTbl.putToParameters(MetastoreEvents.CATALOG_SERVICE_ID_PROP_KEY, catalogServiceId);
-    msTbl.putToParameters(MetastoreEvents.CATALOG_VERSION_PROP_KEY,
+    msTbl.putToParameters(
+        MetastoreEventPropertyKey.CATALOG_SERVICE_ID.getKey(),
+        catalogServiceId);
+    msTbl.putToParameters(
+        MetastoreEventPropertyKey.CATALOG_VERSION.getKey(),
         String.valueOf(newCatalogVersion));
   }
 
@@ -3090,17 +3094,21 @@ public class CatalogOpExecutor {
     Preconditions.checkState(msTbl.isSetParameters());
     Map<String, String> tblParams = msTbl.getParameters();
     Preconditions
-        .checkState(tblParams.containsKey(MetastoreEvents.CATALOG_SERVICE_ID_PROP_KEY),
+        .checkState(tblParams.containsKey(
+            MetastoreEventPropertyKey.CATALOG_SERVICE_ID.getKey()),
             "Table parameters must have catalog service identifier before "
                 + "adding it to partition parameters");
     Preconditions
-        .checkState(tblParams.containsKey(MetastoreEvents.CATALOG_VERSION_PROP_KEY),
+        .checkState(tblParams.containsKey(
+            MetastoreEventPropertyKey.CATALOG_VERSION.getKey()),
             "Table parameters must contain catalog version before adding "
                 + "it to partition parameters");
-    partition.putToParameters(MetastoreEvents.CATALOG_SERVICE_ID_PROP_KEY,
-        tblParams.get(MetastoreEvents.CATALOG_SERVICE_ID_PROP_KEY));
-    partition.putToParameters(MetastoreEvents.CATALOG_VERSION_PROP_KEY,
-        tblParams.get(MetastoreEvents.CATALOG_VERSION_PROP_KEY));
+    partition.putToParameters(
+        MetastoreEventPropertyKey.CATALOG_SERVICE_ID.getKey(),
+        tblParams.get(MetastoreEventPropertyKey.CATALOG_SERVICE_ID.getKey()));
+    partition.putToParameters(
+        MetastoreEventPropertyKey.CATALOG_VERSION.getKey(),
+        tblParams.get(MetastoreEventPropertyKey.CATALOG_VERSION.getKey()));
   }
 
   /**
@@ -3955,16 +3963,17 @@ public class CatalogOpExecutor {
   }
 
   /**
-   * Adds the catalog service id and the given catalog version to the database
-   * parameters. No-op if event processing is disabled
+   * Adds the catalog service id and the given catalog version to the database parameters.
+   * No-op if event processing is disabled
    */
   private void addCatalogServiceIdentifiers(
       Db db, String catalogServiceId, long newCatalogVersion) {
     if (!catalog_.isExternalEventProcessingEnabled()) return;
     org.apache.hadoop.hive.metastore.api.Database msDb = db.getMetaStoreDb();
-    msDb.putToParameters(MetastoreEvents.CATALOG_SERVICE_ID_PROP_KEY, catalogServiceId);
-    msDb.putToParameters(
-        MetastoreEvents.CATALOG_VERSION_PROP_KEY, String.valueOf(newCatalogVersion));
+    msDb.putToParameters(MetastoreEventPropertyKey.CATALOG_SERVICE_ID.getKey(),
+        catalogServiceId);
+    msDb.putToParameters(MetastoreEventPropertyKey.CATALOG_VERSION.getKey(),
+        String.valueOf(newCatalogVersion));
   }
 
   private void addDbToCatalogUpdate(Db db, TCatalogUpdateResult result) {
diff --git a/fe/src/main/java/org/apache/impala/util/MetaStoreUtil.java b/fe/src/main/java/org/apache/impala/util/MetaStoreUtil.java
index bffd46a..2b5551d 100644
--- a/fe/src/main/java/org/apache/impala/util/MetaStoreUtil.java
+++ b/fe/src/main/java/org/apache/impala/util/MetaStoreUtil.java
@@ -112,6 +112,14 @@ public class MetaStoreUtil {
   }
 
   /**
+   * Return the value set for the given config in the metastore.
+   */
+  public static String getMetastoreConfigValue(
+      IMetaStoreClient client, String config, String defaultVal) throws TException {
+    return client.getConfigValue(config, defaultVal);
+  }
+
+  /**
    * Fetches all partitions for a table in batches, with each batch containing at most
    * 'maxPartsPerRpc' partitions. Returns a List containing all fetched Partitions.
    * Will throw a MetaException if existing partitions are dropped while a fetch is in
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 7b15057..f6ff4ad 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
@@ -18,6 +18,10 @@
 package org.apache.impala.catalog.events;
 
 import static java.lang.Thread.sleep;
+import static org.apache.impala.catalog.events.EventProcessorConfigValidator.DEFAULT_METASTORE_CONFIG_VALUE;
+import static org.apache.impala.catalog.events.EventProcessorConfigValidator.METASTORE_PARAMETER_EXCLUDE_PATTERNS;
+import static org.apache.impala.catalog.events.EventProcessorConfigValidator.validateMetastoreConfigs;
+import static org.apache.impala.catalog.events.EventProcessorConfigValidator.validateMetastoreEventParameters;
 import static org.apache.impala.catalog.events.MetastoreEvents.MetastoreEventType.ALTER_TABLE;
 import static org.apache.impala.catalog.events.MetastoreEvents.MetastoreEventType.CREATE_DATABASE;
 import static org.apache.impala.catalog.events.MetastoreEvents.MetastoreEventType.DROP_DATABASE;
@@ -28,6 +32,7 @@ import static org.junit.Assert.assertNotNull;
 import static org.junit.Assert.assertNull;
 import static org.junit.Assert.assertTrue;
 import static org.junit.Assert.fail;
+import static org.mockito.Mockito.when;
 
 import com.google.common.base.Preconditions;
 import com.google.common.collect.Iterables;
@@ -43,9 +48,11 @@ import java.util.Random;
 import java.util.Set;
 import java.util.concurrent.atomic.AtomicBoolean;
 import java.util.concurrent.atomic.AtomicLong;
+
 import org.apache.hadoop.fs.FSDataOutputStream;
 import org.apache.hadoop.fs.FileSystem;
 import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.hive.metastore.IMetaStoreClient;
 import org.apache.hadoop.hive.metastore.api.CurrentNotificationEventId;
 import org.apache.hadoop.hive.metastore.api.Database;
 import org.apache.hadoop.hive.metastore.api.FieldSchema;
@@ -71,9 +78,12 @@ import org.apache.impala.catalog.IncompleteTable;
 import org.apache.impala.catalog.MetaStoreClientPool;
 import org.apache.impala.catalog.MetaStoreClientPool.MetaStoreClient;
 import org.apache.impala.catalog.Table;
+import org.apache.impala.catalog.events.EventProcessorConfigValidator.MetastoreEventConfigsToValidate;
+import org.apache.impala.catalog.events.EventProcessorConfigValidator.ValidationResult;
 import org.apache.impala.catalog.events.MetastoreEvents.AlterTableEvent;
 import org.apache.impala.catalog.events.MetastoreEvents.InsertEvent;
 import org.apache.impala.catalog.events.MetastoreEvents.MetastoreEvent;
+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.common.FileSystemUtil;
@@ -120,6 +130,7 @@ import org.apache.impala.thrift.TTableRowFormat;
 import org.apache.impala.thrift.TTypeNode;
 import org.apache.impala.thrift.TTypeNodeType;
 import org.apache.impala.thrift.TUniqueId;
+import org.apache.impala.util.MetaStoreUtil;
 import org.apache.thrift.TException;
 import org.junit.After;
 import org.junit.AfterClass;
@@ -131,6 +142,7 @@ import org.junit.Test;
 
 import com.google.common.collect.Lists;
 
+import org.mockito.Mockito;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
@@ -150,6 +162,33 @@ public class MetastoreEventsProcessorTest {
   private static CatalogServiceCatalog catalog_;
   private static CatalogOpExecutor catalogOpExecutor_;
   private static MetastoreEventsProcessor eventsProcessor_;
+
+  // This enum contains incorrect config values for each key, to test that event processor
+  // fails when incorrect value is set in Metastore for the required configs. It also
+  // has the correct value to verify the case that event processor does not fail with
+  // correct config values.
+  private enum TestIncorrectMetastoreEventConfigs {
+    ADD_THRIFT_OBJECTS("hive.metastore.notifications.add.thrift.objects", "true",
+        "false"),
+    ALTER_NOTIFICATIONS_BASIC("hive.metastore.alter.notifications.basic", "false",
+        "true"),
+    FIRE_EVENTS_FOR_DML("hive.metastore.dml.events", "true", "false");
+
+    private String conf_, correctValue_, incorrectValue_;
+
+    TestIncorrectMetastoreEventConfigs(String conf, String correctValue,
+        String incorrectValue) {
+      this.conf_ = conf;
+      this.correctValue_ = correctValue;
+      this.incorrectValue_ = incorrectValue;
+    }
+
+    @Override
+    public String toString() {
+      return "Config : " + conf_ + ", Expected Value : " + correctValue_;
+    }
+  }
+
   private static final Logger LOG =
       LoggerFactory.getLogger(MetastoreEventsProcessorTest.class);
 
@@ -222,6 +261,132 @@ public class MetastoreEventsProcessorTest {
   }
 
   /**
+   * Test each Metastore config required for event processing. This test just validates
+   * that if event processor starts, the required configs are set.
+   */
+  @Test
+  public void testConfigValidation() throws TException {
+    assertEquals(EventProcessorStatus.ACTIVE, eventsProcessor_.getStatus());
+
+    IMetaStoreClient metaStoreClient = catalog_.getMetaStoreClient().getHiveClient();
+    for (MetastoreEventConfigsToValidate config : MetastoreEventConfigsToValidate
+        .values()) {
+      assertEquals(config.getExpectedValue(),
+          MetaStoreUtil.getMetastoreConfigValue(metaStoreClient, config.getConf(),
+              DEFAULT_METASTORE_CONFIG_VALUE));
+    }
+  }
+
+  /**
+   * Mock the eventsProcessor#getConfigValueFromMetastore method and make it return
+   * incorrect values for the required configs for event processing. Verify that the
+   * incorrect config leads to invalid result for each config.
+   */
+  @Test
+  public void testConfigValidationWithIncorrectValues() throws TException {
+    MetastoreEventsProcessor mockEventsProcessor =
+        Mockito.mock(MetastoreEventsProcessor.class);
+    for (TestIncorrectMetastoreEventConfigs config :
+        TestIncorrectMetastoreEventConfigs.values()) {
+      when(mockEventsProcessor.getConfigValueFromMetastore(config.conf_,
+          DEFAULT_METASTORE_CONFIG_VALUE))
+          .thenReturn(config.incorrectValue_);
+      ValidationResult testResult =
+          validateMetastoreConfigs(mockEventsProcessor);
+      assertFalse(testResult.isValid());
+      assertTrue(testResult.getReason().isPresent());
+      assertEquals(String.format("Incorrect configuration for %s. "
+              + "Found : %s", config.toString(), config.incorrectValue_),
+          testResult.getReason().get());
+
+      when(mockEventsProcessor.getConfigValueFromMetastore(config.conf_,
+          DEFAULT_METASTORE_CONFIG_VALUE))
+          .thenReturn(config.correctValue_);
+    }
+  }
+
+  /**
+   * Test that when HiveConf.METASTORE_PARAMETER_EXCLUDE_PATTERNS contains a regex
+   * that filters out any parameter required for event processing, the config
+   * validation fails. Config validation should succeed when the regex does not match
+   * any of the required parameters.
+   */
+  @Test
+  public void testParameterFilterValidation() throws TException {
+    MetastoreEventsProcessor mockEventsProcessor =
+        Mockito.mock(MetastoreEventsProcessor.class);
+
+    //Regex to filter all parameters starting with impala
+    when(mockEventsProcessor.getConfigValueFromMetastore(
+        METASTORE_PARAMETER_EXCLUDE_PATTERNS,
+        DEFAULT_METASTORE_CONFIG_VALUE)).thenReturn("^impala");
+    ValidationResult testResult =
+        validateMetastoreEventParameters(mockEventsProcessor);
+    assertFalse(testResult.isValid());
+    assertTrue(testResult.getReason().isPresent());
+
+    when(mockEventsProcessor.getConfigValueFromMetastore(
+        METASTORE_PARAMETER_EXCLUDE_PATTERNS,
+        DEFAULT_METASTORE_CONFIG_VALUE)).thenReturn("impala*");
+    testResult =
+        validateMetastoreEventParameters(mockEventsProcessor);
+    assertFalse(testResult.isValid());
+    assertTrue(testResult.getReason().isPresent());
+
+    when(mockEventsProcessor.getConfigValueFromMetastore(
+        METASTORE_PARAMETER_EXCLUDE_PATTERNS,
+        DEFAULT_METASTORE_CONFIG_VALUE)).thenReturn("");
+    testResult =
+        validateMetastoreEventParameters(mockEventsProcessor);
+    assertTrue(testResult.isValid());
+    assertFalse(testResult.getReason().isPresent());
+
+    // Test with default return value
+    when(mockEventsProcessor.getConfigValueFromMetastore(
+        METASTORE_PARAMETER_EXCLUDE_PATTERNS,
+        DEFAULT_METASTORE_CONFIG_VALUE))
+        .thenReturn(DEFAULT_METASTORE_CONFIG_VALUE);
+    testResult =
+        validateMetastoreEventParameters(mockEventsProcessor);
+    assertTrue(testResult.isValid());
+    assertFalse(testResult.getReason().isPresent());
+
+    when(mockEventsProcessor.getConfigValueFromMetastore(
+        METASTORE_PARAMETER_EXCLUDE_PATTERNS,
+        DEFAULT_METASTORE_CONFIG_VALUE)).thenReturn("randomString1, impala"
+        + ".disableHmsSync, randomString2");
+    testResult =
+        validateMetastoreEventParameters(mockEventsProcessor);
+    assertFalse(testResult.isValid());
+    assertTrue(testResult.getReason().isPresent());
+
+    //Test when a required parameter is given as regex
+    String requiredParameter = MetastoreEventPropertyKey.CATALOG_SERVICE_ID.getKey();
+    when(mockEventsProcessor.getConfigValueFromMetastore(
+        METASTORE_PARAMETER_EXCLUDE_PATTERNS, DEFAULT_METASTORE_CONFIG_VALUE))
+        .thenReturn(requiredParameter);
+    testResult =
+        validateMetastoreEventParameters(mockEventsProcessor);
+    assertFalse(testResult.isValid());
+    assertTrue(testResult.getReason().isPresent());
+    // Verify that the error message is correct
+    assertEquals(String.format("Failed config validation. "
+            + "Required Impala parameter %s is"
+            + " filtered out using the Hive configuration %s=%s", requiredParameter,
+        METASTORE_PARAMETER_EXCLUDE_PATTERNS, requiredParameter),
+        testResult.getReason().get());
+
+    when(mockEventsProcessor.getConfigValueFromMetastore(
+        METASTORE_PARAMETER_EXCLUDE_PATTERNS,
+        DEFAULT_METASTORE_CONFIG_VALUE)).thenReturn("^impala.events"
+        + ".catalogServiceId");
+    testResult =
+        validateMetastoreEventParameters(mockEventsProcessor);
+    assertFalse(testResult.isValid());
+    assertTrue(testResult.getReason().isPresent());
+  }
+
+  /**
    * Checks that database exists after processing a CREATE_DATABASE event
    */
   @Test
@@ -814,7 +979,8 @@ public class MetastoreEventsProcessorTest {
   private static class HMSFetchNotificationsEventProcessor
       extends MetastoreEventsProcessor {
     HMSFetchNotificationsEventProcessor(
-        CatalogServiceCatalog catalog, long startSyncFromId, long pollingFrequencyInSec) {
+        CatalogServiceCatalog catalog, long startSyncFromId, long pollingFrequencyInSec)
+        throws CatalogException {
       super(catalog, startSyncFromId, pollingFrequencyInSec);
     }
 
@@ -1086,13 +1252,13 @@ public class MetastoreEventsProcessorTest {
 
   /**
    * Test checks if the events are processed or ignored when the value of parameter
-   * <code>MetastoreEvents.DISABLE_EVENT_HMS_SYNC_KEY</code> is changed. Currently,
-   * this test only changes the flags for at the table level, since alter_database
-   * events are not supported currently. In order to confirm that the event processing
-   * happens as expected, this test generates a alter_table event using a mock
-   * notificationEvent and a mock catalog which returns the dbFlag flag as expected.
-   * Then it makes sure that the <code>isEventProcessingDisabled</code> method of the
-   * AlterTableEvent returns the expected result, given the flags. And then
+   * <code>MetastoreEventPropertyKey.DISABLE_EVENT_HMS_SYNC</code> is changed.
+   * Currently, this test only changes the flags for at the table level, since
+   * alter_database events are not supported currently. In order to confirm that the
+   * event processing happens as expected, this test generates a alter_table event
+   * using a mock notificationEvent and a mock catalog which returns the dbFlag flag as
+   * expected. Then it makes sure that the <code>isEventProcessingDisabled</code>
+   * method of the AlterTableEvent returns the expected result, given the flags. And then
    * generates a additional alter table event to make sure that the subsequent event is
    * processed/skipped based on the new flag values
    */
@@ -1149,11 +1315,12 @@ public class MetastoreEventsProcessorTest {
       for (Pair<String, String> tblTransition : tblFlagTransitions) {
         Map<String, String> dbParams = new HashMap<>(1);
         if (dbFlag != null) {
-          dbParams.put(MetastoreEvents.DISABLE_EVENT_HMS_SYNC_KEY, dbFlag);
+          dbParams.put(MetastoreEventPropertyKey.DISABLE_EVENT_HMS_SYNC.getKey(), dbFlag);
         }
         Map<String, String> tblParams = new HashMap<>(1);
         if (tblTransition.first != null) {
-          tblParams.put(MetastoreEvents.DISABLE_EVENT_HMS_SYNC_KEY, tblTransition.first);
+          tblParams.put(MetastoreEventPropertyKey.DISABLE_EVENT_HMS_SYNC.getKey(),
+              tblTransition.first);
         }
         createDatabase(TEST_DB_NAME, dbParams);
         createTable(TEST_DB_NAME, testTblName, tblParams, false);
@@ -1162,7 +1329,8 @@ public class MetastoreEventsProcessorTest {
         assertNull(catalog_.getTable(TEST_DB_NAME, testTblName));
         // now turn on the flag
         alterTableAddParameter(testTblName,
-            MetastoreEvents.DISABLE_EVENT_HMS_SYNC_KEY, tblTransition.second);
+            MetastoreEventPropertyKey.DISABLE_EVENT_HMS_SYNC.getKey(),
+            tblTransition.second);
         eventsProcessor_.processEvents();
         assertEquals(EventProcessorStatus.NEEDS_INVALIDATE, eventsProcessor_.getStatus());
         // issue a catalog reset to make sure that table comes back again and event
@@ -1226,7 +1394,7 @@ public class MetastoreEventsProcessorTest {
     }
 
     private static final List<String> TABLE_SYNC_PROPERTYLIST =
-        Arrays.asList(MetastoreEvents.DISABLE_EVENT_HMS_SYNC_KEY);
+        Arrays.asList(MetastoreEventPropertyKey.DISABLE_EVENT_HMS_SYNC.getKey());
 
     @Override
     public List<String> getTableProperties(
@@ -1252,14 +1420,16 @@ public class MetastoreEventsProcessorTest {
     beforeParams.put(Table.TBL_PROP_LAST_DDL_TIME, String.valueOf(1000));
     if (tblFlagTransition.first != null) {
       beforeParams
-          .put(MetastoreEvents.DISABLE_EVENT_HMS_SYNC_KEY, tblFlagTransition.first);
+          .put(MetastoreEventPropertyKey.DISABLE_EVENT_HMS_SYNC.getKey(),
+              tblFlagTransition.first);
     }
 
     Map<String, String> afterParams = new HashMap<>(2);
     afterParams.put(Table.TBL_PROP_LAST_DDL_TIME, String.valueOf(1001));
     if (tblFlagTransition.second != null) {
       afterParams
-          .put(MetastoreEvents.DISABLE_EVENT_HMS_SYNC_KEY, tblFlagTransition.second);
+          .put(MetastoreEventPropertyKey.DISABLE_EVENT_HMS_SYNC.getKey(),
+              tblFlagTransition.second);
     }
 
     org.apache.hadoop.hive.metastore.api.Table tableBefore =
@@ -1269,7 +1439,7 @@ public class MetastoreEventsProcessorTest {
 
     Map<String, String> dbParams = new HashMap<>(1);
     if (dbFlag != null) {
-      dbParams.put(MetastoreEvents.DISABLE_EVENT_HMS_SYNC_KEY, dbFlag);
+      dbParams.put(MetastoreEventPropertyKey.DISABLE_EVENT_HMS_SYNC.getKey(), dbFlag);
     }
 
     CatalogServiceCatalog fakeCatalog = FakeCatalogServiceCatalogForFlagTests.create();
@@ -1342,7 +1512,7 @@ public class MetastoreEventsProcessorTest {
     // event 1
     createDatabase(TEST_DB_NAME, null);
     Map<String, String> tblParams = new HashMap<>(1);
-    tblParams.put(MetastoreEvents.DISABLE_EVENT_HMS_SYNC_KEY, "true");
+    tblParams.put(MetastoreEventPropertyKey.DISABLE_EVENT_HMS_SYNC.getKey(), "true");
     // event 2
     createTable(TEST_DB_NAME, "tbl_should_skipped", tblParams, true);
     // event 3
@@ -1450,15 +1620,17 @@ public class MetastoreEventsProcessorTest {
     Map<String, String> tblParams = new HashMap<>(1);
     if (dbFlag == null) {
       // if null, remove the flag
-      dbParams.remove(MetastoreEvents.DISABLE_EVENT_HMS_SYNC_KEY);
+      dbParams.remove(MetastoreEventPropertyKey.DISABLE_EVENT_HMS_SYNC.getKey());
     } else {
-      dbParams.put(MetastoreEvents.DISABLE_EVENT_HMS_SYNC_KEY, String.valueOf(dbFlag));
+      dbParams.put(MetastoreEventPropertyKey.DISABLE_EVENT_HMS_SYNC.getKey(),
+          String.valueOf(dbFlag));
     }
     if (tblFlag == null) {
-      tblParams.remove(MetastoreEvents.DISABLE_EVENT_HMS_SYNC_KEY);
+      tblParams.remove(MetastoreEventPropertyKey.DISABLE_EVENT_HMS_SYNC.getKey());
     } else {
       tblParams
-          .put(MetastoreEvents.DISABLE_EVENT_HMS_SYNC_KEY, String.valueOf(tblFlag));
+          .put(MetastoreEventPropertyKey.DISABLE_EVENT_HMS_SYNC.getKey(),
+              String.valueOf(tblFlag));
     }
 
     final String testTblName = "runDDLTestsWithFlags";
@@ -1664,7 +1836,8 @@ public class MetastoreEventsProcessorTest {
     final String testTblName = "testAlterDisableFlagFromDb";
     eventsProcessor_.processEvents();
     Database alteredDb = catalog_.getDb(TEST_DB_NAME).getMetaStoreDb().deepCopy();
-    alteredDb.putToParameters(MetastoreEvents.DISABLE_EVENT_HMS_SYNC_KEY, "true");
+    alteredDb.putToParameters(MetastoreEventPropertyKey.DISABLE_EVENT_HMS_SYNC.getKey(),
+        "true");
     alterDatabase(alteredDb);
 
     createTable(testTblName, false);
diff --git a/fe/src/test/java/org/apache/impala/catalog/events/SynchronousHMSEventProcessorForTests.java b/fe/src/test/java/org/apache/impala/catalog/events/SynchronousHMSEventProcessorForTests.java
index 2976bbe..f71dfc6 100644
--- a/fe/src/test/java/org/apache/impala/catalog/events/SynchronousHMSEventProcessorForTests.java
+++ b/fe/src/test/java/org/apache/impala/catalog/events/SynchronousHMSEventProcessorForTests.java
@@ -17,6 +17,7 @@
 
 package org.apache.impala.catalog.events;
 
+import org.apache.impala.catalog.CatalogException;
 import org.apache.impala.catalog.CatalogServiceCatalog;
 
 /**
@@ -25,7 +26,8 @@ import org.apache.impala.catalog.CatalogServiceCatalog;
  */
 public class SynchronousHMSEventProcessorForTests extends MetastoreEventsProcessor {
   SynchronousHMSEventProcessorForTests(
-      CatalogServiceCatalog catalog, long startSyncFromId, long pollingFrequencyInSec) {
+      CatalogServiceCatalog catalog, long startSyncFromId, long pollingFrequencyInSec)
+          throws CatalogException {
     super(catalog, startSyncFromId, pollingFrequencyInSec);
   }