You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@impala.apache.org by st...@apache.org on 2021/12/15 08:38:36 UTC

[impala] branch master updated (157086c -> b25e250)

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

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


    from 157086c  IMPALA-10771: Add Tencent COS support
     new ad29ce7  IMPALA-11040: Remove unnecessary reset() method in class 'UnionStmt'
     new 8d92ba1  IMPALA-11055: Remove "exit" from HBase data loading script.
     new 151dce8  IMPALA-10956: datasketches UDFs: memory leak and merge overhead
     new b25e250  IMPALA-10926: Improve catalogd consistency and self events detection

The 4 revisions listed above as "new" are entirely new to this
repository and will be described in separate emails.  The revisions
listed as "add" were already present in the repository and have only
been added to this reference.


Summary of changes:
 be/src/catalog/catalog-server.cc                   |    6 +
 be/src/exprs/aggregate-functions-ir.cc             |  468 ++++---
 be/src/util/backend-gflag-util.cc                  |    3 +
 common/thrift/BackendGflags.thrift                 |    2 +
 .../java/org/apache/impala/analysis/UnionStmt.java |   20 -
 .../impala/catalog/CatalogServiceCatalog.java      |  184 ++-
 fe/src/main/java/org/apache/impala/catalog/Db.java |   32 +
 .../main/java/org/apache/impala/catalog/Table.java |   37 +-
 .../org/apache/impala/catalog/TableLoader.java     |   77 +-
 .../apache/impala/catalog/events/EventFactory.java |    6 +-
 .../impala/catalog/events/MetastoreEvents.java     |  400 +++++-
 .../catalog/events/MetastoreEventsProcessor.java   |  169 ++-
 .../impala/catalog/events/NoOpEventProcessor.java  |    2 +-
 .../metastore/CatalogMetastoreServiceHandler.java  | 1301 +++++++++++++++++++-
 .../impala/catalog/metastore/HmsApiNameEnum.java   |   42 +-
 .../catalog/metastore/MetastoreServiceHandler.java |  204 ++-
 .../org/apache/impala/service/BackendConfig.java   |   14 +
 .../apache/impala/service/CatalogOpExecutor.java   |  142 ++-
 .../java/org/apache/impala/service/JniCatalog.java |    6 +
 .../apache/impala/catalog/AlterDatabaseTest.java   |    8 +-
 .../impala/catalog/MetastoreApiTestUtils.java      |  169 +++
 .../catalog/events/EventsProcessorStressTest.java  |    6 +-
 .../events/MetastoreEventsProcessorTest.java       |  333 +++--
 .../SynchronousHMSEventProcessorForTests.java      |   13 +-
 .../metastore/AbstractCatalogMetastoreTest.java    |    6 +-
 .../CatalogHmsSyncToLatestEventIdTest.java         |  777 ++++++++++++
 .../impala/testutil/CatalogServiceTestCatalog.java |   34 +-
 testdata/bin/generate-schema-statements.py         |    2 -
 .../functional-query/queries/QueryTest/union.test  |  129 ++
 tests/custom_cluster/test_metastore_service.py     |   15 +-
 30 files changed, 4076 insertions(+), 531 deletions(-)
 create mode 100644 fe/src/test/java/org/apache/impala/catalog/MetastoreApiTestUtils.java
 create mode 100644 fe/src/test/java/org/apache/impala/catalog/metastore/CatalogHmsSyncToLatestEventIdTest.java

[impala] 03/04: IMPALA-10956: datasketches UDFs: memory leak and merge overhead

Posted by st...@apache.org.
This is an automated email from the ASF dual-hosted git repository.

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

commit 151dce86ca8810252c0331cb321b6e77e09327cb
Author: AlexanderSaydakov <al...@apache.org>
AuthorDate: Fri Sep 24 14:05:43 2021 -0700

    IMPALA-10956: datasketches UDFs: memory leak and merge overhead
    
    - call destructors of sketch and union objects
    - avoid overhead of constructing union and getting result from it every time
    
    Change-Id: I8dd0e6736f4266f74f5f265f58d40a4e4707287f
    Reviewed-on: http://gerrit.cloudera.org:8080/17869
    Reviewed-by: Impala Public Jenkins <im...@cloudera.com>
    Tested-by: Impala Public Jenkins <im...@cloudera.com>
---
 be/src/exprs/aggregate-functions-ir.cc | 468 +++++++++++++++++++--------------
 1 file changed, 273 insertions(+), 195 deletions(-)

diff --git a/be/src/exprs/aggregate-functions-ir.cc b/be/src/exprs/aggregate-functions-ir.cc
index 45998e4..a658e39 100644
--- a/be/src/exprs/aggregate-functions-ir.cc
+++ b/be/src/exprs/aggregate-functions-ir.cc
@@ -1674,18 +1674,20 @@ StringVal SerializeDsThetaIntersection(
   return StringVal::null();
 }
 
+// This is for functions with different state during update and merge.
+enum agg_phase { UPDATE, MERGE };
+using agg_state = std::pair<agg_phase, void*>;
+
 void AggregateFunctions::DsHllInit(FunctionContext* ctx, StringVal* dst) {
-  AllocBuffer(ctx, dst, sizeof(datasketches::hll_sketch));
+  AllocBuffer(ctx, dst, sizeof(agg_state));
   if (UNLIKELY(dst->is_null)) {
     DCHECK(!ctx->impl()->state()->GetQueryStatus().ok());
     return;
   }
-  // Note, that hll_sketch will always have the same size regardless of the amount of data
-  // it keeps track. This is because it's a wrapper class that holds all the inserted data
-  // on heap. Here, we put only the wrapper class into a StringVal.
-  datasketches::hll_sketch* sketch_ptr =
-      reinterpret_cast<datasketches::hll_sketch*>(dst->ptr);
-  *sketch_ptr = datasketches::hll_sketch(DS_SKETCH_CONFIG, DS_HLL_TYPE);
+  auto agg_state_ptr = reinterpret_cast<agg_state*>(dst->ptr);
+  agg_state_ptr->first = agg_phase::UPDATE;
+  agg_state_ptr->second = new (ctx->Allocate<datasketches::hll_sketch>())
+      datasketches::hll_sketch(DS_SKETCH_CONFIG, DS_HLL_TYPE);
 }
 
 template <typename T>
@@ -1693,9 +1695,10 @@ void AggregateFunctions::DsHllUpdate(FunctionContext* ctx, const T& src,
     StringVal* dst) {
   if (src.is_null) return;
   DCHECK(!dst->is_null);
-  DCHECK_EQ(dst->len, sizeof(datasketches::hll_sketch));
-  datasketches::hll_sketch* sketch_ptr =
-      reinterpret_cast<datasketches::hll_sketch*>(dst->ptr);
+  DCHECK_EQ(dst->len, sizeof(agg_state));
+  auto agg_state_ptr = reinterpret_cast<agg_state*>(dst->ptr);
+  DCHECK_EQ(agg_state_ptr->first, agg_phase::UPDATE);
+  auto sketch_ptr = reinterpret_cast<datasketches::hll_sketch*>(agg_state_ptr->second);
   sketch_ptr->update(src.val);
 }
 
@@ -1705,19 +1708,29 @@ void AggregateFunctions::DsHllUpdate(
     FunctionContext* ctx, const StringVal& src, StringVal* dst) {
   if (src.is_null || src.len == 0) return;
   DCHECK(!dst->is_null);
-  DCHECK_EQ(dst->len, sizeof(datasketches::hll_sketch));
-  datasketches::hll_sketch* sketch_ptr =
-      reinterpret_cast<datasketches::hll_sketch*>(dst->ptr);
+  DCHECK_EQ(dst->len, sizeof(agg_state));
+  auto agg_state_ptr = reinterpret_cast<agg_state*>(dst->ptr);
+  DCHECK_EQ(agg_state_ptr->first, agg_phase::UPDATE);
+  auto sketch_ptr = reinterpret_cast<datasketches::hll_sketch*>(agg_state_ptr->second);
   sketch_ptr->update(reinterpret_cast<char*>(src.ptr), src.len);
 }
 
 StringVal AggregateFunctions::DsHllSerialize(FunctionContext* ctx,
     const StringVal& src) {
   DCHECK(!src.is_null);
-  DCHECK_EQ(src.len, sizeof(datasketches::hll_sketch));
-  datasketches::hll_sketch* sketch_ptr =
-      reinterpret_cast<datasketches::hll_sketch*>(src.ptr);
-  StringVal dst = SerializeCompactDsHllSketch(ctx, *sketch_ptr);
+  DCHECK_EQ(src.len, sizeof(agg_state));
+  StringVal dst;
+  auto agg_state_ptr = reinterpret_cast<agg_state*>(src.ptr);
+  if (agg_state_ptr->first == agg_phase::UPDATE) { // the agg state is a sketch
+    auto sketch_ptr = reinterpret_cast<datasketches::hll_sketch*>(agg_state_ptr->second);
+    dst = SerializeCompactDsHllSketch(ctx, *sketch_ptr);
+    sketch_ptr->~hll_sketch_alloc();
+  } else { // the agg state is a union
+    auto union_ptr = reinterpret_cast<datasketches::hll_union*>(agg_state_ptr->second);
+    dst = SerializeDsHllUnion(ctx, *union_ptr);
+    union_ptr->~hll_union_alloc();
+  }
+  ctx->Free(reinterpret_cast<uint8_t*>(agg_state_ptr->second));
   ctx->Free(src.ptr);
   return dst;
 }
@@ -1726,26 +1739,44 @@ void AggregateFunctions::DsHllMerge(
     FunctionContext* ctx, const StringVal& src, StringVal* dst) {
   DCHECK(!src.is_null);
   DCHECK(!dst->is_null);
-  DCHECK_EQ(dst->len, sizeof(datasketches::hll_sketch));
-  datasketches::hll_sketch src_sketch =
-      datasketches::hll_sketch::deserialize((void*)src.ptr, src.len);
-
-  datasketches::hll_sketch* dst_sketch_ptr =
-      reinterpret_cast<datasketches::hll_sketch*>(dst->ptr);
+  DCHECK_EQ(dst->len, sizeof(agg_state));
+  auto agg_state_ptr = reinterpret_cast<agg_state*>(dst->ptr);
+  if (agg_state_ptr->first == agg_phase::MERGE) { // was already switched to union
+    auto dst_union_ptr =
+        reinterpret_cast<datasketches::hll_union*>(agg_state_ptr->second);
+    dst_union_ptr->update(datasketches::hll_sketch::deserialize(src.ptr, src.len));
+  } else { // must be the first call. the state is still a sketch
+    auto dst_sketch_ptr =
+        reinterpret_cast<datasketches::hll_sketch*>(agg_state_ptr->second);
 
-  datasketches::hll_union union_sketch(DS_SKETCH_CONFIG);
-  union_sketch.update(src_sketch);
-  union_sketch.update(*dst_sketch_ptr);
+    datasketches::hll_union u(DS_SKETCH_CONFIG);
+    u.update(*dst_sketch_ptr);
+    u.update(datasketches::hll_sketch::deserialize(src.ptr, src.len));
 
-  *dst_sketch_ptr = union_sketch.get_result(DS_HLL_TYPE);
+    // swich to union
+    dst_sketch_ptr->~hll_sketch_alloc();
+    ctx->Free(reinterpret_cast<uint8_t*>(agg_state_ptr->second));
+    agg_state_ptr->second = new (ctx->Allocate<datasketches::hll_union>())
+        datasketches::hll_union(std::move(u));
+    agg_state_ptr->first = agg_phase::MERGE;
+  }
 }
 
 BigIntVal AggregateFunctions::DsHllFinalize(FunctionContext* ctx, const StringVal& src) {
   DCHECK(!src.is_null);
-  DCHECK_EQ(src.len, sizeof(datasketches::hll_sketch));
-  datasketches::hll_sketch* sketch_ptr =
-      reinterpret_cast<datasketches::hll_sketch*>(src.ptr);
-  BigIntVal estimate = sketch_ptr->get_estimate();
+  DCHECK_EQ(src.len, sizeof(agg_state));
+  BigIntVal estimate;
+  auto agg_state_ptr = reinterpret_cast<agg_state*>(src.ptr);
+  if (agg_state_ptr->first == agg_phase::UPDATE) { // the agg state is a sketch
+    auto sketch_ptr = reinterpret_cast<datasketches::hll_sketch*>(agg_state_ptr->second);
+    estimate = sketch_ptr->get_estimate();
+    sketch_ptr->~hll_sketch_alloc();
+  } else { // the agg state is a union
+    auto union_ptr = reinterpret_cast<datasketches::hll_union*>(agg_state_ptr->second);
+    estimate = union_ptr->get_result().get_estimate();
+    union_ptr->~hll_union_alloc();
+  }
+  ctx->Free(reinterpret_cast<uint8_t*>(agg_state_ptr->second));
   ctx->Free(src.ptr);
   return (estimate == 0) ? BigIntVal::null() : estimate;
 }
@@ -1753,15 +1784,26 @@ BigIntVal AggregateFunctions::DsHllFinalize(FunctionContext* ctx, const StringVa
 StringVal AggregateFunctions::DsHllFinalizeSketch(FunctionContext* ctx,
     const StringVal& src) {
   DCHECK(!src.is_null);
-  DCHECK_EQ(src.len, sizeof(datasketches::hll_sketch));
-  datasketches::hll_sketch* sketch_ptr =
-      reinterpret_cast<datasketches::hll_sketch*>(src.ptr);
-  StringVal result_str = StringVal::null();
-  if (sketch_ptr->get_estimate() > 0.0) {
-    result_str = SerializeCompactDsHllSketch(ctx, *sketch_ptr);
+  DCHECK_EQ(src.len, sizeof(agg_state));
+  StringVal result = StringVal::null();
+  auto agg_state_ptr = reinterpret_cast<agg_state*>(src.ptr);
+  if (agg_state_ptr->first == agg_phase::UPDATE) { // the agg state is a sketch
+    auto sketch_ptr = reinterpret_cast<datasketches::hll_sketch*>(agg_state_ptr->second);
+    if (!sketch_ptr->is_empty()) {
+      result = SerializeCompactDsHllSketch(ctx, *sketch_ptr);
+    }
+    sketch_ptr->~hll_sketch_alloc();
+  } else { // the agg state is a union
+    auto union_ptr = reinterpret_cast<datasketches::hll_union*>(agg_state_ptr->second);
+    auto sketch = union_ptr->get_result(DS_HLL_TYPE);
+    if (!sketch.is_empty()) {
+      result = SerializeCompactDsHllSketch(ctx, sketch);
+    }
+    union_ptr->~hll_union_alloc();
   }
+  ctx->Free(reinterpret_cast<uint8_t*>(agg_state_ptr->second));
   ctx->Free(src.ptr);
-  return result_str;
+  return result;
 }
 
 void AggregateFunctions::DsHllUnionInit(FunctionContext* ctx, StringVal* slot) {
@@ -1770,9 +1812,8 @@ void AggregateFunctions::DsHllUnionInit(FunctionContext* ctx, StringVal* slot) {
     DCHECK(!ctx->impl()->state()->GetQueryStatus().ok());
     return;
   }
-  datasketches::hll_union* union_ptr =
-      reinterpret_cast<datasketches::hll_union*>(slot->ptr);
-  *union_ptr = datasketches::hll_union(DS_SKETCH_CONFIG);
+  auto union_ptr = reinterpret_cast<datasketches::hll_union*>(slot->ptr);
+  new (union_ptr) datasketches::hll_union(DS_SKETCH_CONFIG);
 }
 
 void AggregateFunctions::DsHllUnionUpdate(FunctionContext* ctx, const StringVal& src,
@@ -1795,6 +1836,7 @@ StringVal AggregateFunctions::DsHllUnionSerialize(FunctionContext* ctx,
   datasketches::hll_union* union_ptr =
       reinterpret_cast<datasketches::hll_union*>(src.ptr);
   StringVal dst = SerializeDsHllUnion(ctx, *union_ptr);
+  union_ptr->~hll_union_alloc();
   ctx->Free(src.ptr);
   return dst;
 }
@@ -1817,39 +1859,37 @@ StringVal AggregateFunctions::DsHllUnionFinalize(FunctionContext* ctx,
     const StringVal& src) {
   DCHECK(!src.is_null);
   DCHECK_EQ(src.len, sizeof(datasketches::hll_union));
-  datasketches::hll_union* union_ptr =
-      reinterpret_cast<datasketches::hll_union*>(src.ptr);
-  datasketches::hll_sketch sketch = union_ptr->get_result(DS_HLL_TYPE);
-  if (sketch.get_estimate() == 0.0) {
-    ctx->Free(src.ptr);
-    return StringVal::null();
+  auto union_ptr = reinterpret_cast<datasketches::hll_union*>(src.ptr);
+  auto sketch = union_ptr->get_result(DS_HLL_TYPE);
+  StringVal result = StringVal::null();
+  if (!sketch.is_empty()) {
+    result = SerializeCompactDsHllSketch(ctx, sketch);
   }
-  StringVal result = SerializeCompactDsHllSketch(ctx, sketch);
+  union_ptr->~hll_union_alloc();
   ctx->Free(src.ptr);
   return result;
 }
 
 void AggregateFunctions::DsCpcInit(FunctionContext* ctx, StringVal* dst) {
-  AllocBuffer(ctx, dst, sizeof(datasketches::cpc_sketch));
+  AllocBuffer(ctx, dst, sizeof(agg_state));
   if (UNLIKELY(dst->is_null)) {
     DCHECK(!ctx->impl()->state()->GetQueryStatus().ok());
     return;
   }
-  // Note, that cpc_sketch will always have the same size regardless of the amount of data
-  // it keeps track. This is because it's a wrapper class that holds all the inserted data
-  // on heap. Here, we put only the wrapper class into a StringVal.
-  datasketches::cpc_sketch* sketch_ptr =
-      reinterpret_cast<datasketches::cpc_sketch*>(dst->ptr);
-  *sketch_ptr = datasketches::cpc_sketch(DS_CPC_SKETCH_CONFIG);
+  auto agg_state_ptr = reinterpret_cast<agg_state*>(dst->ptr);
+  agg_state_ptr->first = agg_phase::UPDATE;
+  agg_state_ptr->second = new (ctx->Allocate<datasketches::cpc_sketch>())
+      datasketches::cpc_sketch(DS_CPC_SKETCH_CONFIG);
 }
 
 template <typename T>
 void AggregateFunctions::DsCpcUpdate(FunctionContext* ctx, const T& src, StringVal* dst) {
   if (src.is_null) return;
   DCHECK(!dst->is_null);
-  DCHECK_EQ(dst->len, sizeof(datasketches::cpc_sketch));
-  datasketches::cpc_sketch* sketch_ptr =
-      reinterpret_cast<datasketches::cpc_sketch*>(dst->ptr);
+  DCHECK_EQ(dst->len, sizeof(agg_state));
+  auto agg_state_ptr = reinterpret_cast<agg_state*>(dst->ptr);
+  DCHECK_EQ(agg_state_ptr->first, agg_phase::UPDATE);
+  auto sketch_ptr = reinterpret_cast<datasketches::cpc_sketch*>(agg_state_ptr->second);
   sketch_ptr->update(src.val);
 }
 
@@ -1859,18 +1899,28 @@ void AggregateFunctions::DsCpcUpdate(
     FunctionContext* ctx, const StringVal& src, StringVal* dst) {
   if (src.is_null || src.len == 0) return;
   DCHECK(!dst->is_null);
-  DCHECK_EQ(dst->len, sizeof(datasketches::cpc_sketch));
-  datasketches::cpc_sketch* sketch_ptr =
-      reinterpret_cast<datasketches::cpc_sketch*>(dst->ptr);
+  DCHECK_EQ(dst->len, sizeof(agg_state));
+  auto agg_state_ptr = reinterpret_cast<agg_state*>(dst->ptr);
+  DCHECK(agg_state_ptr->first == agg_phase::UPDATE);
+  auto sketch_ptr = reinterpret_cast<datasketches::cpc_sketch*>(agg_state_ptr->second);
   sketch_ptr->update(reinterpret_cast<char*>(src.ptr), src.len);
 }
 
 StringVal AggregateFunctions::DsCpcSerialize(FunctionContext* ctx, const StringVal& src) {
   DCHECK(!src.is_null);
-  DCHECK_EQ(src.len, sizeof(datasketches::cpc_sketch));
-  datasketches::cpc_sketch* sketch_ptr =
-      reinterpret_cast<datasketches::cpc_sketch*>(src.ptr);
-  StringVal dst = SerializeDsSketch(ctx, *sketch_ptr);
+  DCHECK_EQ(src.len, sizeof(agg_state));
+  StringVal dst;
+  auto agg_state_ptr = reinterpret_cast<agg_state*>(src.ptr);
+  if (agg_state_ptr->first == agg_phase::UPDATE) { // the agg state is a sketch
+    auto sketch_ptr = reinterpret_cast<datasketches::cpc_sketch*>(agg_state_ptr->second);
+    dst = SerializeDsSketch(ctx, *sketch_ptr);
+    sketch_ptr->~cpc_sketch_alloc();
+  } else { // the agg state is a union
+    auto union_ptr = reinterpret_cast<datasketches::cpc_union*>(agg_state_ptr->second);
+    dst = SerializeDsCpcUnion(ctx, *union_ptr);
+    union_ptr->~cpc_union_alloc();
+  }
+  ctx->Free(reinterpret_cast<uint8_t*>(agg_state_ptr->second));
   ctx->Free(src.ptr);
   return dst;
 }
@@ -1879,27 +1929,49 @@ void AggregateFunctions::DsCpcMerge(
     FunctionContext* ctx, const StringVal& src, StringVal* dst) {
   DCHECK(!src.is_null);
   DCHECK(!dst->is_null);
-  DCHECK_EQ(dst->len, sizeof(datasketches::cpc_sketch));
+  DCHECK_EQ(dst->len, sizeof(agg_state));
+  auto agg_state_ptr = reinterpret_cast<agg_state*>(dst->ptr);
+  if (agg_state_ptr->first == agg_phase::MERGE) { // was already switched to union
+    auto dst_union_ptr =
+        reinterpret_cast<datasketches::cpc_union*>(agg_state_ptr->second);
+    dst_union_ptr->update(datasketches::cpc_sketch::deserialize(src.ptr, src.len));
+  } else { // must be the first call. the state is still a sketch
+    auto dst_sketch_ptr =
+        reinterpret_cast<datasketches::cpc_sketch*>(agg_state_ptr->second);
+
+    datasketches::cpc_union u(DS_CPC_SKETCH_CONFIG);
+    u.update(*dst_sketch_ptr);
+    try {
+      u.update(datasketches::cpc_sketch::deserialize(src.ptr, src.len));
+    } catch (const std::exception& e) {
+      LogSketchDeserializationError(ctx, e);
+      return;
+    }
 
-  datasketches::cpc_union union_sketch(DS_CPC_SKETCH_CONFIG);
-  try {
-    union_sketch.update(datasketches::cpc_sketch::deserialize(src.ptr, src.len));
-  } catch (const std::exception& e) {
-    LogSketchDeserializationError(ctx, e);
-    return;
+    // switch to union
+    dst_sketch_ptr->~cpc_sketch_alloc();
+    ctx->Free(reinterpret_cast<uint8_t*>(agg_state_ptr->second));
+    agg_state_ptr->second = new (ctx->Allocate<datasketches::cpc_union>())
+        datasketches::cpc_union(std::move(u));
+    agg_state_ptr->first = agg_phase::MERGE;
   }
-  datasketches::cpc_sketch* dst_sketch_ptr =
-      reinterpret_cast<datasketches::cpc_sketch*>(dst->ptr);
-  union_sketch.update(*dst_sketch_ptr);
-  *dst_sketch_ptr = union_sketch.get_result();
 }
 
 BigIntVal AggregateFunctions::DsCpcFinalize(FunctionContext* ctx, const StringVal& src) {
   DCHECK(!src.is_null);
-  DCHECK_EQ(src.len, sizeof(datasketches::cpc_sketch));
-  datasketches::cpc_sketch* sketch_ptr =
-      reinterpret_cast<datasketches::cpc_sketch*>(src.ptr);
-  BigIntVal estimate = sketch_ptr->get_estimate();
+  DCHECK_EQ(src.len, sizeof(agg_state));
+  BigIntVal estimate;
+  auto agg_state_ptr = reinterpret_cast<agg_state*>(src.ptr);
+  if (agg_state_ptr->first == agg_phase::UPDATE) { // the agg state is a sketch
+    auto sketch_ptr = reinterpret_cast<datasketches::cpc_sketch*>(agg_state_ptr->second);
+    estimate = sketch_ptr->get_estimate();
+    sketch_ptr->~cpc_sketch_alloc();
+  } else { // the agg state is a union
+    auto union_ptr = reinterpret_cast<datasketches::cpc_union*>(agg_state_ptr->second);
+    estimate = union_ptr->get_result().get_estimate();
+    union_ptr->~cpc_union_alloc();
+  }
+  ctx->Free(reinterpret_cast<uint8_t*>(agg_state_ptr->second));
   ctx->Free(src.ptr);
   return (estimate == 0) ? BigIntVal::null() : estimate;
 }
@@ -1907,15 +1979,26 @@ BigIntVal AggregateFunctions::DsCpcFinalize(FunctionContext* ctx, const StringVa
 StringVal AggregateFunctions::DsCpcFinalizeSketch(
     FunctionContext* ctx, const StringVal& src) {
   DCHECK(!src.is_null);
-  DCHECK_EQ(src.len, sizeof(datasketches::cpc_sketch));
-  datasketches::cpc_sketch* sketch_ptr =
-      reinterpret_cast<datasketches::cpc_sketch*>(src.ptr);
-  StringVal result_str = StringVal::null();
-  if (sketch_ptr->get_estimate() > 0.0) {
-    result_str = SerializeDsSketch(ctx, *sketch_ptr);
+  DCHECK_EQ(src.len, sizeof(agg_state));
+  StringVal result = StringVal::null();
+  auto agg_state_ptr = reinterpret_cast<agg_state*>(src.ptr);
+  if (agg_state_ptr->first == agg_phase::UPDATE) { // the agg state is a sketch
+    auto sketch_ptr = reinterpret_cast<datasketches::cpc_sketch*>(agg_state_ptr->second);
+    if (!sketch_ptr->is_empty()) {
+      result = SerializeDsSketch(ctx, *sketch_ptr);
+    }
+    sketch_ptr->~cpc_sketch_alloc();
+  } else { // the agg state is a union
+    auto union_ptr = reinterpret_cast<datasketches::cpc_union*>(agg_state_ptr->second);
+    auto sketch = union_ptr->get_result();
+    if (!sketch.is_empty()) {
+      result = SerializeDsSketch(ctx, sketch);
+    }
+    union_ptr->~cpc_union_alloc();
   }
+  ctx->Free(reinterpret_cast<uint8_t*>(agg_state_ptr->second));
   ctx->Free(src.ptr);
-  return result_str;
+  return result;
 }
 
 void AggregateFunctions::DsCpcUnionInit(FunctionContext* ctx, StringVal* slot) {
@@ -1924,9 +2007,8 @@ void AggregateFunctions::DsCpcUnionInit(FunctionContext* ctx, StringVal* slot) {
     DCHECK(!ctx->impl()->state()->GetQueryStatus().ok());
     return;
   }
-  datasketches::cpc_union* union_ptr =
-      reinterpret_cast<datasketches::cpc_union*>(slot->ptr);
-  *union_ptr = datasketches::cpc_union(DS_SKETCH_CONFIG);
+  auto union_ptr = reinterpret_cast<datasketches::cpc_union*>(slot->ptr);
+  new (union_ptr) datasketches::cpc_union(DS_CPC_SKETCH_CONFIG);
 }
 
 void AggregateFunctions::DsCpcUnionUpdate(
@@ -1946,9 +2028,9 @@ StringVal AggregateFunctions::DsCpcUnionSerialize(
     FunctionContext* ctx, const StringVal& src) {
   DCHECK(!src.is_null);
   DCHECK_EQ(src.len, sizeof(datasketches::cpc_union));
-  datasketches::cpc_union* union_ptr =
-      reinterpret_cast<datasketches::cpc_union*>(src.ptr);
+  auto union_ptr = reinterpret_cast<datasketches::cpc_union*>(src.ptr);
   StringVal dst = SerializeDsCpcUnion(ctx, *union_ptr);
+  union_ptr->~cpc_union_alloc();
   ctx->Free(src.ptr);
   return dst;
 }
@@ -1971,32 +2053,28 @@ StringVal AggregateFunctions::DsCpcUnionFinalize(
     FunctionContext* ctx, const StringVal& src) {
   DCHECK(!src.is_null);
   DCHECK_EQ(src.len, sizeof(datasketches::cpc_union));
-  datasketches::cpc_union* union_ptr =
-      reinterpret_cast<datasketches::cpc_union*>(src.ptr);
-  datasketches::cpc_sketch sketch = union_ptr->get_result();
-  if (sketch.get_estimate() == 0.0) {
-    ctx->Free(src.ptr);
-    return StringVal::null();
+  auto union_ptr = reinterpret_cast<datasketches::cpc_union*>(src.ptr);
+  auto sketch = union_ptr->get_result();
+  StringVal result = StringVal::null();
+  if (!sketch.is_empty()) {
+    result = SerializeDsSketch(ctx, sketch);
   }
-  StringVal result = SerializeDsSketch(ctx, sketch);
+  union_ptr->~cpc_union_alloc();
   ctx->Free(src.ptr);
   return result;
 }
 
 void AggregateFunctions::DsThetaInit(FunctionContext* ctx, StringVal* dst) {
-  AllocBuffer(ctx, dst, sizeof(datasketches::update_theta_sketch));
+  AllocBuffer(ctx, dst, sizeof(agg_state));
   if (UNLIKELY(dst->is_null)) {
     DCHECK(!ctx->impl()->state()->GetQueryStatus().ok());
     return;
   }
-  // Note, that update_theta_sketch will always have the same size regardless of the
-  // amount of data it keeps track. This is because it's a wrapper class that holds all
-  // the inserted data on heap. Here, we put only the wrapper class into a StringVal.
-  datasketches::update_theta_sketch* sketch_ptr =
-      reinterpret_cast<datasketches::update_theta_sketch*>(dst->ptr);
-  datasketches::update_theta_sketch sketch =
-      datasketches::update_theta_sketch::builder().build();
-  std::uninitialized_fill_n(sketch_ptr, 1, sketch);
+  auto agg_state_ptr = reinterpret_cast<agg_state*>(dst->ptr);
+  agg_state_ptr->first = agg_phase::UPDATE;
+  agg_state_ptr->second = new (ctx->Allocate<datasketches::update_theta_sketch>())
+      datasketches::update_theta_sketch(
+          datasketches::update_theta_sketch::builder().build());
 }
 
 template <typename T>
@@ -2004,9 +2082,11 @@ void AggregateFunctions::DsThetaUpdate(
     FunctionContext* ctx, const T& src, StringVal* dst) {
   if (src.is_null) return;
   DCHECK(!dst->is_null);
-  DCHECK_EQ(dst->len, sizeof(datasketches::update_theta_sketch));
-  datasketches::update_theta_sketch* sketch_ptr =
-      reinterpret_cast<datasketches::update_theta_sketch*>(dst->ptr);
+  DCHECK_EQ(dst->len, sizeof(agg_state));
+  auto agg_state_ptr = reinterpret_cast<agg_state*>(dst->ptr);
+  DCHECK_EQ(agg_state_ptr->first, agg_phase::UPDATE);
+  auto sketch_ptr =
+      reinterpret_cast<datasketches::update_theta_sketch*>(agg_state_ptr->second);
   sketch_ptr->update(src.val);
 }
 
@@ -2016,25 +2096,31 @@ void AggregateFunctions::DsThetaUpdate(
     FunctionContext* ctx, const StringVal& src, StringVal* dst) {
   if (src.is_null || src.len == 0) return;
   DCHECK(!dst->is_null);
-  DCHECK_EQ(dst->len, sizeof(datasketches::update_theta_sketch));
-  datasketches::update_theta_sketch* sketch_ptr =
-      reinterpret_cast<datasketches::update_theta_sketch*>(dst->ptr);
+  DCHECK_EQ(dst->len, sizeof(agg_state));
+  auto agg_state_ptr = reinterpret_cast<agg_state*>(dst->ptr);
+  DCHECK_EQ(agg_state_ptr->first, agg_phase::UPDATE);
+  auto sketch_ptr =
+      reinterpret_cast<datasketches::update_theta_sketch*>(agg_state_ptr->second);
   sketch_ptr->update(reinterpret_cast<char*>(src.ptr), src.len);
 }
 
 StringVal AggregateFunctions::DsThetaSerialize(
     FunctionContext* ctx, const StringVal& src) {
   DCHECK(!src.is_null);
-  DCHECK(src.len == sizeof(datasketches::update_theta_sketch)
-      || src.len == sizeof(datasketches::theta_union));
+  DCHECK_EQ(src.len, sizeof(agg_state));
   StringVal dst;
-  if (src.len == sizeof(datasketches::update_theta_sketch)) {
-    auto sketch_ptr = reinterpret_cast<datasketches::update_theta_sketch*>(src.ptr);
+  auto agg_state_ptr = reinterpret_cast<agg_state*>(src.ptr);
+  if (agg_state_ptr->first == agg_phase::UPDATE) { // the agg state is a sketch
+    auto sketch_ptr =
+        reinterpret_cast<datasketches::update_theta_sketch*>(agg_state_ptr->second);
     dst = SerializeDsSketch(ctx, sketch_ptr->compact());
-  } else {
-    auto union_ptr = reinterpret_cast<datasketches::theta_union*>(src.ptr);
+    sketch_ptr->~update_theta_sketch_alloc();
+  } else { // the agg state is a union
+    auto union_ptr = reinterpret_cast<datasketches::theta_union*>(agg_state_ptr->second);
     dst = SerializeDsThetaUnion(ctx, *union_ptr);
+    union_ptr->~theta_union_alloc();
   }
+  ctx->Free(reinterpret_cast<uint8_t*>(agg_state_ptr->second));
   ctx->Free(src.ptr);
   return dst;
 }
@@ -2043,12 +2129,13 @@ void AggregateFunctions::DsThetaMerge(
     FunctionContext* ctx, const StringVal& src, StringVal* dst) {
   DCHECK(!src.is_null);
   DCHECK(!dst->is_null);
-  DCHECK(dst->len == sizeof(datasketches::update_theta_sketch)
-      or dst->len == sizeof(datasketches::theta_union));
+  DCHECK_EQ(dst->len, sizeof(agg_state));
+  auto agg_state_ptr = reinterpret_cast<agg_state*>(dst->ptr);
 
   // Note, 'src' is a serialized compact_theta_sketch.
-  if (dst->len == sizeof(datasketches::theta_union)) {
-    auto dst_union_ptr = reinterpret_cast<datasketches::theta_union*>(dst->ptr);
+  if (agg_state_ptr->first == agg_phase::MERGE) { // was already switched to union
+    auto dst_union_ptr =
+        reinterpret_cast<datasketches::theta_union*>(agg_state_ptr->second);
     try {
       dst_union_ptr->update(datasketches::compact_theta_sketch::deserialize(src.ptr,
           src.len));
@@ -2056,47 +2143,45 @@ void AggregateFunctions::DsThetaMerge(
       LogSketchDeserializationError(ctx, e);
       return;
     }
-  } else if (dst->len == sizeof(datasketches::update_theta_sketch)) {
-    auto dst_sketch_ptr = reinterpret_cast<datasketches::update_theta_sketch*>(dst->ptr);
+  } else { // must be the first call. the state is still a sketch
+    auto dst_sketch_ptr =
+        reinterpret_cast<datasketches::update_theta_sketch*>(agg_state_ptr->second);
 
-    datasketches::theta_union union_sketch = datasketches::theta_union::builder().build();
-    union_sketch.update(*dst_sketch_ptr);
+    auto u = datasketches::theta_union::builder().build();
+    u.update(*dst_sketch_ptr);
     try {
-      union_sketch.update(datasketches::compact_theta_sketch::deserialize(src.ptr,
-          src.len));
+      u.update(datasketches::compact_theta_sketch::deserialize(src.ptr, src.len));
     } catch (const std::exception& e) {
       LogSketchDeserializationError(ctx, e);
       return;
     }
 
-    // theta_union.get_result() returns a compact sketch, does not support updating, and
-    // is inconsistent with the initial underlying type of dst. This is different from
-    // the HLL sketch. Here use theta_union as the underlying type of dst.
-    ctx->Free(dst->ptr);
-    AllocBuffer(ctx, dst, sizeof(datasketches::theta_union));
-    if (UNLIKELY(dst->is_null)) {
-      DCHECK(!ctx->impl()->state()->GetQueryStatus().ok());
-      return;
-    }
-    datasketches::theta_union* union_ptr =
-        reinterpret_cast<datasketches::theta_union*>(dst->ptr);
-    std::uninitialized_fill_n(union_ptr, 1, union_sketch);
+    // switch to union
+    dst_sketch_ptr->~update_theta_sketch_alloc();
+    ctx->Free(reinterpret_cast<uint8_t*>(agg_state_ptr->second));
+    agg_state_ptr->second = new (ctx->Allocate<datasketches::theta_union>())
+        datasketches::theta_union(std::move(u));
+    agg_state_ptr->first = agg_phase::MERGE;
   }
 }
 
 BigIntVal AggregateFunctions::DsThetaFinalize(
     FunctionContext* ctx, const StringVal& src) {
   DCHECK(!src.is_null);
-  DCHECK(src.len == sizeof(datasketches::update_theta_sketch)
-      or src.len == sizeof(datasketches::theta_union));
+  DCHECK_EQ(src.len, sizeof(agg_state));
   BigIntVal estimate;
-  if (src.len == sizeof(datasketches::update_theta_sketch)) {
-    auto sketch_ptr = reinterpret_cast<datasketches::update_theta_sketch*>(src.ptr);
+  auto agg_state_ptr = reinterpret_cast<agg_state*>(src.ptr);
+  if (agg_state_ptr->first == agg_phase::UPDATE) { // the agg state is a sketch
+    auto sketch_ptr =
+        reinterpret_cast<datasketches::update_theta_sketch*>(agg_state_ptr->second);
     estimate = sketch_ptr->get_estimate();
-  } else {
-    auto union_ptr = reinterpret_cast<datasketches::theta_union*>(src.ptr);
+    sketch_ptr->~update_theta_sketch_alloc();
+  } else { // the agg state is a union
+    auto union_ptr = reinterpret_cast<datasketches::theta_union*>(agg_state_ptr->second);
     estimate = union_ptr->get_result().get_estimate();
+    union_ptr->~theta_union_alloc();
   }
+  ctx->Free(reinterpret_cast<uint8_t*>(agg_state_ptr->second));
   ctx->Free(src.ptr);
   return estimate;
 }
@@ -2104,16 +2189,20 @@ BigIntVal AggregateFunctions::DsThetaFinalize(
 StringVal AggregateFunctions::DsThetaFinalizeSketch(
     FunctionContext* ctx, const StringVal& src) {
   DCHECK(!src.is_null);
-  DCHECK(src.len == sizeof(datasketches::update_theta_sketch)
-      or src.len == sizeof(datasketches::theta_union));
+  DCHECK_EQ(src.len, sizeof(agg_state));
   StringVal result;
-  if (src.len == sizeof(datasketches::update_theta_sketch)) {
-    auto sketch_ptr = reinterpret_cast<datasketches::update_theta_sketch*>(src.ptr);
+  auto agg_state_ptr = reinterpret_cast<agg_state*>(src.ptr);
+  if (agg_state_ptr->first == agg_phase::UPDATE) { // the agg state is a sketch
+    auto sketch_ptr =
+        reinterpret_cast<datasketches::update_theta_sketch*>(agg_state_ptr->second);
     result = SerializeDsSketch(ctx, sketch_ptr->compact());
-  } else {
-    auto union_ptr = reinterpret_cast<datasketches::theta_union*>(src.ptr);
+    sketch_ptr->~update_theta_sketch_alloc();
+  } else { // the agg state is a union
+    auto union_ptr = reinterpret_cast<datasketches::theta_union*>(agg_state_ptr->second);
     result = SerializeDsThetaUnion(ctx, *union_ptr);
+    union_ptr->~theta_union_alloc();
   }
+  ctx->Free(reinterpret_cast<uint8_t*>(agg_state_ptr->second));
   ctx->Free(src.ptr);
   return result;
 }
@@ -2124,10 +2213,8 @@ void AggregateFunctions::DsThetaUnionInit(FunctionContext* ctx, StringVal* dst)
     DCHECK(!ctx->impl()->state()->GetQueryStatus().ok());
     return;
   }
-  datasketches::theta_union* union_ptr =
-      reinterpret_cast<datasketches::theta_union*>(dst->ptr);
-  datasketches::theta_union union_sketch = datasketches::theta_union::builder().build();
-  std::uninitialized_fill_n(union_ptr, 1, union_sketch);
+  auto union_ptr = reinterpret_cast<datasketches::theta_union*>(dst->ptr);
+  new (union_ptr) datasketches::theta_union(datasketches::theta_union::builder().build());
 }
 
 void AggregateFunctions::DsThetaUnionUpdate(
@@ -2147,9 +2234,9 @@ StringVal AggregateFunctions::DsThetaUnionSerialize(
     FunctionContext* ctx, const StringVal& src) {
   DCHECK(!src.is_null);
   DCHECK_EQ(src.len, sizeof(datasketches::theta_union));
-  datasketches::theta_union* union_ptr =
-      reinterpret_cast<datasketches::theta_union*>(src.ptr);
+  auto union_ptr = reinterpret_cast<datasketches::theta_union*>(src.ptr);
   StringVal dst = SerializeDsThetaUnion(ctx, *union_ptr);
+  union_ptr->~theta_union_alloc();
   ctx->Free(src.ptr);
   return dst;
 }
@@ -2172,14 +2259,13 @@ StringVal AggregateFunctions::DsThetaUnionFinalize(
     FunctionContext* ctx, const StringVal& src) {
   DCHECK(!src.is_null);
   DCHECK_EQ(src.len, sizeof(datasketches::theta_union));
-  datasketches::theta_union* union_ptr =
-      reinterpret_cast<datasketches::theta_union*>(src.ptr);
+  auto union_ptr = reinterpret_cast<datasketches::theta_union*>(src.ptr);
   auto sketch = union_ptr->get_result();
-  if (sketch.is_empty()) {
-    ctx->Free(src.ptr);
-    return StringVal::null();
+  StringVal result = StringVal::null();
+  if (!sketch.is_empty()) {
+    result = SerializeDsSketch(ctx, sketch);
   }
-  StringVal result = SerializeDsSketch(ctx, sketch);
+  union_ptr->~theta_union_alloc();
   ctx->Free(src.ptr);
   return result;
 }
@@ -2190,9 +2276,8 @@ void AggregateFunctions::DsThetaIntersectInit(FunctionContext* ctx, StringVal* s
     DCHECK(!ctx->impl()->state()->GetQueryStatus().ok());
     return;
   }
-  datasketches::theta_intersection* intersection_ptr =
-      reinterpret_cast<datasketches::theta_intersection*>(slot->ptr);
-  *intersection_ptr = datasketches::theta_intersection();
+  auto intersection_ptr = reinterpret_cast<datasketches::theta_intersection*>(slot->ptr);
+  new (intersection_ptr) datasketches::theta_intersection();
 }
 
 void AggregateFunctions::DsThetaIntersectUpdate(
@@ -2212,9 +2297,9 @@ StringVal AggregateFunctions::DsThetaIntersectSerialize(
     FunctionContext* ctx, const StringVal& src) {
   DCHECK(!src.is_null);
   DCHECK_EQ(src.len, sizeof(datasketches::theta_intersection));
-  datasketches::theta_intersection* intersection_ptr =
-      reinterpret_cast<datasketches::theta_intersection*>(src.ptr);
+  auto intersection_ptr = reinterpret_cast<datasketches::theta_intersection*>(src.ptr);
   StringVal dst = SerializeDsThetaIntersection(ctx, *intersection_ptr);
+  intersection_ptr->~theta_intersection_alloc();
   ctx->Free(src.ptr);
   return dst;
 }
@@ -2238,14 +2323,12 @@ StringVal AggregateFunctions::DsThetaIntersectFinalize(
     FunctionContext* ctx, const StringVal& src) {
   DCHECK(!src.is_null);
   DCHECK_EQ(src.len, sizeof(datasketches::theta_intersection));
-  datasketches::theta_intersection* intersection_ptr =
-      reinterpret_cast<datasketches::theta_intersection*>(src.ptr);
-  if (!intersection_ptr->has_result()) {
-    ctx->Free(src.ptr);
-    return StringVal::null();
+  auto intersection_ptr = reinterpret_cast<datasketches::theta_intersection*>(src.ptr);
+  StringVal result = StringVal::null();
+  if (intersection_ptr->has_result()) {
+    result = SerializeDsSketch(ctx, intersection_ptr->get_result());
   }
-  auto sketch = intersection_ptr->get_result();
-  StringVal result = SerializeDsSketch(ctx, sketch);
+  intersection_ptr->~theta_intersection_alloc();
   ctx->Free(src.ptr);
   return result;
 }
@@ -2259,18 +2342,17 @@ void AggregateFunctions::DsKllInitHelper(FunctionContext* ctx, StringVal* slot)
   // Note, that kll_sketch will always have the same size regardless of the amount of
   // data it keeps track of. This is because it's a wrapper class that holds all the
   // inserted data on heap. Here, we put only the wrapper class into a StringVal.
-  datasketches::kll_sketch<float>* sketch_ptr =
-      reinterpret_cast<datasketches::kll_sketch<float>*>(slot->ptr);
-  *sketch_ptr = datasketches::kll_sketch<float>();
+  auto sketch_ptr = reinterpret_cast<datasketches::kll_sketch<float>*>(slot->ptr);
+  new (sketch_ptr) datasketches::kll_sketch<float>();
 }
 
 StringVal AggregateFunctions::DsKllSerializeHelper(FunctionContext* ctx,
     const StringVal& src) {
   DCHECK(!src.is_null);
   DCHECK_EQ(src.len, sizeof(datasketches::kll_sketch<float>));
-  datasketches::kll_sketch<float>* sketch_ptr =
-      reinterpret_cast<datasketches::kll_sketch<float>*>(src.ptr);
+  auto sketch_ptr = reinterpret_cast<datasketches::kll_sketch<float>*>(src.ptr);
   StringVal dst = SerializeDsSketch(ctx, *sketch_ptr);
+  sketch_ptr->~kll_sketch();
   ctx->Free(src.ptr);
   return dst;
 }
@@ -2281,8 +2363,7 @@ void AggregateFunctions::DsKllMergeHelper(FunctionContext* ctx, const StringVal&
   DCHECK(!dst->is_null);
   DCHECK_EQ(dst->len, sizeof(datasketches::kll_sketch<float>));
 
-  datasketches::kll_sketch<float>* dst_sketch_ptr =
-      reinterpret_cast<datasketches::kll_sketch<float>*>(dst->ptr);
+  auto dst_sketch_ptr = reinterpret_cast<datasketches::kll_sketch<float>*>(dst->ptr);
   try {
     dst_sketch_ptr->merge(datasketches::kll_sketch<float>::deserialize(src.ptr, src.len));
   } catch (const std::exception& e) {
@@ -2296,13 +2377,12 @@ StringVal AggregateFunctions::DsKllFinalizeHelper(FunctionContext* ctx,
     const StringVal& src) {
   DCHECK(!src.is_null);
   DCHECK_EQ(src.len, sizeof(datasketches::kll_sketch<float>));
-  datasketches::kll_sketch<float>* sketch_ptr =
-      reinterpret_cast<datasketches::kll_sketch<float>*>(src.ptr);
-  if (sketch_ptr->get_n() == 0) {
-    ctx->Free(src.ptr);
-    return StringVal::null();
+  auto sketch_ptr = reinterpret_cast<datasketches::kll_sketch<float>*>(src.ptr);
+  StringVal dst = StringVal::null();
+  if (!sketch_ptr->is_empty()) {
+    dst = SerializeDsSketch(ctx, *sketch_ptr);
   }
-  StringVal dst = SerializeDsSketch(ctx, *sketch_ptr);
+  sketch_ptr->~kll_sketch();
   ctx->Free(src.ptr);
   return dst;
 }
@@ -2316,8 +2396,7 @@ void AggregateFunctions::DsKllUpdate(FunctionContext* ctx, const FloatVal& src,
   if (src.is_null) return;
   DCHECK(!dst->is_null);
   DCHECK_EQ(dst->len, sizeof(datasketches::kll_sketch<float>));
-  datasketches::kll_sketch<float>* sketch_ptr =
-      reinterpret_cast<datasketches::kll_sketch<float>*>(dst->ptr);
+  auto sketch_ptr = reinterpret_cast<datasketches::kll_sketch<float>*>(dst->ptr);
   sketch_ptr->update(src.val);
 }
 
@@ -2348,8 +2427,7 @@ void AggregateFunctions::DsKllUnionUpdate(FunctionContext* ctx, const StringVal&
   if (src.is_null) return;
   DCHECK(!dst->is_null);
   DCHECK_EQ(dst->len, sizeof(datasketches::kll_sketch<float>));
-  datasketches::kll_sketch<float>* dst_sketch =
-      reinterpret_cast<datasketches::kll_sketch<float>*>(dst->ptr);
+  auto dst_sketch = reinterpret_cast<datasketches::kll_sketch<float>*>(dst->ptr);
   try {
     dst_sketch->merge(datasketches::kll_sketch<float>::deserialize(src.ptr, src.len));
   } catch (const std::exception& e) {

[impala] 01/04: IMPALA-11040: Remove unnecessary reset() method in class 'UnionStmt'

Posted by st...@apache.org.
This is an automated email from the ASF dual-hosted git repository.

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

commit ad29ce70b3f4cfab4105be52ef664129060fa04e
Author: skyyws <sk...@163.com>
AuthorDate: Wed Dec 1 19:40:57 2021 +0800

    IMPALA-11040: Remove unnecessary reset() method in class 'UnionStmt'
    
    When query contains multiple nested union stmt, more than twenty or
    thirty, and needs 'reAnalyze', such as rewrite expr. Query would
    execute slowly, even failed due to 'reset' method called in class
    'UnionStmt' and 'SetOperationStmt'.
    'SetOperationStmt' is added in IMPALA-9943 and IMPALA-4974. Multiple
    nested union stmt will lead to 'reset' called numbers to grow
    exponentially. Since 'operands_' will be reset in two class' reset()
    method, and handle with their children recursively. Too many nested
    union stmt will cause deep nesting.
    UnionStmt.reset() content is exactly same as SetOperationStmt.reset().
    This patch  removed this method in 'UnionStmt'. After this, the
    original query would execute quickly.
    An example already add in file 'union.test', without this patch, the
    example query would execute slowly, or even fail.
    
    Testing:
    - Added new test case in 'union.test'
    
    Change-Id: I408a396d40d9622f2ae6c459f49cbfcc19affd14
    Reviewed-on: http://gerrit.cloudera.org:8080/18061
    Reviewed-by: Qifan Chen <qc...@cloudera.com>
    Reviewed-by: Zoltan Borok-Nagy <bo...@cloudera.com>
    Tested-by: Impala Public Jenkins <im...@cloudera.com>
---
 .../java/org/apache/impala/analysis/UnionStmt.java |  20 ----
 .../functional-query/queries/QueryTest/union.test  | 129 +++++++++++++++++++++
 2 files changed, 129 insertions(+), 20 deletions(-)

diff --git a/fe/src/main/java/org/apache/impala/analysis/UnionStmt.java b/fe/src/main/java/org/apache/impala/analysis/UnionStmt.java
index 12a1e99..90ef70a 100644
--- a/fe/src/main/java/org/apache/impala/analysis/UnionStmt.java
+++ b/fe/src/main/java/org/apache/impala/analysis/UnionStmt.java
@@ -53,24 +53,4 @@ public class UnionStmt extends SetOperationStmt {
   public SetOperationStmt clone() {
     return new UnionStmt(this);
   }
-
-  /**
-   * Undoes all changes made by analyze() except distinct propagation and unnesting.
-   * After analysis, operands_ contains the list of unnested operands with qualifiers
-   * adjusted to reflect distinct propagation. Every operand in that list is reset().
-   * The unionDistinctOperands_ and unionAllOperands_ are cleared because they are
-   * redundant
-   * with operands_.
-   */
-  @Override
-  public void reset() {
-    super.reset();
-    for (SetOperand op : operands_) op.reset();
-    unionDistinctOperands_.clear();
-    unionAllOperands_.clear();
-    distinctAggInfo_ = null;
-    tupleId_ = null;
-    toSqlString_ = null;
-    widestExprs_ = null;
-  }
 }
diff --git a/testdata/workloads/functional-query/queries/QueryTest/union.test b/testdata/workloads/functional-query/queries/QueryTest/union.test
index 18889f6..fc5927e 100644
--- a/testdata/workloads/functional-query/queries/QueryTest/union.test
+++ b/testdata/workloads/functional-query/queries/QueryTest/union.test
@@ -1182,3 +1182,132 @@ group by id
 ---- TYPES
 double, int
 =====
+---- QUERY
+# IMPALA-11040: Verify query performance after removed reset() method in class 'UnionStmt'
+# Without this patch, the query would execute slowly, or even fail.
+SELECT t1.dt
+FROM
+(SELECT *
+FROM
+(SELECT *
+FROM
+(SELECT *
+FROM
+(SELECT *
+FROM
+(SELECT *
+FROM
+(SELECT *
+FROM
+(SELECT *
+FROM
+(SELECT *
+FROM
+(SELECT *
+FROM
+(SELECT *
+FROM
+(SELECT *
+FROM
+(SELECT *
+FROM
+(SELECT *
+FROM
+(SELECT *
+FROM
+(SELECT *
+FROM
+(SELECT *
+FROM
+(SELECT *
+FROM
+(SELECT *
+FROM
+(SELECT *
+FROM
+(SELECT *
+FROM
+(SELECT *
+FROM
+(SELECT *
+FROM
+(SELECT *
+FROM
+(SELECT *
+FROM
+(SELECT *
+FROM
+(SELECT *
+FROM
+(SELECT *
+FROM
+(SELECT *
+FROM
+(SELECT '20210101' AS dt
+UNION ALL SELECT '20210102' AS dt) AS t
+UNION ALL SELECT '20210103' AS dt) AS t
+UNION ALL SELECT '20210104' AS dt) AS t
+UNION ALL SELECT '20210105' AS dt) AS t
+UNION ALL SELECT '20210106' AS dt) AS t
+UNION ALL SELECT '20210107' AS dt) AS t
+UNION ALL SELECT '20210108' AS dt) AS t
+UNION ALL SELECT '20210109' AS dt) AS t
+UNION ALL SELECT '20210110' AS dt) AS t
+UNION ALL SELECT '20210111' AS dt) AS t
+UNION ALL SELECT '20210112' AS dt) AS t
+UNION ALL SELECT '20210113' AS dt) AS t
+UNION ALL SELECT '20210114' AS dt) AS t
+UNION ALL SELECT '20210115' AS dt) AS t
+UNION ALL SELECT '20210116' AS dt) AS t
+UNION ALL SELECT '20210117' AS dt) AS t
+UNION ALL SELECT '20210118' AS dt) AS t
+UNION ALL SELECT '20210119' AS dt) AS t
+UNION ALL SELECT '20210120' AS dt) AS t
+UNION ALL SELECT '20210121' AS dt) AS t
+UNION ALL SELECT '20210122' AS dt) AS t
+UNION ALL SELECT '20210123' AS dt) AS t
+UNION ALL SELECT '20210124' AS dt) AS t
+UNION ALL SELECT '20210125' AS dt) AS t
+UNION ALL SELECT '20210126' AS dt) AS t
+UNION ALL SELECT '20210127' AS dt) AS t
+UNION ALL SELECT '20210128' AS dt) AS t
+UNION ALL SELECT '20210129' AS dt) AS t
+UNION ALL SELECT '20210130' AS dt) AS t1
+CROSS JOIN
+(SELECT 1000+1000) ASt2
+GROUP BY t1.dt
+ORDER BY t1.dt;
+---- RESULTS
+'20210101'
+'20210102'
+'20210103'
+'20210104'
+'20210105'
+'20210106'
+'20210107'
+'20210108'
+'20210109'
+'20210110'
+'20210111'
+'20210112'
+'20210113'
+'20210114'
+'20210115'
+'20210116'
+'20210117'
+'20210118'
+'20210119'
+'20210120'
+'20210121'
+'20210122'
+'20210123'
+'20210124'
+'20210125'
+'20210126'
+'20210127'
+'20210128'
+'20210129'
+'20210130'
+---- TYPES
+string
+=====

[impala] 02/04: IMPALA-11055: Remove "exit" from HBase data loading script.

Posted by st...@apache.org.
This is an automated email from the ASF dual-hosted git repository.

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

commit 8d92ba15a56c20a685d96a45602e6a5f23d04e0b
Author: Riza Suminto <ri...@cloudera.com>
AuthorDate: Wed Dec 8 15:10:23 2021 -0800

    IMPALA-11055: Remove "exit" from HBase data loading script.
    
    load-functional-query-exhaustive-hbase-generated.create failed with
    newer HBase shell from version 2.4.6. HBase shell throws the following
    error at the end of script execution:
    
    ERROR NoMethodError: private method `exit' called for nil:NilClass
    
    Since we run the script in non-interactive way, it is safe for us to
    remove this last "exit" command from script.
    
    Testing:
    - Complete data loading without error.
    - Pass core tests.
    
    Change-Id: I9185704b02c51c7a9cb9aa7fd2a7d1103c8b7cbb
    Reviewed-on: http://gerrit.cloudera.org:8080/18079
    Reviewed-by: Impala Public Jenkins <im...@cloudera.com>
    Tested-by: Impala Public Jenkins <im...@cloudera.com>
---
 testdata/bin/generate-schema-statements.py | 2 --
 1 file changed, 2 deletions(-)

diff --git a/testdata/bin/generate-schema-statements.py b/testdata/bin/generate-schema-statements.py
index d0d715d..061601f 100755
--- a/testdata/bin/generate-schema-statements.py
+++ b/testdata/bin/generate-schema-statements.py
@@ -811,10 +811,8 @@ def generate_statements(output_name, test_vectors, sections,
         (output_name, file_format, codec, compression_type))
 
   if hbase_output:
-    hbase_output.create.append("exit")
     hbase_output.write_to_file('load-' + output_name + '-hbase-generated.create')
   if hbase_post_load:
-    hbase_post_load.load.append("exit")
     hbase_post_load.write_to_file('post-load-' + output_name + '-hbase-generated.sql')
   impala_invalidate.write_to_file("invalidate-" + output_name + "-impala-generated.sql")
 

[impala] 04/04: IMPALA-10926: Improve catalogd consistency and self events detection

Posted by st...@apache.org.
This is an automated email from the ASF dual-hosted git repository.

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

commit b25e250d321f329b98e017c648df75d052497963
Author: Sourabh Goyal <so...@cloudera.com>
AuthorDate: Mon Sep 20 16:24:13 2021 -0700

    IMPALA-10926: Improve catalogd consistency and self events detection
    
    In the current design, catalogd cache gets updated from 2 sources:
    1. Impala shell
    2. MetastoreEventProcessor
    
    The updates from the Impala shell are applied in place whereas
    MetastoreEventProcessor runs as a background thread which polls HMS
    events from notifications log table and apply them asynchronously.
    These two stream of updates cause consistency issues. For example
    consider the following sequence of alter table events on a table
    t1 as per HMS:
    
    1. alter table t1 from source s1 say other Impala cluster
    2. alter table t1 from source s2 say other Hive cluster
    3. alter table t1 from local Impala cluster
    
    The #3 alter table ddl operation would get reflected in the local
    cache immediately. However, later on event processor would process
    events from #1 and #2 above and try to alter the table. In an ideal
    scenario, these alters should have been applied before #3 i.e in the
    same order as they appear in HMS notification log. This leaves table
    t1 in an inconsistent state.
    
    Proposed solution:
    
    The main idea of the solution is to keep track of the last event id
    for a given table as eventId which the catalogd has synced to in the
    Db/Table object. The events processor ignores any event whose EVENT_ID
    is less than or equal to the eventId stored in the table. Once the
    events processor successfully processes a given event, it updates the
    value of eventId in the table before releasing the table lock. Also,
    any DDL or refresh operation on the catalogd from both catalog HMS
    metastore server and Impala shell) will follow the following steps
    to update the event id for the table:
    
    1. Acquire write lock on the table
    2. Perform ddl operation in HMS
    3. Sync table till the latest event id (as per HMS) since its last
       synced event id
    
    The above steps ensure that any concurrent updates applied on a same
    db/table from multiple sources like Hive, Impala or say multiple
    Impala clusters, get reflected in the local catalogd cache in the
    same order as they appear in HMS thus removing any inconsistencies.
    Also the solution relies on the existing locking mechanism in the
    catalogd to prevent any other concurrent updates to the table (even
    via EventsProcessor). In case of database objects, we will also have
    a similar eventId which represents the events on the database object
    (CREATE, DROP, ALTER database) to which the catalogd as synced to.
    
    This patch addresses the following:
    1. Add a new flag enable_sync_to_latest_event_on_ddls to enable/disable
       this improvement. It is turned off by default.
    2. If flag in #1 is enabled then apart from Impala shell and
       MetastoreEventProcessor the cache would also get updated for ddls
       executed via catalog HMS endpoints. And while excuting a ddl,
       db/table will be synced till latest event id.
    3. Event processor skips processing an event if db/table is already
       synced till that event id. Sets that event id in db/table if
       the event is processed.
    4. When EventProcessor detects a self event, it sets the last synced
       event id in db/table before skipping the processing of an event.
    5. Full table refresh sets the last event processed in table cache.
    
    Future Work:
    1. Sync db/table to latest event id for ddls executed from Impala
       shell (execDdlRequest() in catalogOpExecutor)
    
    Testing:
    
    1. Added new unit tests and modified existing ones
    2. Ran exhaustive tests with flag both turned on and off
    
    Change-Id: I36364e401911352c4666674eb98c8d61bbaae9b9
    Reviewed-on: http://gerrit.cloudera.org:8080/17859
    Reviewed-by: Vihang Karajgaonkar <vi...@cloudera.com>
    Tested-by: Impala Public Jenkins <im...@cloudera.com>
---
 be/src/catalog/catalog-server.cc                   |    6 +
 be/src/util/backend-gflag-util.cc                  |    3 +
 common/thrift/BackendGflags.thrift                 |    2 +
 .../impala/catalog/CatalogServiceCatalog.java      |  184 ++-
 fe/src/main/java/org/apache/impala/catalog/Db.java |   32 +
 .../main/java/org/apache/impala/catalog/Table.java |   37 +-
 .../org/apache/impala/catalog/TableLoader.java     |   77 +-
 .../apache/impala/catalog/events/EventFactory.java |    6 +-
 .../impala/catalog/events/MetastoreEvents.java     |  400 +++++-
 .../catalog/events/MetastoreEventsProcessor.java   |  169 ++-
 .../impala/catalog/events/NoOpEventProcessor.java  |    2 +-
 .../metastore/CatalogMetastoreServiceHandler.java  | 1301 +++++++++++++++++++-
 .../impala/catalog/metastore/HmsApiNameEnum.java   |   42 +-
 .../catalog/metastore/MetastoreServiceHandler.java |  204 ++-
 .../org/apache/impala/service/BackendConfig.java   |   14 +
 .../apache/impala/service/CatalogOpExecutor.java   |  142 ++-
 .../java/org/apache/impala/service/JniCatalog.java |    6 +
 .../apache/impala/catalog/AlterDatabaseTest.java   |    8 +-
 .../impala/catalog/MetastoreApiTestUtils.java      |  169 +++
 .../catalog/events/EventsProcessorStressTest.java  |    6 +-
 .../events/MetastoreEventsProcessorTest.java       |  333 +++--
 .../SynchronousHMSEventProcessorForTests.java      |   13 +-
 .../metastore/AbstractCatalogMetastoreTest.java    |    6 +-
 .../CatalogHmsSyncToLatestEventIdTest.java         |  777 ++++++++++++
 .../impala/testutil/CatalogServiceTestCatalog.java |   34 +-
 tests/custom_cluster/test_metastore_service.py     |   15 +-
 26 files changed, 3674 insertions(+), 314 deletions(-)

diff --git a/be/src/catalog/catalog-server.cc b/be/src/catalog/catalog-server.cc
index de8f4a8..0c5169b 100644
--- a/be/src/catalog/catalog-server.cc
+++ b/be/src/catalog/catalog-server.cc
@@ -118,6 +118,12 @@ DEFINE_bool(hms_event_incremental_refresh_transactional_table, true, "When set t
     "level events. Otherwise, it will always reload the whole table for transactional "
     "tables.");
 
+DEFINE_bool(enable_sync_to_latest_event_on_ddls, false, "This configuration is "
+    "used to sync db/table in catalogd cache to latest HMS event id whenever DDL "
+    "operations are performed from Impala shell and catalog metastore server "
+    "(if enabled). If this config is enabled, then the flag invalidate_hms_cache_on_ddls "
+    "should be disabled");
+
 DECLARE_string(state_store_host);
 DECLARE_int32(state_store_subscriber_port);
 DECLARE_int32(state_store_port);
diff --git a/be/src/util/backend-gflag-util.cc b/be/src/util/backend-gflag-util.cc
index 206b7fc..c76666c 100644
--- a/be/src/util/backend-gflag-util.cc
+++ b/be/src/util/backend-gflag-util.cc
@@ -100,6 +100,7 @@ DECLARE_string(kudu_sasl_protocol_name);
 DECLARE_bool(invalidate_hms_cache_on_ddls);
 DECLARE_bool(hms_event_incremental_refresh_transactional_table);
 DECLARE_bool(auto_check_compaction);
+DECLARE_bool(enable_sync_to_latest_event_on_ddls);
 
 // HS2 SAML2.0 configuration
 // Defined here because TAG_FLAG caused issues in global-flags.cc
@@ -321,6 +322,8 @@ Status PopulateThriftBackendGflags(TBackendGflags& cfg) {
   cfg.__set_hms_event_incremental_refresh_transactional_table(
       FLAGS_hms_event_incremental_refresh_transactional_table);
   cfg.__set_auto_check_compaction(FLAGS_auto_check_compaction);
+  cfg.__set_enable_sync_to_latest_event_on_ddls(
+      FLAGS_enable_sync_to_latest_event_on_ddls);
   return Status::OK();
 }
 
diff --git a/common/thrift/BackendGflags.thrift b/common/thrift/BackendGflags.thrift
index dcd0a24..bb9111e 100644
--- a/common/thrift/BackendGflags.thrift
+++ b/common/thrift/BackendGflags.thrift
@@ -221,4 +221,6 @@ struct TBackendGflags {
   98: required bool enable_shell_based_groups_mapping_support
 
   99: required bool auto_check_compaction
+
+  100: required bool enable_sync_to_latest_event_on_ddls
 }
diff --git a/fe/src/main/java/org/apache/impala/catalog/CatalogServiceCatalog.java b/fe/src/main/java/org/apache/impala/catalog/CatalogServiceCatalog.java
index ae5d363..e4e4de1 100644
--- a/fe/src/main/java/org/apache/impala/catalog/CatalogServiceCatalog.java
+++ b/fe/src/main/java/org/apache/impala/catalog/CatalogServiceCatalog.java
@@ -59,7 +59,10 @@ import org.apache.impala.authorization.AuthorizationPolicy;
 import org.apache.impala.catalog.FeFsTable.Utils;
 import org.apache.impala.catalog.HdfsPartition.FileDescriptor;
 import org.apache.impala.catalog.MetaStoreClientPool.MetaStoreClient;
+import org.apache.impala.catalog.events.EventFactory;
 import org.apache.impala.catalog.events.ExternalEventsProcessor;
+import org.apache.impala.catalog.events.MetastoreEvents.EventFactoryForSyncToLatestEvent;
+import org.apache.impala.catalog.events.MetastoreEvents.MetastoreEventFactory;
 import org.apache.impala.catalog.events.MetastoreEventsProcessor;
 import org.apache.impala.catalog.events.MetastoreEventsProcessor.EventProcessorStatus;
 import org.apache.impala.catalog.events.SelfEventContext;
@@ -218,7 +221,7 @@ public class CatalogServiceCatalog extends Catalog {
   //value of timeout for the topic update thread while waiting on the table lock.
   private final long topicUpdateTblLockMaxWaitTimeMs_;
   //Default value of timeout for acquiring a table lock.
-  private static final long LOCK_RETRY_TIMEOUT_MS = 7200000;
+  public static final long LOCK_RETRY_TIMEOUT_MS = 7200000;
   // Time to sleep before retrying to acquire a table lock
   private static final int LOCK_RETRY_DELAY_MS = 10;
   // default value of table id in the GetPartialCatalogObjectRequest
@@ -282,6 +285,7 @@ public class CatalogServiceCatalog extends Catalog {
 
   private ICatalogMetastoreServer catalogMetastoreServer_;
 
+  private MetastoreEventFactory syncToLatestEventFactory_;
   /**
    * See the gflag definition in be/.../catalog-server.cc for details on these modes.
    */
@@ -309,7 +313,6 @@ public class CatalogServiceCatalog extends Catalog {
   private final Set<String> blacklistedDbs_;
   // Tables that will be skipped in loading.
   private final Set<TableName> blacklistedTables_;
-
   /**
    * Initialize the CatalogServiceCatalog using a given MetastoreClientPool impl.
    *
@@ -363,6 +366,7 @@ public class CatalogServiceCatalog extends Catalog {
     metastoreEventProcessor_.start();
   }
 
+
   /**
    * Initializes the Catalog using the default MetastoreClientPool impl.
    * @param initialHmsCnxnTimeoutSec Time (in seconds) CatalogServiceCatalog will wait
@@ -424,6 +428,65 @@ public class CatalogServiceCatalog extends Catalog {
   }
 
   /**
+   * Acquire write lock on multiple tables. If the lock couldn't be acquired on any
+   * table, then release the table lock as well as version lock held due previous tables
+   * and return false.
+   * If write locks were acquired on all tables then release version lock
+   * on all tables except last one before returning true. In case of success, it is
+   * caller's responsibility to release versionLock's writeLock
+   * @param tables: Catalog tables on which write lock has to be acquired
+   * @return true if lock was acquired on all tables successfully. False
+   *         otherwise
+   */
+  public boolean tryWriteLock(Table[] tables) {
+    StringBuilder tableInfo = new StringBuilder();
+    int numTables = tables.length;
+    if (LOG.isDebugEnabled()) {
+      for(int i = 0; i < numTables; i++) {
+        tableInfo.append(tables[i].getFullName());
+        if(i < numTables - 1) {
+          tableInfo.append(", ");
+        }
+      }
+      LOG.debug("Trying to acquire write locks for tables: " +
+          tableInfo);
+    }
+    int tableIndex=-1, versionLockCount = 0;
+    try {
+      for(tableIndex = 0; tableIndex < numTables; tableIndex++) {
+        Table tbl = tables[tableIndex];
+        if (!tryWriteLock(tbl)) {
+          LOG.debug("Could not acquire write lock on table: " + tbl.getFullName());
+          return false;
+        }
+        versionLockCount += 1;
+      }
+      // in case of success, release version write lock for all tables except last
+      if (tableIndex == numTables) {
+        versionLockCount = versionLockCount - 1;
+      }
+      return true;
+    } finally {
+      if (tableIndex != numTables) {
+        // couldn't acquire lock on all tables
+        StringBuilder tablesInfo = new StringBuilder();
+        for(int i = 0; i < tableIndex; i++) {
+          tables[i].releaseWriteLock();
+          tablesInfo.append(tables[i].getFullName() + ((i < tableIndex-1) ? ", " : ""));
+        }
+        if (LOG.isDebugEnabled()) {
+          LOG.debug("Released table write lock on tables: {}", tablesInfo);
+        }
+      }
+      LOG.debug("Unlocking versionLock_ write lock {} number of times", versionLockCount);
+      while (versionLockCount > 0) {
+        versionLock_.writeLock().unlock();
+        versionLockCount = versionLockCount - 1;
+      }
+    }
+  }
+
+  /**
    * Tries to acquire the table similar to described in
    * {@link CatalogServiceCatalog#tryWriteLock(Table)} but with a custom timeout.
    */
@@ -2102,8 +2165,13 @@ public class CatalogServiceCatalog extends Catalog {
       tbl = getTable(dbName, tblName);
       // tbl doesn't exist in the catalog
       if (tbl == null) return null;
+      LOG.trace("table {} exits in cache, last synced id {}", tbl.getFullName(),
+          tbl.getLastSyncedEventId());
       // if no validWriteIdList is provided, we return the tbl if its loaded
-      if (tbl.isLoaded() && validWriteIdList == null) return tbl;
+      if (tbl.isLoaded() && validWriteIdList == null) {
+        LOG.trace("returning already loaded table {}", tbl.getFullName());
+        return tbl;
+      }
       // if a validWriteIdList is provided, we see if the cached table can provided a
       // consistent view of the given validWriteIdList. If yes, we can return the table
       // otherwise we reload the table. It is possible that the cached table is stale
@@ -2154,6 +2222,7 @@ public class CatalogServiceCatalog extends Catalog {
               .inc();
         }
         previousCatalogVersion = tbl.getCatalogVersion();
+        LOG.trace("Loading full table {}", tbl.getFullName());
         loadReq = tableLoadingMgr_.loadAsync(tableName, tbl.getCreateEventId(), reason);
       }
     } finally {
@@ -2204,8 +2273,12 @@ public class CatalogServiceCatalog extends Catalog {
                  && (!(existingTbl instanceof HdfsTable)
                         || AcidUtils.compare((HdfsTable) existingTbl,
                                updatedTbl.getValidWriteIds(), tableId)
-                            >= 0)))
+                            >= 0))) {
+        LOG.trace("returning existing table {} with last synced id: ",
+            existingTbl.getFullName(), existingTbl.getLastSyncedEventId());
         return existingTbl;
+      }
+
 
       if (existingTbl instanceof HdfsTable) {
         // Add the old instance to the deleteLog_ so we can send isDeleted updates for
@@ -2350,12 +2423,23 @@ public class CatalogServiceCatalog extends Catalog {
 
   /**
    * Wrapper around {@link #reloadTable(Table, boolean, CatalogObject.ThriftObjectType,
-   * String)} which passes false for {@code refreshUpdatedPartitions} argument and ignore
-   * the result.
+   * String, long)} which passes false for {@code refreshUpdatedPartitions} argument
+   * and ignore the result.
    */
   public void reloadTable(Table tbl, String reason) throws CatalogException {
+    reloadTable(tbl, reason, -1);
+  }
+
+  /**
+   * Wrapper around {@link #reloadTable(Table, boolean, CatalogObject.ThriftObjectType,
+   * String, long)} which passes false for {@code refreshUpdatedPartitions} argument
+   * and ignore the result.
+   * eventId: HMS event id which triggered reload
+   */
+  public void reloadTable(Table tbl, String reason, long eventId)
+      throws CatalogException {
     reloadTable(tbl, new TResetMetadataRequest(), CatalogObject.ThriftObjectType.NONE,
-        reason);
+        reason, eventId);
   }
 
   /**
@@ -2367,9 +2451,14 @@ public class CatalogServiceCatalog extends Catalog {
    * If {@code refreshUpdatedParts} is true, the refresh logic detects updated
    * partitions in metastore and reloads them too.
    * if {@code wantMinimalResult} is true, returns the result in the minimal form.
+   * if this reload is triggered while processing some HMS event (from example from
+   * MetastoreEventProcessor), then eventId passed would be > -1. In that case
+   * check table's last synced event id and if it is >= eventId then skip reloading
+   * table. Otherwise, set the eventId as table's last synced id after reload is done
    */
   public TCatalogObject reloadTable(Table tbl, TResetMetadataRequest request,
-      CatalogObject.ThriftObjectType resultType, String reason) throws CatalogException {
+      CatalogObject.ThriftObjectType resultType, String reason, long eventId)
+      throws CatalogException {
     LOG.info(String.format("Refreshing table metadata: %s", tbl.getFullName()));
     Preconditions.checkState(!(tbl instanceof IncompleteTable));
     String dbName = tbl.getDb().getName();
@@ -2378,12 +2467,30 @@ public class CatalogServiceCatalog extends Catalog {
       throw new CatalogException(String.format("Error refreshing metadata for table " +
           "%s due to lock contention", tbl.getFullName()));
     }
+    long newCatalogVersion = incrementAndGetCatalogVersion();
+    versionLock_.writeLock().unlock();
+    boolean syncToLatestEventId =
+        BackendConfig.INSTANCE.enableSyncToLatestEventOnDdls();
     final Timer.Context context =
         tbl.getMetrics().getTimer(Table.REFRESH_DURATION_METRIC).time();
     try {
-      long newCatalogVersion = incrementAndGetCatalogVersion();
-      versionLock_.writeLock().unlock();
+      if (eventId != -1 && tbl.getLastSyncedEventId() != -1 &&
+          tbl.getLastSyncedEventId() >= eventId) {
+        LOG.info("Not reloading table for event id: {} since table is already synced "
+            + "till event id: {}", eventId, tbl.getLastSyncedEventId());
+        return tbl.toTCatalogObject(resultType);
+      }
+      long currentHmsEventId = -1;
       try (MetaStoreClient msClient = getMetaStoreClient()) {
+        if (syncToLatestEventId) {
+          try {
+            currentHmsEventId = msClient.getHiveClient().getCurrentNotificationEventId()
+                .getEventId();
+          } catch (TException e) {
+            throw new CatalogException("Failed to reload table: " + tbl.getFullName() +
+                " as there was an error in fetching current event id from HMS", e);
+          }
+        }
         org.apache.hadoop.hive.metastore.api.Table msTbl = null;
         try {
           msTbl = msClient.getHiveClient().getTable(dbName, tblName);
@@ -2399,6 +2506,17 @@ public class CatalogServiceCatalog extends Catalog {
           tbl.load(true, msClient.getHiveClient(), msTbl, reason);
         }
       }
+      if (currentHmsEventId != -1 && syncToLatestEventId) {
+        // fetch latest event id from HMS before starting table load and set that event
+        // id as table's last synced id. It may happen that while the table was being
+        // reloaded, more events were generated for the table and the table reload would
+        // already have reflected those changes. In such scenario, we will again be
+        // replaying those events next time the table is synced to the latest event id.
+        // We are not replaying new events for the table in the end because certain
+        // events like ALTER_TABLE in MetastoreEventProcessor call this method which
+        // might trigger an endless loop cycle
+        tbl.setLastSyncedEventId(currentHmsEventId);
+      }
       tbl.setCatalogVersion(newCatalogVersion);
       LOG.info(String.format("Refreshed table metadata: %s", tbl.getFullName()));
       return tbl.toTCatalogObject(resultType);
@@ -2576,8 +2694,8 @@ public class CatalogServiceCatalog extends Catalog {
    * Refresh table if exists. Returns true if reloadTable() succeeds, false
    * otherwise.
    */
-  public boolean reloadTableIfExists(String dbName, String tblName, long eventId,
-      String reason) throws CatalogException {
+  public boolean reloadTableIfExists(String dbName, String tblName, String reason,
+      long eventId) throws CatalogException {
     try {
       Table table = getTable(dbName, tblName);
       if (table == null || table instanceof IncompleteTable) return false;
@@ -2586,7 +2704,7 @@ public class CatalogServiceCatalog extends Catalog {
             + "event {}.", dbName, tblName, eventId, table.getCreateEventId());
         return false;
       }
-      reloadTable(table, reason);
+      reloadTable(table, reason, eventId);
     } catch (DatabaseNotFoundException | TableLoadingException e) {
       LOG.info(String.format("Reload table if exists failed with: %s", e.getMessage()));
       return false;
@@ -3586,24 +3704,38 @@ public class CatalogServiceCatalog extends Catalog {
           "Error locking table %s for event %d", tbl.getFullName(), eventId));
     }
     try {
+      boolean syncToLatestEvent =
+          BackendConfig.INSTANCE.enableSyncToLatestEventOnDdls();
       long newCatalogVersion = incrementAndGetCatalogVersion();
       versionLock_.writeLock().unlock();
       HdfsTable hdfsTable = (HdfsTable) tbl;
+      // Note: Not doing >= check in below condition because a single event
+      // may invoke multiple method calls and each method might be setting
+      // last synced event id for the table. To ensure that subsequent method calls
+      // don't skip processing the table, hence > check
+      if (hdfsTable.getLastSyncedEventId() > eventId) {
+        LOG.info("EventId: {}, skipping adding writeIds {} with status {} to table {} "
+                + "since it is already synced till event id: {}", eventId, writeIds,
+            status, hdfsTable.getFullName(), hdfsTable.getLastSyncedEventId());
+        return;
+      }
       // A non-acid table could be upgraded to an acid table, and its valid write id list
       // is not yet be loaded. In this case, we just do nothing. The table should be
       // reloaded for the AlterTable event that sets the table as transactional.
       if (hdfsTable.getValidWriteIds() == null) {
-        LOG.debug("Not adding write ids to table {}.{} for event {} since it was just "
+        LOG.info("Not adding write ids to table {}.{} for event {} since it was just "
             + "upgraded to an acid table and it's valid write id list is not loaded",
             dbName, tblName, eventId);
-        return;
       }
-      if (!hdfsTable.addWriteIds(writeIds, status)) {
-        return;
+      if (hdfsTable.getValidWriteIds() != null &&
+          hdfsTable.addWriteIds(writeIds, status)) {
+        tbl.setCatalogVersion(newCatalogVersion);
+        LOG.info("Added {} writeId to table {}: {} for event {}", status,
+            tbl.getFullName(), writeIds, eventId);
+      }
+      if (syncToLatestEvent) {
+        hdfsTable.setLastSyncedEventId(eventId);
       }
-      tbl.setCatalogVersion(newCatalogVersion);
-      LOG.debug("Added {} writeId to table {}: {} for event {}", status,
-          tbl.getFullName(), writeIds, eventId);
     } finally {
       UnlockWriteLockIfErroneouslyLocked();
       tbl.releaseWriteLock();
@@ -3640,4 +3772,16 @@ public class CatalogServiceCatalog extends Catalog {
   public void setCatalogMetastoreServer(ICatalogMetastoreServer catalogMetastoreServer) {
     this.catalogMetastoreServer_ = catalogMetastoreServer;
   }
+
+  public void setEventFactoryForSyncToLatestEvent(MetastoreEventFactory factory) {
+    Preconditions.checkArgument(factory != null, "factory is null");
+    Preconditions.checkArgument(factory instanceof EventFactoryForSyncToLatestEvent,
+        "factory is not an instance of EventFactorySyncToLatestEvent");
+    this.syncToLatestEventFactory_ = factory;
+  }
+
+  public MetastoreEventFactory getEventFactoryForSyncToLatestEvent() {
+    return syncToLatestEventFactory_;
+  }
+
 }
diff --git a/fe/src/main/java/org/apache/impala/catalog/Db.java b/fe/src/main/java/org/apache/impala/catalog/Db.java
index ba16fdf..becbdba 100644
--- a/fe/src/main/java/org/apache/impala/catalog/Db.java
+++ b/fe/src/main/java/org/apache/impala/catalog/Db.java
@@ -113,6 +113,13 @@ public class Db extends CatalogObjectImpl implements FeDb {
   // processing is disabled.
   private long createEventId_ = -1;
 
+  // this field represents the last event id in metastore upto which this db is synced
+  // It is used if the flag sync_to_latest_event_on_ddls is set to true.
+  // Making it as volatile so that read and write of this variable are thread safe.
+  // As an example, EventProcessor can check if it needs to process a db event or not
+  // by reading this flag and without acquiring read lock on db object
+  private volatile long lastSyncedEventId_ = -1;
+
   public Db(String name, org.apache.hadoop.hive.metastore.api.Database msDb) {
     setMetastoreDb(name, msDb);
     tableCache_ = new CatalogObjectCache<>();
@@ -122,7 +129,24 @@ public class Db extends CatalogObjectImpl implements FeDb {
   public long getCreateEventId() { return createEventId_; }
 
   public void setCreateEventId(long eventId) {
+    // TODO: Add a preconditions check for eventId < lastSycnedEventId
     createEventId_ = eventId;
+    LOG.debug("createEventId_ for db: {} set to: {}", getName(), createEventId_);
+    if (lastSyncedEventId_ < eventId) {
+      setLastSyncedEventId(eventId);
+    }
+  }
+
+  public long getLastSyncedEventId() {
+    return lastSyncedEventId_;
+  }
+
+  public void setLastSyncedEventId(long eventId) {
+    // TODO: Add a preconditions check for eventId >= createEventId_
+    LOG.debug("lastSyncedEventId_ for db: {} set from {} to {}", getName(),
+        lastSyncedEventId_, eventId);
+    lastSyncedEventId_ = eventId;
+
   }
 
   public void setIsSystemDb(boolean b) { isSystemDb_ = b; }
@@ -569,4 +593,12 @@ public class Db extends CatalogObjectImpl implements FeDb {
     org.apache.hadoop.hive.metastore.api.Database db = getMetaStoreDb();
     return db == null ? null : db.getOwnerName();
   }
+
+  /**
+   *
+   * @return True if dbLock_ is held by current thread
+   */
+  public boolean isLockHeldByCurrentThread() {
+    return dbLock_.isHeldByCurrentThread();
+  }
 }
diff --git a/fe/src/main/java/org/apache/impala/catalog/Table.java b/fe/src/main/java/org/apache/impala/catalog/Table.java
index 3376a85..3c3bc43 100644
--- a/fe/src/main/java/org/apache/impala/catalog/Table.java
+++ b/fe/src/main/java/org/apache/impala/catalog/Table.java
@@ -57,11 +57,12 @@ import org.apache.impala.thrift.TTableInfoSelector;
 import org.apache.impala.thrift.TTableStats;
 import org.apache.impala.util.AcidUtils;
 import org.apache.impala.util.HdfsCachingUtil;
-import org.apache.log4j.Logger;
 
 import com.codahale.metrics.Timer;
 import com.google.common.base.Preconditions;
 import com.google.common.collect.Lists;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
 
 /**
  * Base class for table metadata.
@@ -73,7 +74,7 @@ import com.google.common.collect.Lists;
  * a key range into a fixed number of buckets).
  */
 public abstract class Table extends CatalogObjectImpl implements FeTable {
-  private static final Logger LOG = Logger.getLogger(Table.class);
+  private static final Logger LOG = LoggerFactory.getLogger(Table.class);
   protected org.apache.hadoop.hive.metastore.api.Table msTable_;
   protected final Db db_;
   protected final String name_;
@@ -183,6 +184,13 @@ public abstract class Table extends CatalogObjectImpl implements FeTable {
   // Table property key to determined if HMS translated a managed table to external table
   public static final String TBL_PROP_EXTERNAL_TABLE_PURGE = "external.table.purge";
 
+  // this field represents the last event id in metastore upto which this table is
+  // synced. It is used if the flag sync_to_latest_event_on_ddls is set to true.
+  // Making it as volatile so that read and write of this variable are thread safe.
+  // As an example, EventProcessor can check if it needs to process a table event or
+  // not by reading this flag and without acquiring read lock on table object
+  protected volatile long lastSyncedEventId_ = -1;
+
   protected Table(org.apache.hadoop.hive.metastore.api.Table msTable, Db db,
       String name, String owner) {
     msTable_ = msTable;
@@ -194,11 +202,34 @@ public abstract class Table extends CatalogObjectImpl implements FeTable {
     initMetrics();
   }
 
+  // TODO: Get rid of get and createEventId
+  // once we implement the logic of setting
+  // last synced id in full table reload
   public long getCreateEventId() { return createEventId_; }
 
   public void setCreateEventId(long eventId) {
-    this.createEventId_ = eventId;
+    // TODO: Add a preconditions check for eventId < lastSycnedEventId
+    createEventId_ = eventId;
+    LOG.debug("createEventId_ for table: {} set to: {}", getFullName(), createEventId_);
+    // TODO: Should we reset lastSyncedEvent Id if it is less than event Id?
+    // If we don't reset it - we may start syncing table from an event id which
+    // is less than create event id
+    if (lastSyncedEventId_ < eventId) {
+      setLastSyncedEventId(eventId);
+    }
+  }
+
+  public long getLastSyncedEventId() {
+    return lastSyncedEventId_;
   }
+
+  public void setLastSyncedEventId(long eventId) {
+    // TODO: Add a preconditions check for eventId >= createEventId_
+    LOG.debug("lastSyncedEventId_ for table: {} set from {} to {}", getFullName(),
+        lastSyncedEventId_, eventId);
+    lastSyncedEventId_ = eventId;
+  }
+
   /**
    * Returns if the given HMS table is an external table (uses table type if
    * available or else uses table properties). Implementation is based on org.apache
diff --git a/fe/src/main/java/org/apache/impala/catalog/TableLoader.java b/fe/src/main/java/org/apache/impala/catalog/TableLoader.java
index 0b263ed..bdd97a3 100644
--- a/fe/src/main/java/org/apache/impala/catalog/TableLoader.java
+++ b/fe/src/main/java/org/apache/impala/catalog/TableLoader.java
@@ -17,16 +17,23 @@
 
 package org.apache.impala.catalog;
 
+import com.google.common.base.Preconditions;
 import java.util.List;
 import java.util.concurrent.TimeUnit;
 
 import org.apache.hadoop.hive.metastore.TableType;
 import org.apache.hadoop.hive.metastore.api.NoSuchObjectException;
 import org.apache.hadoop.hive.metastore.api.NotificationEvent;
+import org.apache.impala.catalog.events.EventFactory;
+import org.apache.impala.catalog.events.MetastoreEvents;
 import org.apache.impala.catalog.events.MetastoreEvents.CreateTableEvent;
 import org.apache.impala.catalog.events.MetastoreEventsProcessor;
+import org.apache.impala.common.Metrics;
 import org.apache.impala.compat.MetastoreShim;
-import org.apache.log4j.Logger;
+import org.apache.impala.service.BackendConfig;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+import org.apache.thrift.TException;
 
 import com.google.common.base.Stopwatch;
 
@@ -38,7 +45,7 @@ import org.apache.impala.util.ThreadNameAnnotator;
  * the Hive Metastore / HDFS / etc.
  */
 public class TableLoader {
-  private static final Logger LOG = Logger.getLogger(TableLoader.class);
+  private static final Logger LOG = LoggerFactory.getLogger(TableLoader.class);
 
   private final CatalogServiceCatalog catalog_;
 
@@ -46,9 +53,12 @@ public class TableLoader {
   // concurrency bugs. Currently used to serialize calls to "getTable()" due to
   // HIVE-5457.
   private static final Object metastoreAccessLock_ = new Object();
+  private Metrics metrics_ = new Metrics();
 
   public TableLoader(CatalogServiceCatalog catalog) {
+    Preconditions.checkNotNull(catalog);
     catalog_ = catalog;
+    initMetrics(metrics_);
   }
 
   /**
@@ -64,7 +74,9 @@ public class TableLoader {
     String fullTblName = db.getName() + "." + tblName;
     String annotation = "Loading metadata for: " + fullTblName + " (" + reason + ")";
     LOG.info(annotation);
-    Table table;
+    Table table = null;
+    boolean syncToLatestEventId =
+        BackendConfig.INSTANCE.enableSyncToLatestEventOnDdls();
     // turn all exceptions into TableLoadingException
     List<NotificationEvent> events = null;
     try (ThreadNameAnnotator tna = new ThreadNameAnnotator(annotation);
@@ -111,8 +123,36 @@ public class TableLoader {
       }
       table.updateHMSLoadTableSchemaTime(hmsLoadTime);
       table.setCreateEventId(eventId);
+      long latestEventId = -1;
+      if (syncToLatestEventId) {
+        // acquire write lock on table since MetastoreEventProcessor.syncToLatestEventId
+        // expects current thread to have write lock on the table
+        if (!catalog_.tryWriteLock(table)) {
+          throw new CatalogException("Couldn't acquire write lock on new table object"
+              + " created when doing a full table reload of " + table.getFullName());
+        }
+        catalog_.getLock().writeLock().unlock();
+        try {
+          latestEventId = msClient.getHiveClient().
+              getCurrentNotificationEventId().getEventId();
+        } catch (TException e) {
+          throw new TableLoadingException("Failed to get latest event id from HMS "
+              + "while loading table: " + table.getFullName(), e);
+        }
+      }
+
       table.load(false, msClient.getHiveClient(), msTbl, reason);
       table.validate();
+      if (syncToLatestEventId) {
+        LOG.debug("After full reload, table {} is synced atleast till event id {}. "
+                + "Checking if there are more events generated for this table "
+                + "while the full reload was in progress", table.getFullName(),
+            latestEventId);
+        table.setLastSyncedEventId(latestEventId);
+        // write lock is not required since it is full table reload
+        MetastoreEventsProcessor.syncToLatestEventId(catalog_, table,
+            catalog_.getEventFactoryForSyncToLatestEvent(), metrics_);
+      }
     } catch (TableLoadingException e) {
       table = IncompleteTable.createFailedMetadataLoadTable(db, tblName, e);
     } catch (NoSuchObjectException e) {
@@ -127,9 +167,40 @@ public class TableLoader {
           db, tblName, new TableLoadingException(
           "Failed to load metadata for table: " + fullTblName + ". Running " +
           "'invalidate metadata " + fullTblName + "' may resolve this problem.", e));
+    } finally {
+      if (table != null && table.isWriteLockedByCurrentThread()) {
+        table.releaseWriteLock();
+      }
     }
     LOG.info("Loaded metadata for: " + fullTblName + " (" +
         sw.elapsed(TimeUnit.MILLISECONDS) + "ms)");
     return table;
   }
+
+  private void initMetrics(Metrics metrics) {
+    metrics.addTimer(
+        MetastoreEventsProcessor.EVENTS_FETCH_DURATION_METRIC);
+    metrics.addTimer(
+        MetastoreEventsProcessor.EVENTS_PROCESS_DURATION_METRIC);
+    metrics.addMeter(
+        MetastoreEventsProcessor.EVENTS_RECEIVED_METRIC);
+    metrics.addCounter(
+        MetastoreEventsProcessor.EVENTS_SKIPPED_METRIC);
+    metrics.addCounter(
+        MetastoreEventsProcessor.NUMBER_OF_TABLE_REFRESHES);
+    metrics.addCounter(
+        MetastoreEventsProcessor.NUMBER_OF_PARTITION_REFRESHES);
+    metrics.addCounter(
+        MetastoreEventsProcessor.NUMBER_OF_TABLES_ADDED);
+    metrics.addCounter(
+        MetastoreEventsProcessor.NUMBER_OF_TABLES_REMOVED);
+    metrics.addCounter(
+        MetastoreEventsProcessor.NUMBER_OF_DATABASES_ADDED);
+    metrics.addCounter(
+        MetastoreEventsProcessor.NUMBER_OF_DATABASES_REMOVED);
+    metrics.addCounter(
+        MetastoreEventsProcessor.NUMBER_OF_PARTITIONS_ADDED);
+    metrics.addCounter(
+        MetastoreEventsProcessor.NUMBER_OF_PARTITIONS_REMOVED);
+  }
 }
diff --git a/fe/src/main/java/org/apache/impala/catalog/events/EventFactory.java b/fe/src/main/java/org/apache/impala/catalog/events/EventFactory.java
index 17c20b5..bd5619d 100644
--- a/fe/src/main/java/org/apache/impala/catalog/events/EventFactory.java
+++ b/fe/src/main/java/org/apache/impala/catalog/events/EventFactory.java
@@ -19,6 +19,7 @@ package org.apache.impala.catalog.events;
 
 import org.apache.hadoop.hive.metastore.api.NotificationEvent;
 import org.apache.impala.catalog.events.MetastoreEvents.MetastoreEvent;
+import org.apache.impala.common.Metrics;
 
 /**
  * Factory interface to generate a {@link MetastoreEvent} from a {@link NotificationEvent}
@@ -29,8 +30,11 @@ public interface EventFactory {
   /**
    * Generates a {@link MetastoreEvent} representing {@link NotificationEvent}
    * @param hmsEvent the event as received from Hive Metastore.
+   * @param metrics metrics which gets updated when MetastoreEvent from this api
+   *                is processed.
    * @return {@link MetastoreEvent} representing hmsEvent.
    * @throws MetastoreNotificationException If the hmsEvent information cannot be parsed.
    */
-  MetastoreEvent get(NotificationEvent hmsEvent) throws MetastoreNotificationException;
+  MetastoreEvent get(NotificationEvent hmsEvent, Metrics metrics)
+      throws MetastoreNotificationException;
 }
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 7fd86da..1d56c81 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
@@ -58,8 +58,10 @@ import org.apache.impala.analysis.TableName;
 import org.apache.impala.catalog.CatalogException;
 import org.apache.impala.catalog.CatalogServiceCatalog;
 import org.apache.impala.catalog.DatabaseNotFoundException;
+import org.apache.impala.catalog.Db;
 import org.apache.impala.catalog.HdfsTable;
 import org.apache.impala.catalog.MetaStoreClientPool;
+import org.apache.impala.catalog.IncompleteTable;
 import org.apache.impala.catalog.TableLoadingException;
 import org.apache.impala.catalog.TableNotFoundException;
 import org.apache.impala.catalog.TableNotLoadedException;
@@ -68,6 +70,7 @@ import org.apache.impala.common.Metrics;
 import org.apache.impala.common.Reference;
 import org.apache.impala.compat.MetastoreShim;
 import org.apache.impala.hive.common.MutableValidWriteIdList;
+
 import org.apache.impala.service.BackendConfig;
 import org.apache.impala.service.CatalogOpExecutor;
 import org.apache.impala.thrift.TPartitionKeyValue;
@@ -162,23 +165,23 @@ public class MetastoreEvents {
         LoggerFactory.getLogger(MetastoreEventFactory.class);
 
     // catalog service instance to be used for creating eventHandlers
-    private final CatalogServiceCatalog catalog_;
+    protected final CatalogServiceCatalog catalog_;
     // metrics registry to be made available for each events to publish metrics
-    private final Metrics metrics_;
+    //protected final Metrics metrics_;
     // catalogOpExecutor needed for the create/drop events for table and database.
-    private final CatalogOpExecutor catalogOpExecutor_;
+    protected final CatalogOpExecutor catalogOpExecutor_;
+    private static MetastoreEventFactory INSTANCE = null;
 
-    public MetastoreEventFactory(CatalogOpExecutor catalogOpExecutor, Metrics metrics) {
+    public MetastoreEventFactory(CatalogOpExecutor catalogOpExecutor) {
       this.catalogOpExecutor_ = Preconditions.checkNotNull(catalogOpExecutor);
       this.catalog_ = Preconditions.checkNotNull(catalogOpExecutor.getCatalog());
-      this.metrics_ = Preconditions.checkNotNull(metrics);
     }
 
     /**
      * creates instance of <code>MetastoreEvent</code> used to process a given event type.
      * If the event type is unknown, returns a IgnoredEvent
      */
-    public MetastoreEvent get(NotificationEvent event)
+    public MetastoreEvent get(NotificationEvent event, Metrics metrics)
         throws MetastoreNotificationException {
       Preconditions.checkNotNull(event.getEventType());
       MetastoreEventType metastoreEventType =
@@ -186,37 +189,37 @@ public class MetastoreEvents {
       if (BackendConfig.INSTANCE.getHMSEventIncrementalRefreshTransactionalTable()) {
         switch (metastoreEventType) {
           case ALLOC_WRITE_ID_EVENT:
-            return new AllocWriteIdEvent(catalogOpExecutor_, metrics_, event);
+            return new AllocWriteIdEvent(catalogOpExecutor_, metrics, event);
           case COMMIT_TXN:
-            return new CommitTxnEvent(catalogOpExecutor_, metrics_, event);
+            return new CommitTxnEvent(catalogOpExecutor_, metrics, event);
           case ABORT_TXN:
-            return new AbortTxnEvent(catalogOpExecutor_, metrics_, event);
+            return new AbortTxnEvent(catalogOpExecutor_, metrics, event);
         }
       }
       switch (metastoreEventType) {
         case CREATE_TABLE:
-          return new CreateTableEvent(catalogOpExecutor_, metrics_, event);
+          return new CreateTableEvent(catalogOpExecutor_, metrics, event);
         case DROP_TABLE:
-          return new DropTableEvent(catalogOpExecutor_, metrics_, event);
+          return new DropTableEvent(catalogOpExecutor_, metrics, event);
         case ALTER_TABLE:
-          return new AlterTableEvent(catalogOpExecutor_, metrics_, event);
+          return new AlterTableEvent(catalogOpExecutor_, metrics, event);
         case CREATE_DATABASE:
-          return new CreateDatabaseEvent(catalogOpExecutor_, metrics_, event);
+          return new CreateDatabaseEvent(catalogOpExecutor_, metrics, event);
         case DROP_DATABASE:
-          return new DropDatabaseEvent(catalogOpExecutor_, metrics_, event);
+          return new DropDatabaseEvent(catalogOpExecutor_, metrics, event);
         case ALTER_DATABASE:
-          return new AlterDatabaseEvent(catalogOpExecutor_, metrics_, event);
+          return new AlterDatabaseEvent(catalogOpExecutor_, metrics, event);
         case ADD_PARTITION:
-          return new AddPartitionEvent(catalogOpExecutor_, metrics_, event);
+          return new AddPartitionEvent(catalogOpExecutor_, metrics, event);
         case DROP_PARTITION:
-          return new DropPartitionEvent(catalogOpExecutor_, metrics_, event);
+          return new DropPartitionEvent(catalogOpExecutor_, metrics, event);
         case ALTER_PARTITION:
-          return new AlterPartitionEvent(catalogOpExecutor_, metrics_, event);
+          return new AlterPartitionEvent(catalogOpExecutor_, metrics, event);
         case INSERT:
-          return new InsertEvent(catalogOpExecutor_, metrics_, event);
+          return new InsertEvent(catalogOpExecutor_, metrics, event);
         default:
           // ignore all the unknown events by creating a IgnoredEvent
-          return new IgnoredEvent(catalogOpExecutor_, metrics_, event);
+          return new IgnoredEvent(catalogOpExecutor_, metrics, event);
       }
     }
 
@@ -233,18 +236,19 @@ public class MetastoreEvents {
      * dropped. In such a case, the create event can be ignored
      *
      * @param events NotificationEvents fetched from metastore
+     * @param metrics Metrics to update while filtering events
      * @return A list of MetastoreEvents corresponding to the given the NotificationEvents
      * @throws MetastoreNotificationException if a NotificationEvent could not be
      *     parsed into MetastoreEvent
      */
-    List<MetastoreEvent> getFilteredEvents(List<NotificationEvent> events)
-        throws MetastoreNotificationException {
+    List<MetastoreEvent> getFilteredEvents(List<NotificationEvent> events,
+        Metrics metrics) throws MetastoreNotificationException {
       Preconditions.checkNotNull(events);
       if (events.isEmpty()) return Collections.emptyList();
 
       List<MetastoreEvent> metastoreEvents = new ArrayList<>(events.size());
       for (NotificationEvent event : events) {
-        metastoreEvents.add(get(event));
+        metastoreEvents.add(get(event, metrics));
       }
       // filter out the create events which has a corresponding drop event later
       int sizeBefore = metastoreEvents.size();
@@ -279,11 +283,11 @@ public class MetastoreEvents {
       }
       LOG.info(String.format("Total number of events received: %d Total number of events "
           + "filtered out: %d", sizeBefore, numFilteredEvents));
-      metrics_.getCounter(MetastoreEventsProcessor.EVENTS_SKIPPED_METRIC)
+      metrics.getCounter(MetastoreEventsProcessor.EVENTS_SKIPPED_METRIC)
               .inc(numFilteredEvents);
-      LOG.debug("Incremented skipped metric to " + metrics_
+      LOG.debug("Incremented skipped metric to " + metrics
           .getCounter(MetastoreEventsProcessor.EVENTS_SKIPPED_METRIC).getCount());
-      return createBatchEvents(metastoreEvents);
+      return createBatchEvents(metastoreEvents, metrics);
     }
 
     /**
@@ -292,7 +296,7 @@ public class MetastoreEvents {
      * events depending on whether it finds any events which could be batched together.
      */
     @VisibleForTesting
-    List<MetastoreEvent> createBatchEvents(List<MetastoreEvent> events) {
+    List<MetastoreEvent> createBatchEvents(List<MetastoreEvent> events, Metrics metrics) {
       if (events.size() < 2) return events;
       int i = 0, j = 1;
       List<MetastoreEvent> batchedEventList = new ArrayList<>();
@@ -309,7 +313,7 @@ public class MetastoreEvents {
           if (current.getNumberOfEvents() > 1) {
             current.infoLog("Created a batch event for {} events from {} to {}",
                 current.getNumberOfEvents(), startEventId, current.getEventId());
-            metrics_.getCounter(MetastoreEventsProcessor.NUMBER_OF_BATCH_EVENTS).inc();
+            metrics.getCounter(MetastoreEventsProcessor.NUMBER_OF_BATCH_EVENTS).inc();
           }
           batchedEventList.add(current);
           current = next;
@@ -323,13 +327,89 @@ public class MetastoreEvents {
       if (current.getNumberOfEvents() > 1) {
         current.infoLog("Created a batch event for {} events from {} to {}",
             current.getNumberOfEvents(), startEventId, current.getEventId());
-        metrics_.getCounter(MetastoreEventsProcessor.NUMBER_OF_BATCH_EVENTS).inc();
+        metrics.getCounter(MetastoreEventsProcessor.NUMBER_OF_BATCH_EVENTS).inc();
       }
       batchedEventList.add(current);
       return batchedEventList;
     }
   }
 
+  // A factory class for creating metastore events for syncing to latest event id
+  // We can't reuse existing event factory because of the following scenario:
+  // A ddl is executed from Impala shell. As a result of it, a self event generated
+  // for it should be ignored by event factory in MetastoreEventProcessor. If
+  // MetastoreEventProcessor also uses EventFactoryForSyncToLatestEvent then it would
+  // skip self event check and end up re processing the event which defeats the purpose
+  // of self event code. The reason for this behavior is - ddl ops from Impala shell
+  // i.e catalogOpExecutor don't sync db/table to latest event id. After
+  // IMPALA-10976 is merged, we can use one event factory and disable self event
+  // check in that.
+
+  public static class EventFactoryForSyncToLatestEvent extends
+      MetastoreEvents.MetastoreEventFactory {
+
+    private static final Logger LOG =
+        LoggerFactory.getLogger(MetastoreEventFactory.class);
+
+    public EventFactoryForSyncToLatestEvent(CatalogOpExecutor catalogOpExecutor) {
+      super(catalogOpExecutor);
+    }
+
+    public MetastoreEvent get(NotificationEvent event, Metrics metrics)
+        throws MetastoreNotificationException {
+      Preconditions.checkNotNull(event.getEventType());
+      MetastoreEventType metastoreEventType =
+          MetastoreEventType.from(event.getEventType());
+      switch (metastoreEventType) {
+        case ALTER_DATABASE:
+          return new AlterDatabaseEvent(catalogOpExecutor_, metrics, event) {
+            @Override
+            protected boolean isSelfEvent() {
+              return false;
+            }
+          };
+        case ALTER_TABLE:
+          return new AlterTableEvent(catalogOpExecutor_, metrics, event) {
+            @Override
+            protected boolean isSelfEvent() {
+              return false;
+            }
+          };
+        case ADD_PARTITION:
+          return new AddPartitionEvent(catalogOpExecutor_, metrics, event) {
+            @Override
+            public boolean isSelfEvent() {
+              return false;
+            }
+          };
+        case ALTER_PARTITION:
+          return new AlterPartitionEvent(catalogOpExecutor_, metrics, event) {
+            @Override
+            public boolean isSelfEvent() {
+              return false;
+            }
+          };
+        case DROP_PARTITION:
+          return new DropPartitionEvent(catalogOpExecutor_, metrics, event) {
+            @Override
+            public boolean isSelfEvent() {
+              return false;
+            }
+          };
+        case INSERT:
+          return new InsertEvent(catalogOpExecutor_, metrics, event) {
+            @Override
+            public boolean isSelfEvent() {
+              return false;
+            }
+          };
+        default:
+          return super.get(event, metrics);
+      }
+    }
+  }
+
+
   /**
    * Abstract base class for all MetastoreEvents. A MetastoreEvent is a object used to
    * process a NotificationEvent received from metastore. It is self-contained with all
@@ -426,6 +506,12 @@ public class MetastoreEvents {
             .getCounter(MetastoreEventsProcessor.EVENTS_SKIPPED_METRIC).getCount());
         return;
       }
+      if (BackendConfig.INSTANCE.enableSyncToLatestEventOnDdls()) {
+        if (shouldSkipWhenSyncingToLatestEventId()) {
+          metrics_.getCounter(MetastoreEventsProcessor.EVENTS_SKIPPED_METRIC).inc();
+          return;
+        }
+      }
       process();
     }
 
@@ -469,6 +555,18 @@ public class MetastoreEvents {
     protected MetastoreEventType getBatchEventType() { return null; }
 
     /**
+     * Evaluates whether processing of this event should be skipped
+     * if sync to latest event id is enabled. The event should
+     * be skipped if the db/table is already synced atleast till
+     * this event.
+     * @return true if the event should be skipped. False
+     *          otherwise
+     * @throws CatalogException
+     */
+    protected abstract boolean shouldSkipWhenSyncingToLatestEventId()
+        throws CatalogException;
+
+    /**
      * Process the information available in the NotificationEvent to take appropriate
      * action on Catalog
      *
@@ -604,6 +702,17 @@ public class MetastoreEvents {
       }
       return false;
     }
+
+    public final boolean isDropEvent() {
+      return (this instanceof DropTableEvent ||
+          this instanceof DropDatabaseEvent ||
+          this instanceof DropPartitionEvent);
+    }
+
+    @Override
+    public String toString() {
+      return String.format(STR_FORMAT_EVENT_ID_TYPE, eventId_, eventType_);
+    }
   }
 
   public static String getStringProperty(
@@ -723,10 +832,10 @@ public class MetastoreEvents {
     protected boolean reloadTableFromCatalog(String operation, boolean isTransactional)
         throws CatalogException {
       try {
-        if (!catalog_.reloadTableIfExists(dbName_, tblName_, getEventId(),
-            "Processing " + operation + " event from HMS")) {
+        if (!catalog_.reloadTableIfExists(dbName_, tblName_,
+            "Processing " + operation + " event from HMS", getEventId())) {
           debugLog("Automatic refresh on table {} failed as the table "
-              + "either does not exist anymore or is not in loaded state.",
+                  + "either does not exist anymore or is not in loaded state.",
               getFullyQualifiedTblName());
           return false;
         }
@@ -776,6 +885,95 @@ public class MetastoreEvents {
             getFullyQualifiedTblName());
       }
     }
+
+    /**
+     * To decide whether to skip processing this event, fetch table from cache
+     * and compare the last synced event id of cache table with this event id.
+     * Skip this event if the table is already synced till this event id. Otherwise,
+     * process this event.
+     * @return true if processing of this event should be skipped. False otherwise
+     * @throws CatalogException
+     */
+    protected boolean shouldSkipWhenSyncingToLatestEventId() throws CatalogException {
+      Preconditions.checkState(
+          BackendConfig.INSTANCE.enableSyncToLatestEventOnDdls(),
+          "sync to latest event flag is not set to true");
+      long eventId = this.getEventId();
+      Preconditions.checkState(eventId > 0,
+          "Invalid event id %s. Should be greater than "
+              + "0", eventId);
+      org.apache.impala.catalog.Table tbl = null;
+      try {
+        tbl = catalog_.getTable(dbName_, tblName_);
+        if (tbl == null) {
+          infoLog("Skipping on table {}.{} since it does not exist in cache", dbName_,
+              tblName_);
+          return true;
+        }
+        // During alter table rename, the renamed table is created as Incomplete table
+        // with create event id set to alter_table event id (i.e not -1) and therefore
+        // should *NOT* be skipped in event processing
+        if (tbl instanceof IncompleteTable && tbl.getLastSyncedEventId() == -1) {
+          infoLog("Skipping on an incomplete table {} since last synced event id is "
+              + "set to {}", tbl.getFullName(), tbl.getLastSyncedEventId());
+          return true;
+        }
+      } catch (DatabaseNotFoundException e) {
+        infoLog("Skipping on table {} because db {} not found in cache", tblName_,
+            dbName_);
+        return true;
+      }
+      boolean shouldSkip = false;
+      // do not acquire read lock on tbl because lastSyncedEventId_ is volatile.
+      // It is fine if this method returns false because at the time of actual
+      // processing of the event, we would again check lastSyncedEventId_ after
+      // acquiring write lock on table and if the table was already synced till
+      // this event id, the event processing would be skipped.
+      if (tbl.getLastSyncedEventId() >= eventId) {
+        infoLog("Skipping on table {} since it is already synced till event id {}",
+            tbl.getFullName(), tbl.getLastSyncedEventId());
+        shouldSkip = true;
+      }
+      return shouldSkip;
+    }
+
+    /**
+     * Overrides parent's isSelfEvent method. If the event turns out to be a self event
+     * then this implementation checks and sets table's lastSyncedEvent if it is less
+     * than this event id. It is done so that when syncing table to latest event id on
+     * subsequent ddl operations, the self event is not processed again.
+     * @return
+     */
+    @Override
+    protected boolean isSelfEvent() {
+      boolean isSelfEvent = super.isSelfEvent();
+      if (!isSelfEvent || !BackendConfig.INSTANCE.enableSyncToLatestEventOnDdls()) {
+        return isSelfEvent;
+      }
+      org.apache.impala.catalog.Table tbl = null;
+      try {
+        tbl = catalog_.getTable(getDbName(), getTableName());
+
+        if (tbl != null && catalog_.tryWriteLock(tbl)) {
+          catalog_.getLock().writeLock().unlock();
+          if (tbl.getLastSyncedEventId() < getEventId()) {
+            infoLog("is a self event. last synced event id for "
+                    + "table {} is {}. Setting it to {}", tbl.getFullName(),
+                tbl.getLastSyncedEventId(), getEventId());
+            tbl.setLastSyncedEventId(getEventId());
+          }
+        }
+      } catch (CatalogException e) {
+        debugLog("ignoring exception when trying to set latest event for a self event "
+            + "on table {}.{}", getDbName(), getTableName(), e);
+      } finally {
+        catalogOpExecutor_.UnlockWriteLockIfErronouslyLocked();
+        if (tbl != null && tbl.isWriteLockedByCurrentThread()) {
+          tbl.releaseWriteLock();
+        }
+      }
+      return true;
+    }
   }
 
   /**
@@ -803,6 +1001,44 @@ public class MetastoreEvents {
     protected boolean isEventProcessingDisabled() {
       return false;
     }
+
+    protected boolean shouldSkipWhenSyncingToLatestEventId() throws CatalogException {
+      return false;
+    }
+
+    /**
+     * Overrides parent's isSelfEvent method. If the event turns out to be a self event
+     * then this implementation checks and sets db's lastSyncedEvent if it is less
+     * than this event id. It is done so that when syncing db to latest event id on
+     * subsequent ddl operations, the self event is not processed again.
+     * @return
+     */
+    @Override
+    protected boolean isSelfEvent() {
+      boolean isSelfEvent = super.isSelfEvent();
+      if (!isSelfEvent || !BackendConfig.INSTANCE.enableSyncToLatestEventOnDdls()) {
+        return isSelfEvent;
+      }
+      Db db = null;
+      try {
+        db = catalog_.getDb(getDbName());
+        if (db != null && catalog_.tryLockDb(db)) {
+          catalog_.getLock().writeLock().unlock();
+          if (db.getLastSyncedEventId() < getEventId()) {
+            infoLog("is a self event. last synced event id for "
+                    + "db {} is {}. Setting it to {}", getDbName(),
+                db.getLastSyncedEventId(), getEventId());
+            db.setLastSyncedEventId(getEventId());
+          }
+        }
+      } finally {
+        catalogOpExecutor_.UnlockWriteLockIfErronouslyLocked();
+        if (db != null && db.isLockHeldByCurrentThread()) {
+          db.getLock().unlock();
+        }
+      }
+      return true;
+    }
   }
 
   /**
@@ -905,6 +1141,10 @@ public class MetastoreEvents {
     public Table getTable() {
       return msTbl_;
     }
+
+    protected boolean shouldSkipWhenSyncingToLatestEventId() {
+      return false;
+    }
   }
 
   /**
@@ -1054,7 +1294,7 @@ public class MetastoreEvents {
     // the table object after alter operation, as parsed from the NotificationEvent
     protected org.apache.hadoop.hive.metastore.api.Table tableAfter_;
     // true if this alter event was due to a rename operation
-    private final boolean isRename_;
+    protected final boolean isRename_;
     // value of event sync flag for this table before the alter operation
     private final Boolean eventSyncBeforeFlag_;
     // value of the event sync flag if available at this table after the alter operation
@@ -1124,6 +1364,23 @@ public class MetastoreEvents {
             .getCounter(MetastoreEventsProcessor.EVENTS_SKIPPED_METRIC).getCount());
       }
     }
+
+    /**
+     * If the alter table event is generated because of table rename then event
+     * should *NOT* be skipped if old table is not synced this till event AND
+     * new table doesn't exist in cache. Skip otherwise
+     * @return true if event should be skipped. false otherwise
+     * @throws CatalogException
+     */
+    protected boolean shouldSkipWhenSyncingToLatestEventId() throws CatalogException {
+      // always process rename since renameTableFromEvent will make sure that
+      // the old table was removed and new was added
+      if (isRename_) {
+        return false;
+      }
+      return super.shouldSkipWhenSyncingToLatestEventId();
+    }
+
     /**
      * If the ALTER_TABLE event is due a table rename, this method removes the old table
      * and creates a new table with the new name. Else, this just issues a refresh
@@ -1135,6 +1392,7 @@ public class MetastoreEvents {
         processRename();
         return;
       }
+
       // Determine whether this is an event which we have already seen or if it is a new
       // event
       if (isSelfEvent()) {
@@ -1396,13 +1654,13 @@ public class MetastoreEvents {
       }
       Preconditions.checkNotNull(alteredDatabase_);
       // If not self event, copy Db object from event to catalog
-      if (!catalog_.updateDbIfExists(alteredDatabase_)) {
+      if (!catalogOpExecutor_.alterDbIfExists(getEventId(), alteredDatabase_)) {
         // Okay to skip this event. Events processor will not error out.
         debugLog("Update database {} failed as the database is not present in the "
             + "catalog.", alteredDatabase_.getName());
       } else {
-        infoLog("Database {} updated after alter database event.",
-            alteredDatabase_.getName());
+        infoLog("Database {} updated after alter database event id {}",
+            alteredDatabase_.getName(), getEventId());
       }
     }
 
@@ -1410,6 +1668,38 @@ public class MetastoreEvents {
     protected SelfEventContext getSelfEventContext() {
       return new SelfEventContext(dbName_, null, alteredDatabase_.getParameters());
     }
+
+    /**
+     * Skip processing this event if either db does not exist in cache or is already
+     * synced atleast to this event id.
+     * @return
+     * @throws CatalogException
+     */
+    @Override
+    protected boolean shouldSkipWhenSyncingToLatestEventId() throws CatalogException {
+      Preconditions.checkState(
+          BackendConfig.INSTANCE.enableSyncToLatestEventOnDdls(),
+          "sync to latest event id flag should be set");
+      long eventId = this.getEventId();
+      Db db = catalog_.getDb(getDbName());
+      if (db == null) {
+        infoLog("Skipping since db {} does not exist in cache", getDbName());
+        return true;
+      }
+      if (!catalog_.tryLockDb(db)) {
+        throw new CatalogException(String.format("Couldn't acquire lock on db %s",
+            db.getName()));
+      }
+      catalog_.getLock().writeLock().unlock();
+      boolean shouldSkip = false;
+      if (db.getLastSyncedEventId() >= eventId) {
+        infoLog("Skipping on db {} since db is already synced till event id {}",
+            getDbName(), db.getLastSyncedEventId());
+        shouldSkip = true;
+      }
+      db.getLock().unlock();
+      return shouldSkip;
+    }
   }
 
   /**
@@ -1450,6 +1740,11 @@ public class MetastoreEvents {
           + "this event");
     }
 
+    @Override
+    protected boolean shouldSkipWhenSyncingToLatestEventId() {
+      return false;
+    }
+
     /**
      * Process the drop database event. This handler removes the db object from catalog
      * only if the CREATION_TIME of the catalog's database object is lesser than or equal
@@ -2069,7 +2364,7 @@ public class MetastoreEvents {
   }
 
   /**
-   * Metastore event handler for COMMIT_TXN events. Handles commit event for transactinal
+   * Metastore event handler for COMMIT_TXN events. Handles commit event for transactional
    * tables.
    */
   public static class CommitTxnEvent extends MetastoreEvent {
@@ -2155,7 +2450,8 @@ public class MetastoreEvents {
           try {
             catalogOpExecutor_.addCommittedWriteIdsAndReloadPartitionsIfExist(
                 getEventId(), entry.getKey().getDb(), entry.getKey().getTbl(),
-                writeIdsForTable, partsForTable, "CommitTxnEvent");
+                writeIdsForTable, partsForTable, "Processing event id: " +
+                    getEventId() + ", event type: " + getEventType());
           } catch (TableNotLoadedException e) {
             debugLog("Ignoring reloading since table {} is not loaded",
                 entry.getKey());
@@ -2165,7 +2461,7 @@ public class MetastoreEvents {
           }
         } else {
           catalog_.reloadTableIfExists(entry.getKey().getDb(), entry.getKey().getTbl(),
-              getEventId(), "CommitTxnEvent");
+              "CommitTxnEvent", getEventId());
         }
       }
     }
@@ -2180,6 +2476,15 @@ public class MetastoreEvents {
       throw new UnsupportedOperationException("Self-event evaluation is not needed for "
           + "this event type");
     }
+
+    /*
+    Not skipping the event since there can be multiple tables involved. The actual
+    processing of event would skip or process the event on a table by table basis
+     */
+    @Override
+    protected boolean shouldSkipWhenSyncingToLatestEventId() {
+      return false;
+    }
   }
 
   /**
@@ -2218,8 +2523,7 @@ public class MetastoreEvents {
         throws CatalogException {
       for (TableWriteId tableWriteId: tableWriteIds) {
         catalog_.addWriteIdsToTable(tableWriteId.getDbName(), tableWriteId.getTblName(),
-            getEventId(),
-            Collections.singletonList(tableWriteId.getWriteId()),
+            getEventId(), Collections.singletonList(tableWriteId.getWriteId()),
             MutableValidWriteIdList.WriteIdStatus.ABORTED);
       }
     }
@@ -2234,6 +2538,15 @@ public class MetastoreEvents {
       throw new UnsupportedOperationException("Self-event evaluation is not needed for "
           + "this event type");
     }
+
+    /*
+    Not skipping the event since there can be multiple tables involved. The actual
+    processing of event would skip or process the event on a table by table basis
+     */
+    @Override
+    protected boolean shouldSkipWhenSyncingToLatestEventId() {
+      return false;
+    }
   }
 
   /**
@@ -2261,6 +2574,11 @@ public class MetastoreEvents {
     }
 
     @Override
+    protected boolean shouldSkipWhenSyncingToLatestEventId() {
+      return false;
+    }
+
+    @Override
     protected SelfEventContext getSelfEventContext() {
       throw new UnsupportedOperationException("Self-event evaluation is not needed for "
           + "this event type");
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 b089fd9..1c8a795 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
@@ -40,8 +40,12 @@ import org.apache.hadoop.hive.metastore.api.NotificationEventResponse;
 import org.apache.hadoop.hive.metastore.messaging.MessageDeserializer;
 import org.apache.impala.catalog.CatalogException;
 import org.apache.impala.catalog.CatalogServiceCatalog;
+import org.apache.impala.catalog.Db;
 import org.apache.impala.catalog.MetaStoreClientPool.MetaStoreClient;
+import org.apache.impala.catalog.Table;
+import org.apache.impala.catalog.IncompleteTable;
 import org.apache.impala.catalog.events.ConfigValidator.ValidationResult;
+import org.apache.impala.catalog.events.MetastoreEvents.DropDatabaseEvent;
 import org.apache.impala.catalog.events.MetastoreEvents.MetastoreEvent;
 import org.apache.impala.catalog.events.MetastoreEvents.MetastoreEventFactory;
 import org.apache.impala.common.Metrics;
@@ -50,6 +54,7 @@ import org.apache.impala.service.CatalogOpExecutor;
 import org.apache.impala.thrift.TEventProcessorMetrics;
 import org.apache.impala.thrift.TEventProcessorMetricsSummaryResponse;
 import org.apache.impala.util.MetaStoreUtil;
+import org.apache.impala.util.ThreadNameAnnotator;
 import org.apache.thrift.TException;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
@@ -307,6 +312,164 @@ public class MetastoreEventsProcessor implements ExternalEventsProcessor {
     }
   }
 
+  /**
+   * Sync table to latest event id starting from last synced
+   * event id.
+   * @param catalog
+   * @param tbl: Catalog table to be synced
+   * @param eventFactory
+   * @throws CatalogException
+   * @throws MetastoreNotificationException
+   */
+  public static void syncToLatestEventId(CatalogServiceCatalog catalog,
+      org.apache.impala.catalog.Table tbl, EventFactory eventFactory, Metrics metrics)
+      throws CatalogException, MetastoreNotificationException {
+    Preconditions.checkArgument(tbl != null, "tbl is null");
+    Preconditions.checkState(!(tbl instanceof IncompleteTable) &&
+        tbl.isLoaded(), "table %s is either incomplete or not loaded",
+        tbl.getFullName());
+    Preconditions.checkState(tbl.isWriteLockedByCurrentThread(),
+        String.format("Write lock is not held on table %s by current thread",
+            tbl.getFullName()));
+    long lastEventId = tbl.getLastSyncedEventId();
+    Preconditions.checkArgument(lastEventId > 0, "lastEvent " +
+        " Id %s for table %s should be greater than 0", lastEventId, tbl.getFullName());
+
+    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));
+
+      if (events.isEmpty()) {
+        LOG.debug("table {} synced till event id {}. No new HMS events to process from "
+                + "event id: {}", tbl.getFullName(), lastEventId, lastEventId + 1);
+        return;
+      }
+      MetastoreEvents.MetastoreEvent currentEvent = null;
+      for (NotificationEvent event : events) {
+        currentEvent = eventFactory.get(event, metrics);
+        LOG.trace("for table {}, processing event {}", tbl.getFullName(), currentEvent);
+        currentEvent.processIfEnabled();
+        if (currentEvent.isDropEvent()) {
+          // currentEvent can only be DropPartition or DropTable
+          Preconditions.checkNotNull(currentEvent.getDbName());
+          Preconditions.checkNotNull(currentEvent.getTableName());
+          String key = DeleteEventLog.getTblKey(currentEvent.getDbName(),
+              currentEvent.getTableName());
+          catalog.getMetastoreEventProcessor().getDeleteEventLog()
+              .addRemovedObject(currentEvent.getEventId(), key);
+        }
+        if (currentEvent instanceof MetastoreEvents.DropTableEvent) {
+          // return after processing table drop event
+          return;
+        }
+      }
+      // setting HMS Event ID after all the events
+      // are successfully processed only if table was
+      // not dropped
+      // Certain events like alter_table, do an incremental reload which sets the event
+      // id to the current hms event id at that time. Therefore, check table's last
+      // synced event id again before setting currentEvent id
+      if (currentEvent.getEventId() > tbl.getLastSyncedEventId()) {
+        tbl.setLastSyncedEventId(currentEvent.getEventId());
+      }
+      LOG.info("Synced table {} till HMS event:  {}", tbl.getFullName(),
+          tbl.getLastSyncedEventId());
+    }
+  }
+
+  /**
+   * Sync database to latest event id starting from the last synced
+   * event id
+   * @param catalog
+   * @param db
+   * @param eventFactory
+   * @throws CatalogException
+   * @throws MetastoreNotificationException
+   */
+  public static void syncToLatestEventId(CatalogServiceCatalog catalog,
+      org.apache.impala.catalog.Db db, EventFactory eventFactory, Metrics metrics)
+      throws CatalogException, MetastoreNotificationException {
+    Preconditions.checkArgument(db != null, "db is null");
+    long lastEventId = db.getLastSyncedEventId();
+    Preconditions.checkArgument(lastEventId > 0, "Invalid " +
+        "last synced event ID %s for db %s ", lastEventId, db.getName());
+    Preconditions.checkState(db.isLockHeldByCurrentThread(),
+        "Current thread does not hold lock on db: %s", db.getName());
+
+    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));
+
+      if (events.isEmpty()) {
+        LOG.debug("db {} already synced till event id: {}, no new hms events from "
+            + "event id: {}", db.getName(), lastEventId, lastEventId+1);
+        return;
+      }
+
+      MetastoreEvents.MetastoreEvent currentEvent = null;
+      for (NotificationEvent event : events) {
+        currentEvent = eventFactory.get(event, metrics);
+        LOG.trace("for db {}, processing event: {}", db.getName(), currentEvent);
+        currentEvent.processIfEnabled();
+        if (currentEvent.isDropEvent()) {
+          Preconditions.checkState(currentEvent instanceof DropDatabaseEvent,
+              "invalid drop event {} ", currentEvent);
+          Preconditions.checkNotNull(currentEvent.getDbName());
+          String key = DeleteEventLog.getDbKey(currentEvent.getDbName());
+          catalog.getMetastoreEventProcessor().getDeleteEventLog()
+              .addRemovedObject(currentEvent.getEventId(), key);
+          // return after processing drop db event
+          return;
+        }
+      }
+      // setting HMS Event Id after all the events
+      // are successfully processed only if db was not dropped
+      db.setLastSyncedEventId(currentEvent.getEventId());
+      LOG.info("Synced db {} till HMS event {}", db.getName(),
+          currentEvent);
+    }
+  }
+
+  /*
+  This filter is used when syncing events for a table to the latest HMS event id.
+  It filters all events except db related ones.
+   */
+  private static NotificationFilter getTableNotificationEventFilter(Table tbl) {
+    NotificationFilter filter = new NotificationFilter() {
+      @Override
+      public boolean accept(NotificationEvent event) {
+        if (event.getDbName() != null && event.getTableName() != null) {
+          return tbl.getDb().getName().equalsIgnoreCase(event.getDbName()) &&
+              tbl.getName().equalsIgnoreCase(event.getTableName());
+        }
+        // filter all except db events
+        return event.getDbName() == null;
+      }
+    };
+    return filter;
+  }
+
+  /*
+  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) {
+    NotificationFilter filter = new NotificationFilter() {
+      @Override
+      public boolean accept(NotificationEvent event) {
+        if (event.getDbName() != null && event.getTableName() == null) {
+          return db.getName().equalsIgnoreCase(event.getDbName());
+        }
+        // filter all events except table events
+        return event.getTableName() == null;
+      }
+    };
+    return filter;
+  }
+
   // possible status of event processor
   public enum EventProcessorStatus {
     PAUSED, // event processor is paused because catalog is being reset concurrently
@@ -355,9 +518,9 @@ public class MetastoreEventsProcessor implements ExternalEventsProcessor {
     this.catalog_ = Preconditions.checkNotNull(catalogOpExecutor.getCatalog());
     validateConfigs();
     lastSyncedEventId_.set(startSyncFromId);
-    metastoreEventFactory_ = new MetastoreEventFactory(catalogOpExecutor, metrics_);
-    pollingFrequencyInSec_ = pollingFrequencyInSec;
     initMetrics();
+    metastoreEventFactory_ = new MetastoreEventFactory(catalogOpExecutor);
+    pollingFrequencyInSec_ = pollingFrequencyInSec;
   }
 
   /**
@@ -755,7 +918,7 @@ public class MetastoreEventsProcessor implements ExternalEventsProcessor {
         metrics_.getTimer(EVENTS_PROCESS_DURATION_METRIC).time();
     try {
       List<MetastoreEvent> filteredEvents =
-          metastoreEventFactory_.getFilteredEvents(events);
+          metastoreEventFactory_.getFilteredEvents(events, metrics_);
       if (filteredEvents.isEmpty()) {
         lastSyncedEventId_.set(events.get(events.size() - 1).getEventId());
         return;
diff --git a/fe/src/main/java/org/apache/impala/catalog/events/NoOpEventProcessor.java b/fe/src/main/java/org/apache/impala/catalog/events/NoOpEventProcessor.java
index 991cd5f..b0653e8 100644
--- a/fe/src/main/java/org/apache/impala/catalog/events/NoOpEventProcessor.java
+++ b/fe/src/main/java/org/apache/impala/catalog/events/NoOpEventProcessor.java
@@ -99,7 +99,7 @@ public class NoOpEventProcessor implements ExternalEventsProcessor {
 
   @Override
   public EventFactory getEventsFactory() {
-    return hmsEvent -> null;
+    return (hmsEvent, metrics) -> null;
   }
 
   @Override
diff --git a/fe/src/main/java/org/apache/impala/catalog/metastore/CatalogMetastoreServiceHandler.java b/fe/src/main/java/org/apache/impala/catalog/metastore/CatalogMetastoreServiceHandler.java
index 2fe4942..cdd4226 100644
--- a/fe/src/main/java/org/apache/impala/catalog/metastore/CatalogMetastoreServiceHandler.java
+++ b/fe/src/main/java/org/apache/impala/catalog/metastore/CatalogMetastoreServiceHandler.java
@@ -19,42 +19,79 @@ package org.apache.impala.catalog.metastore;
 
 import static org.apache.impala.catalog.metastore.HmsApiNameEnum.GET_PARTITION_BY_NAMES;
 
-import org.apache.hadoop.hive.metastore.api.GetFieldsRequest;
-import org.apache.hadoop.hive.metastore.api.GetFieldsResponse;
+import com.facebook.fb303.fb_status;
+import com.google.common.base.Preconditions;
+import java.util.List;
+import java.util.Map;
+
+import org.apache.commons.lang.exception.ExceptionUtils;
+import org.apache.hadoop.hive.metastore.PartitionExpressionProxy;
+import org.apache.hadoop.hive.metastore.api.AddPartitionsRequest;
+import org.apache.hadoop.hive.metastore.api.AddPartitionsResult;
+import org.apache.hadoop.hive.metastore.api.AlreadyExistsException;
+import org.apache.hadoop.hive.metastore.api.AlterPartitionsRequest;
+import org.apache.hadoop.hive.metastore.api.AlterPartitionsResponse;
+import org.apache.hadoop.hive.metastore.api.AlterTableRequest;
+import org.apache.hadoop.hive.metastore.api.AlterTableResponse;
+import org.apache.hadoop.hive.metastore.api.CreateTableRequest;
+import org.apache.hadoop.hive.metastore.api.CurrentNotificationEventId;
+import org.apache.hadoop.hive.metastore.api.Database;
+import org.apache.hadoop.hive.metastore.api.DropPartitionsRequest;
+import org.apache.hadoop.hive.metastore.api.DropPartitionsResult;
+import org.apache.hadoop.hive.metastore.api.EnvironmentContext;
 import org.apache.hadoop.hive.metastore.api.GetPartitionsByNamesRequest;
 import org.apache.hadoop.hive.metastore.api.GetPartitionsByNamesResult;
-import org.apache.hadoop.hive.metastore.api.GetPartitionNamesPsResponse;
-import org.apache.hadoop.hive.metastore.api.GetPartitionNamesPsRequest;
-import org.apache.hadoop.hive.metastore.api.GetPartitionRequest;
-import org.apache.hadoop.hive.metastore.api.GetPartitionResponse;
-import org.apache.hadoop.hive.metastore.api.GetPartitionsPsWithAuthRequest;
-import org.apache.hadoop.hive.metastore.api.GetPartitionsPsWithAuthResponse;
-import org.apache.hadoop.hive.metastore.api.GetSchemaRequest;
-import org.apache.hadoop.hive.metastore.api.GetSchemaResponse;
+import org.apache.hadoop.hive.metastore.api.GetPartitionsRequest;
 import org.apache.hadoop.hive.metastore.api.GetTableRequest;
 import org.apache.hadoop.hive.metastore.api.GetTableResult;
-import org.apache.hadoop.hive.metastore.api.MaxAllocatedTableWriteIdRequest;
-import org.apache.hadoop.hive.metastore.api.MaxAllocatedTableWriteIdResponse;
+import org.apache.hadoop.hive.metastore.api.GetTablesRequest;
+import org.apache.hadoop.hive.metastore.api.GetTablesResult;
+import org.apache.hadoop.hive.metastore.api.InvalidObjectException;
+import org.apache.hadoop.hive.metastore.api.InvalidOperationException;
 import org.apache.hadoop.hive.metastore.api.MetaException;
+import org.apache.hadoop.hive.metastore.api.NoSuchLockException;
 import org.apache.hadoop.hive.metastore.api.NoSuchObjectException;
+import org.apache.hadoop.hive.metastore.api.NotificationEvent;
+import org.apache.hadoop.hive.metastore.api.Partition;
+import org.apache.hadoop.hive.metastore.api.PartitionSpec;
 import org.apache.hadoop.hive.metastore.api.PartitionsByExprRequest;
 import org.apache.hadoop.hive.metastore.api.PartitionsByExprResult;
 import org.apache.hadoop.hive.metastore.api.PartitionsRequest;
 import org.apache.hadoop.hive.metastore.api.PartitionsResponse;
-import org.apache.hadoop.hive.metastore.api.SeedTableWriteIdsRequest;
-import org.apache.hadoop.hive.metastore.api.SeedTxnIdRequest;
-import org.apache.impala.catalog.CatalogException;
+import org.apache.hadoop.hive.metastore.api.RenamePartitionRequest;
+import org.apache.hadoop.hive.metastore.api.RenamePartitionResponse;
+import org.apache.hadoop.hive.metastore.api.SQLCheckConstraint;
+import org.apache.hadoop.hive.metastore.api.SQLDefaultConstraint;
+import org.apache.hadoop.hive.metastore.api.SQLForeignKey;
+import org.apache.hadoop.hive.metastore.api.SQLNotNullConstraint;
+import org.apache.hadoop.hive.metastore.api.SQLPrimaryKey;
+import org.apache.hadoop.hive.metastore.api.SQLUniqueConstraint;
+import org.apache.hadoop.hive.metastore.api.Table;
+import org.apache.hadoop.hive.metastore.api.TruncateTableRequest;
+import org.apache.hadoop.hive.metastore.api.TruncateTableResponse;
+import org.apache.hadoop.hive.metastore.utils.MetaStoreUtils;
 import org.apache.impala.catalog.CatalogHmsAPIHelper;
-import org.apache.impala.catalog.CatalogServiceCatalog;
+import org.apache.impala.catalog.events.DeleteEventLog;
+import org.apache.impala.catalog.events.EventFactory;
+import org.apache.impala.catalog.events.MetastoreEvents;
+import org.apache.impala.catalog.events.MetastoreEvents.CreateTableEvent;
+import org.apache.impala.catalog.events.MetastoreEvents.MetastoreEvent;
+import org.apache.impala.catalog.events.MetastoreEvents.MetastoreEventFactory;
+import org.apache.impala.catalog.events.MetastoreEventsProcessor;
+import org.apache.impala.common.ImpalaException;
 import org.apache.impala.common.Metrics;
-import org.apache.impala.catalog.MetaStoreClientPool.MetaStoreClient;
-import org.apache.impala.compat.MetastoreShim;
+import org.apache.impala.common.Pair;
 import org.apache.impala.service.BackendConfig;
 import org.apache.impala.service.CatalogOpExecutor;
+import org.apache.impala.thrift.TTableName;
 import org.apache.thrift.TException;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
+import org.apache.impala.catalog.Db;
+import org.apache.impala.catalog.CatalogException;
+import org.apache.impala.compat.MetastoreShim;
+
 /**
  * This class implements the HMS APIs that are served by CatalogD
  * and is exposed via {@link CatalogMetastoreServer}.
@@ -66,6 +103,11 @@ public class CatalogMetastoreServiceHandler extends MetastoreServiceHandler {
   private static final Logger LOG = LoggerFactory
       .getLogger(CatalogMetastoreServiceHandler.class);
 
+  private final String SYNC_TABLE_LATEST_EVENT_ID_ERR_MSG = "Failed to sync table %s "
+      + "to latest event id while executing %s";
+  private final String SYNC_DB_LATEST_EVENT_ID_ERR_MSG = "Failed to sync db %s to " +
+          "latest event id while executing %s";
+
   public CatalogMetastoreServiceHandler(CatalogOpExecutor catalogOpExecutor,
       boolean fallBackToHMSOnErrors) {
     super(catalogOpExecutor, fallBackToHMSOnErrors);
@@ -102,7 +144,6 @@ public class CatalogMetastoreServiceHandler extends MetastoreServiceHandler {
   @Override
   public PartitionsByExprResult get_partitions_by_expr(
       PartitionsByExprRequest partitionsByExprRequest) throws TException {
-
     if (!BackendConfig.INSTANCE.enableCatalogdHMSCache()) {
       return super.get_partitions_by_expr(partitionsByExprRequest);
     }
@@ -139,7 +180,6 @@ public class CatalogMetastoreServiceHandler extends MetastoreServiceHandler {
   @Override
   public GetPartitionsByNamesResult get_partitions_by_names_req(
       GetPartitionsByNamesRequest getPartitionsByNamesRequest) throws TException {
-
     if (!BackendConfig.INSTANCE.enableCatalogdHMSCache()) {
       return super.get_partitions_by_names_req(getPartitionsByNamesRequest);
     }
@@ -156,4 +196,1225 @@ public class CatalogMetastoreServiceHandler extends MetastoreServiceHandler {
     LOG.info(String.format(HMS_FALLBACK_MSG_FORMAT, GET_PARTITION_BY_NAMES, tblName));
     return super.get_partitions_by_names_req(getPartitionsByNamesRequest);
   }
+
+  @Override
+  public void create_database(Database msDb)
+      throws AlreadyExistsException, InvalidObjectException, MetaException, TException {
+    if (!BackendConfig.INSTANCE.enableCatalogdHMSCache() ||
+            !BackendConfig.INSTANCE.enableSyncToLatestEventOnDdls()) {
+      super.create_database(msDb);
+      return;
+    }
+    catalogOpExecutor_.getMetastoreDdlLock().lock();
+    String dbName = msDb.getName();
+    String apiName = HmsApiNameEnum.CREATE_DATABASE.apiName();
+    Db db = null;
+    long fromEventId = -1;
+    try {
+      try {
+        fromEventId = super.get_current_notificationEventId().getEventId();
+        super.create_database(msDb);
+      } catch (Exception e) {
+        LOG.error("Caught exception when creating database {} in hms", dbName);
+        if (!(e instanceof AlreadyExistsException)) {
+          throw e;
+        }
+        if (catalog_.getDb(dbName) != null) {
+          LOG.error("can not create database {} as it already exists in " +
+                  "metastore and catalog", dbName);
+          throw e;
+        }
+      }
+      List<NotificationEvent> events =
+          MetastoreEventsProcessor.getNextMetastoreEventsInBatches(catalog_, fromEventId,
+              notificationEvent ->
+                  MetastoreEvents.CreateDatabaseEvent.CREATE_DATABASE_EVENT_TYPE
+                      .equals(notificationEvent.getEventType())
+                      && dbName.equalsIgnoreCase(notificationEvent.getDbName()));
+
+      Preconditions.checkArgument(events.size() == 1,
+          "Db %s was recreated in metastore " +
+              "while the current db creation was in progress", dbName);
+      long createEventId = events.get(0).getEventId();
+      catalog_.addDb(dbName, msDb, createEventId);
+      LOG.info("Created database {} with create event id: {}", dbName, createEventId);
+      // sync to latest event ID
+      db = getDbAndAcquireLock(dbName, apiName);
+      syncToLatestEventId(db, apiName);
+    } catch (Exception e) {
+      rethrowException(e, apiName);
+    } finally {
+      catalogOpExecutor_.UnlockWriteLockIfErronouslyLocked();
+      catalogOpExecutor_.getMetastoreDdlLock().unlock();
+      if (db != null && db.isLockHeldByCurrentThread()) {
+        db.getLock().unlock();
+      }
+    }
+  }
+
+  @Override
+  public void drop_database(String databaseName, boolean deleteData,
+      boolean ignoreUnknownDb) throws NoSuchObjectException,
+          InvalidOperationException, MetaException, TException {
+    if (!BackendConfig.INSTANCE.enableCatalogdHMSCache() ||
+            !BackendConfig.INSTANCE.enableSyncToLatestEventOnDdls()) {
+      super.drop_database(databaseName, deleteData, ignoreUnknownDb);
+      return;
+    }
+    // TODO: The complete logic can be moved to
+    // drop_database in MetastoreserviceHandler
+    String apiName = HmsApiNameEnum.DROP_DATABASE.apiName();
+    String dbName = MetaStoreUtils.parseDbName(databaseName, serverConf_)[1];
+    long currentEventId = -1;
+    catalogOpExecutor_.getMetastoreDdlLock().lock();
+    try  {
+      try {
+        currentEventId = super.get_current_notificationEventId().getEventId();
+        super.drop_database(databaseName, deleteData, ignoreUnknownDb);
+      } catch (NoSuchObjectException e) {
+        // db does not exist in metastore, remove it from
+        // catalog if exists
+        if (catalog_.removeDb(dbName) != null) {
+          LOG.info("Db {} not known to metastore, removed it from catalog for " +
+              "metastore api {}", dbName, apiName);
+        }
+        throw e;
+      }
+      dropDbIfExists(databaseName, ignoreUnknownDb, currentEventId, apiName);
+    } finally {
+      catalogOpExecutor_.getMetastoreDdlLock().unlock();
+    }
+  }
+
+  @Override
+  public void alter_database(String databaseName, Database database)
+      throws MetaException, NoSuchObjectException, TException {
+    if (!BackendConfig.INSTANCE.enableCatalogdHMSCache() ||
+        !BackendConfig.INSTANCE.enableSyncToLatestEventOnDdls()) {
+      super.alter_database(databaseName, database);
+      return;
+    }
+    String dbname = MetaStoreUtils.parseDbName(databaseName, serverConf_)[1];
+    String apiName = HmsApiNameEnum.ALTER_DATABASE.apiName();
+    Db db = getDbAndAcquireLock(dbname, apiName);
+    catalog_.getLock().writeLock().unlock();
+    try {
+      super.alter_database(dbname, database);
+      syncToLatestEventId(db, apiName);
+    } catch (Exception e) {
+      rethrowException(e, apiName);
+    } finally {
+      catalogOpExecutor_.UnlockWriteLockIfErronouslyLocked();
+      if (db != null && db.isLockHeldByCurrentThread()) {
+        db.getLock().unlock();
+      }
+    }
+  }
+
+  @Override
+  public void create_table(org.apache.hadoop.hive.metastore.api.Table table)
+      throws TException {
+    if (!BackendConfig.INSTANCE.enableCatalogdHMSCache() ||
+        !BackendConfig.INSTANCE.enableSyncToLatestEventOnDdls()) {
+      super.create_table(table);
+      return;
+    }
+    CreateTableTask<Void> task = new CreateTableTask<Void>() {
+      @Override
+      public Void execute() throws TException {
+        CatalogMetastoreServiceHandler.super.create_table(table);
+        return null;
+      }
+    };
+    String apiName = HmsApiNameEnum.CREATE_TABLE.apiName();
+    createTableCore(table.getDbName(), table.getTableName(), apiName, task);
+  }
+
+  @Override
+  public void create_table_req(CreateTableRequest req)
+      throws TException {
+    if (!BackendConfig.INSTANCE.enableCatalogdHMSCache() ||
+        !BackendConfig.INSTANCE.enableSyncToLatestEventOnDdls()) {
+      super.create_table_req(req);
+      return;
+    }
+    CreateTableTask<Void> task = new CreateTableTask<Void>() {
+      @Override
+      public Void execute() throws TException {
+        CatalogMetastoreServiceHandler.super.create_table_req(req);
+        return null;
+      }
+    };
+    String apiName = HmsApiNameEnum.CREATE_TABLE_REQ.apiName();
+    org.apache.hadoop.hive.metastore.api.Table table = req.getTable();
+    createTableCore(table.getDbName(), table.getTableName(), apiName, task);
+  }
+
+
+  @Override
+  public void create_table_with_constraints(
+      org.apache.hadoop.hive.metastore.api.Table table,
+      List<SQLPrimaryKey> sqlPrimaryKeys, List<SQLForeignKey> sqlForeignKeys,
+      List<SQLUniqueConstraint> sqlUniqueConstraints,
+      List<SQLNotNullConstraint> sqlNotNullConstraints,
+      List<SQLDefaultConstraint> sqlDefaultConstraints,
+      List<SQLCheckConstraint> sqlCheckConstraints) throws TException {
+    if (!BackendConfig.INSTANCE.enableCatalogdHMSCache() ||
+        !BackendConfig.INSTANCE.enableSyncToLatestEventOnDdls()) {
+      super.create_table_with_constraints(table,
+          sqlPrimaryKeys, sqlForeignKeys, sqlUniqueConstraints, sqlNotNullConstraints,
+          sqlDefaultConstraints, sqlCheckConstraints);
+      return;
+    }
+    CreateTableTask<Void> task = new CreateTableTask<Void>() {
+      @Override
+      public Void execute() throws TException {
+        CatalogMetastoreServiceHandler.super.create_table_with_constraints(table,
+            sqlPrimaryKeys, sqlForeignKeys, sqlUniqueConstraints, sqlNotNullConstraints,
+            sqlDefaultConstraints, sqlCheckConstraints);
+        return null;
+      }
+    };
+    String apiName = HmsApiNameEnum.CREATE_TABLE_WITH_CONSTRAINTS.apiName();
+    createTableCore(table.getDbName(), table.getTableName(), apiName, task);
+  }
+
+  @Override
+  public void create_table_with_environment_context(
+      org.apache.hadoop.hive.metastore.api.Table table,
+      EnvironmentContext environmentContext) throws TException {
+    if (!BackendConfig.INSTANCE.enableCatalogdHMSCache() ||
+        !BackendConfig.INSTANCE.enableSyncToLatestEventOnDdls()) {
+      super.create_table_with_environment_context(table, environmentContext);
+      return;
+    }
+    CreateTableTask<Void> task = new CreateTableTask<Void>() {
+      @Override
+      public Void execute() throws TException {
+        CatalogMetastoreServiceHandler.super
+            .create_table_with_environment_context(table, environmentContext);
+        return null;
+      }
+    };
+    String apiName = HmsApiNameEnum.CREATE_TABLE_WITH_ENVIRONMENT_CONTEXT.apiName();
+    createTableCore(table.getDbName(), table.getTableName(), apiName, task);
+  }
+
+  @Override
+  public void alter_table(String dbname, String tblName,
+      org.apache.hadoop.hive.metastore.api.Table newTable)
+      throws InvalidOperationException, MetaException, TException {
+
+    if (!BackendConfig.INSTANCE.enableCatalogdHMSCache() ||
+        !BackendConfig.INSTANCE.enableSyncToLatestEventOnDdls()) {
+      super.alter_table(dbname, tblName, newTable);
+      return;
+    }
+
+    AlterTableTask<Void> task = new AlterTableTask<Void>() {
+      @Override
+      public Void execute() throws InvalidOperationException, MetaException,
+          TException {
+        CatalogMetastoreServiceHandler.super.alter_table(dbname, tblName, newTable);
+        return null;
+      }
+    };
+    String apiName = HmsApiNameEnum.ALTER_TABLE.apiName();
+    alterTableCore(dbname, tblName, newTable, apiName, task);
+  }
+
+  @Override
+  public void alter_table_with_environment_context(String dbname, String tblName,
+      org.apache.hadoop.hive.metastore.api.Table table, EnvironmentContext envContext)
+      throws InvalidOperationException, MetaException, TException {
+    if (!BackendConfig.INSTANCE.enableCatalogdHMSCache() ||
+        !BackendConfig.INSTANCE.enableSyncToLatestEventOnDdls()) {
+      super.alter_table_with_environment_context(dbname, tblName, table, envContext);
+      return;
+    }
+    String apiName = HmsApiNameEnum.ALTER_PARTITION_WITH_ENVIRONMENT_CONTEXT.apiName();
+    AlterTableTask<Void> task = new AlterTableTask<Void>() {
+      @Override
+      public Void execute() throws InvalidOperationException, MetaException, TException {
+        CatalogMetastoreServiceHandler.super.alter_table_with_environment_context(dbname,
+            tblName, table, envContext);
+        return null;
+      }
+    };
+    alterTableCore(dbname, tblName, table, apiName, task);
+  }
+
+
+  @Override
+  public void alter_table_with_cascade(String dbname, String tblName,
+      org.apache.hadoop.hive.metastore.api.Table table, boolean cascade)
+      throws InvalidOperationException, MetaException, TException {
+    if (!BackendConfig.INSTANCE.enableCatalogdHMSCache() ||
+        !BackendConfig.INSTANCE.enableSyncToLatestEventOnDdls()) {
+      super.alter_table_with_cascade(dbname, tblName, table, cascade);
+      return;
+    }
+    String apiName = HmsApiNameEnum.ALTER_TABLE_WITH_CASCADE.apiName();
+
+    AlterTableTask<Void> task = new AlterTableTask<Void>() {
+      @Override
+      public Void execute() throws InvalidOperationException, MetaException,
+          TException {
+        CatalogMetastoreServiceHandler.super
+            .alter_table_with_cascade(dbname, tblName, table, cascade);
+        return null;
+      }
+    };
+    alterTableCore(dbname, tblName, table, apiName, task);
+  }
+
+
+  @Override
+  public AlterTableResponse alter_table_req(AlterTableRequest alterTableRequest)
+          throws InvalidOperationException, MetaException, TException {
+    if (!BackendConfig.INSTANCE.enableCatalogdHMSCache() ||
+        !BackendConfig.INSTANCE.enableSyncToLatestEventOnDdls()) {
+      return super.alter_table_req(alterTableRequest);
+    }
+    String apiName = HmsApiNameEnum.ALTER_TABLE_REQ.apiName();
+    String dbname = alterTableRequest.getDbName();
+    String tblName = alterTableRequest.getTableName();
+    org.apache.hadoop.hive.metastore.api.Table newTable =
+        alterTableRequest.getTable();
+    AlterTableTask<AlterTableResponse> task = new AlterTableTask<AlterTableResponse>() {
+      @Override
+      public AlterTableResponse execute() throws InvalidOperationException, MetaException,
+          TException {
+        AlterTableResponse resp =
+            CatalogMetastoreServiceHandler.super.alter_table_req(alterTableRequest);
+        return resp;
+      }
+    };
+    return alterTableCore(dbname, tblName, newTable, apiName, task);
+  }
+
+  @Override
+  public Partition add_partition(Partition partition)
+      throws InvalidObjectException, AlreadyExistsException, MetaException, TException {
+    if (!BackendConfig.INSTANCE.enableCatalogdHMSCache() ||
+        !BackendConfig.INSTANCE.enableSyncToLatestEventOnDdls()) {
+      return super.add_partition(partition);
+    }
+    String apiName = HmsApiNameEnum.ADD_PARTITION.apiName();
+    org.apache.impala.catalog.Table tbl = getTableAndAcquireWriteLock(
+        partition.getDbName(), partition.getTableName(), apiName);
+    catalog_.getLock().writeLock().unlock();
+    Partition addedPartition = super.add_partition(partition);
+    try {
+      syncToLatestEventId(tbl, apiName);
+    } finally {
+      catalogOpExecutor_.UnlockWriteLockIfErronouslyLocked();
+      tbl.releaseWriteLock();
+    }
+    return addedPartition;
+  }
+
+  @Override
+  public Partition add_partition_with_environment_context(Partition partition,
+      EnvironmentContext environmentContext) throws InvalidObjectException,
+      AlreadyExistsException, MetaException, TException {
+    if (!BackendConfig.INSTANCE.enableCatalogdHMSCache() ||
+        !BackendConfig.INSTANCE.enableSyncToLatestEventOnDdls()) {
+      return super.add_partition_with_environment_context(partition, environmentContext);
+    }
+    String apiName = HmsApiNameEnum.ADD_PARTITION_WITH_ENVIRONMENT_CONTEXT.apiName();
+    org.apache.impala.catalog.Table tbl =
+        getTableAndAcquireWriteLock(partition.getDbName(), partition.getTableName(),
+            apiName);
+    catalog_.getLock().writeLock().unlock();
+    Partition addedPartition =
+        super.add_partition_with_environment_context(partition, environmentContext);
+    try {
+      syncToLatestEventId(tbl, apiName);
+    } finally {
+      catalogOpExecutor_.UnlockWriteLockIfErronouslyLocked();
+      tbl.releaseWriteLock();
+    }
+    return addedPartition;
+  }
+
+  @Override
+  public int add_partitions(List<Partition> partitionList)
+      throws InvalidObjectException, AlreadyExistsException, MetaException, TException {
+    if (!BackendConfig.INSTANCE.enableCatalogdHMSCache() ||
+        !BackendConfig.INSTANCE.enableSyncToLatestEventOnDdls() ||
+        partitionList.isEmpty()) {
+      return super.add_partitions(partitionList);
+    }
+    // don't execute the following if partition list is empty
+    // since we can't get db and table info from empty list
+    String apiName = HmsApiNameEnum.ADD_PARTITIONS.apiName();
+    org.apache.hadoop.hive.metastore.api.Partition partition = partitionList.get(0);
+    org.apache.impala.catalog.Table tbl =
+        getTableAndAcquireWriteLock(partition.getDbName(), partition.getTableName(),
+            apiName);
+    catalog_.getLock().writeLock().unlock();
+    int numPartitionsAdded = super.add_partitions(partitionList);
+
+    try {
+      syncToLatestEventId(tbl, apiName);
+    } finally {
+      catalogOpExecutor_.UnlockWriteLockIfErronouslyLocked();
+      tbl.releaseWriteLock();
+    }
+    return numPartitionsAdded;
+  }
+
+  @Override
+  public int add_partitions_pspec(List<PartitionSpec> list)
+      throws InvalidObjectException, AlreadyExistsException, MetaException, TException {
+    if (!BackendConfig.INSTANCE.enableCatalogdHMSCache() ||
+        !BackendConfig.INSTANCE.enableSyncToLatestEventOnDdls() ||
+        list.isEmpty()) {
+      return super.add_partitions_pspec(list);
+    }
+    // don't execute the following if partition list is empty
+    // since we can't get db and table info from empty list
+    String apiName = HmsApiNameEnum.ADD_PARTITIONS_PSPEC.apiName();
+    org.apache.hadoop.hive.metastore.api.PartitionSpec spec = list.get(0);
+    org.apache.impala.catalog.Table tbl =
+        getTableAndAcquireWriteLock(spec.getDbName(), spec.getTableName(),
+            apiName);
+    catalog_.getLock().writeLock().unlock();
+    int numPartitionsAdded = super.add_partitions_pspec(list);
+    try {
+      syncToLatestEventId(tbl, apiName);
+    } finally {
+      catalogOpExecutor_.UnlockWriteLockIfErronouslyLocked();
+      tbl.releaseWriteLock();
+    }
+    return numPartitionsAdded;
+  }
+
+  @Override
+  public AddPartitionsResult add_partitions_req(AddPartitionsRequest request)
+      throws InvalidObjectException, AlreadyExistsException, MetaException, TException {
+    if (!BackendConfig.INSTANCE.enableCatalogdHMSCache() ||
+        !BackendConfig.INSTANCE.enableSyncToLatestEventOnDdls()) {
+      return super.add_partitions_req(request);
+    }
+    String apiName = HmsApiNameEnum.ADD_PARTITIONS_REQ.apiName();
+    org.apache.impala.catalog.Table tbl =
+        getTableAndAcquireWriteLock(request.getDbName(), request.getTblName(),
+            apiName);
+    catalog_.getLock().writeLock().unlock();
+    AddPartitionsResult result = super.add_partitions_req(request);
+
+    try {
+      syncToLatestEventId(tbl, apiName);
+    } finally {
+      catalogOpExecutor_.UnlockWriteLockIfErronouslyLocked();
+      tbl.releaseWriteLock();
+    }
+    return result;
+  }
+
+  @Override
+  public Partition append_partition(String dbname, String tblName, List<String> partVals)
+      throws InvalidObjectException, AlreadyExistsException, MetaException, TException {
+    if (!BackendConfig.INSTANCE.enableCatalogdHMSCache() ||
+        !BackendConfig.INSTANCE.enableSyncToLatestEventOnDdls()) {
+      return super.append_partition(dbname, tblName, partVals);
+    }
+    String apiName = HmsApiNameEnum.APPEND_PARTITION.apiName();
+    org.apache.impala.catalog.Table tbl = getTableAndAcquireWriteLock(dbname,
+        tblName, apiName);
+    catalog_.getLock().writeLock().unlock();
+    Partition partition = super.append_partition(dbname, tblName, partVals);
+    LOG.debug("Successfully executed HMS API: append_partition");
+    try {
+      syncToLatestEventId(tbl, apiName);
+    } finally {
+      catalogOpExecutor_.UnlockWriteLockIfErronouslyLocked();
+      tbl.releaseWriteLock();
+    }
+    return partition;
+  }
+
+  @Override
+  public Partition append_partition_with_environment_context(String dbName,
+      String tblName, List<String> partVals, EnvironmentContext environmentContext)
+      throws InvalidObjectException, AlreadyExistsException, MetaException, TException {
+    if (!BackendConfig.INSTANCE.enableCatalogdHMSCache() ||
+        !BackendConfig.INSTANCE.enableSyncToLatestEventOnDdls()) {
+      return super.append_partition_with_environment_context(dbName, tblName, partVals,
+          environmentContext);
+    }
+    String apiName = HmsApiNameEnum.APPEND_PARTITION_WITH_ENVIRONMENT_CONTEXT.apiName();
+    org.apache.impala.catalog.Table tbl = getTableAndAcquireWriteLock(dbName,
+        tblName, apiName);
+    catalog_.getLock().writeLock().unlock();
+    Partition partition = super.append_partition_with_environment_context(dbName,
+        tblName, partVals, environmentContext);
+    try {
+      syncToLatestEventId(tbl, apiName);
+    } finally {
+      catalogOpExecutor_.UnlockWriteLockIfErronouslyLocked();
+      tbl.releaseWriteLock();
+    }
+    return partition;
+  }
+
+  @Override
+  public Partition append_partition_by_name(String dbName, String tblName,
+      String partName) throws InvalidObjectException, AlreadyExistsException,
+      MetaException, TException {
+    if (!BackendConfig.INSTANCE.enableCatalogdHMSCache() ||
+        !BackendConfig.INSTANCE.enableSyncToLatestEventOnDdls()) {
+      return super.append_partition_by_name(dbName, tblName, partName);
+    }
+    String apiName = HmsApiNameEnum.APPEND_PARTITION_WITH_ENVIRONMENT_CONTEXT.apiName();
+    org.apache.impala.catalog.Table tbl = getTableAndAcquireWriteLock(dbName,
+        tblName, apiName);
+    catalog_.getLock().writeLock().unlock();
+    Partition partition = super.append_partition_by_name(dbName,
+        tblName, partName);
+    try {
+      syncToLatestEventId(tbl, apiName);
+    } finally {
+      catalogOpExecutor_.UnlockWriteLockIfErronouslyLocked();
+      tbl.releaseWriteLock();
+    }
+    return partition;
+  }
+
+  @Override
+  public Partition append_partition_by_name_with_environment_context(String dbName,
+      String tblName, String partName, EnvironmentContext context)
+      throws InvalidObjectException, AlreadyExistsException, MetaException, TException {
+    if (!BackendConfig.INSTANCE.enableCatalogdHMSCache() ||
+        !BackendConfig.INSTANCE.enableSyncToLatestEventOnDdls()) {
+      return super.append_partition_by_name_with_environment_context(dbName, tblName,
+          partName, context);
+    }
+    String apiName =
+        HmsApiNameEnum.APPEND_PARTITION_BY_NAME_WITH_ENVIRONMENT_CONTEXT.apiName();
+    org.apache.impala.catalog.Table tbl = getTableAndAcquireWriteLock(dbName,
+        tblName, apiName);
+    catalog_.getLock().writeLock().unlock();
+    Partition partition = super.append_partition_by_name_with_environment_context(dbName,
+        tblName, partName, context);
+    try {
+      syncToLatestEventId(tbl, apiName);
+    } finally {
+      catalogOpExecutor_.UnlockWriteLockIfErronouslyLocked();
+      tbl.releaseWriteLock();
+    }
+    return partition;
+  }
+
+  @Override
+  public boolean drop_partition(String dbname, String tblname, List<String> partVals,
+      boolean deleteData) throws NoSuchObjectException, MetaException, TException {
+    if (!BackendConfig.INSTANCE.enableCatalogdHMSCache() ||
+        !BackendConfig.INSTANCE.enableSyncToLatestEventOnDdls()) {
+      return super.drop_partition(dbname, tblname, partVals, deleteData);
+    }
+    String apiName = HmsApiNameEnum.DROP_PARTITION.apiName();
+    org.apache.impala.catalog.Table tbl = getTableAndAcquireWriteLock(dbname,
+        tblname, apiName);
+    catalog_.getLock().writeLock().unlock();
+    boolean resp = super.drop_partition(dbname, tblname, partVals, deleteData);
+    try {
+      syncToLatestEventId(tbl, apiName);
+    } finally {
+      catalogOpExecutor_.UnlockWriteLockIfErronouslyLocked();
+      tbl.releaseWriteLock();
+    }
+    return resp;
+  }
+
+  @Override
+  public boolean drop_partition_by_name(String dbname, String tblname, String partName,
+      boolean deleteData) throws NoSuchObjectException, MetaException, TException {
+    if (!BackendConfig.INSTANCE.enableCatalogdHMSCache() ||
+        !BackendConfig.INSTANCE.enableSyncToLatestEventOnDdls()) {
+      return super.drop_partition_by_name(dbname, tblname, partName, deleteData);
+    }
+    String apiName = HmsApiNameEnum.DROP_PARTITION_BY_NAME.apiName();
+    org.apache.impala.catalog.Table tbl = getTableAndAcquireWriteLock(dbname,
+        tblname, apiName);
+    catalog_.getLock().writeLock().unlock();
+    boolean response =
+        super.drop_partition_by_name(dbname, tblname, partName, deleteData);
+    try {
+      syncToLatestEventId(tbl, apiName);
+    } finally {
+      catalogOpExecutor_.UnlockWriteLockIfErronouslyLocked();
+      tbl.releaseWriteLock();
+    }
+    return response;
+  }
+
+  @Override
+  public boolean drop_partition_with_environment_context(String dbname, String tblname,
+      List<String> partNames, boolean deleteData, EnvironmentContext environmentContext)
+      throws NoSuchObjectException, MetaException, TException {
+    if (!BackendConfig.INSTANCE.enableCatalogdHMSCache() ||
+        !BackendConfig.INSTANCE.enableSyncToLatestEventOnDdls()) {
+      return super.drop_partition_with_environment_context(dbname, tblname, partNames,
+          deleteData, environmentContext);
+    }
+    String apiName = HmsApiNameEnum.DROP_PARTITION_WITH_ENVIRONMENT_CONTEXT.apiName();
+    org.apache.impala.catalog.Table tbl = getTableAndAcquireWriteLock(dbname,
+        tblname, apiName);
+    catalog_.getLock().writeLock().unlock();
+    boolean resp = super.drop_partition_with_environment_context(dbname, tblname,
+        partNames, deleteData, environmentContext);
+    try {
+      syncToLatestEventId(tbl, apiName);
+    } finally {
+      catalogOpExecutor_.UnlockWriteLockIfErronouslyLocked();
+      tbl.releaseWriteLock();
+    }
+    return resp;
+  }
+
+  @Override
+  public boolean drop_partition_by_name_with_environment_context(String dbName,
+      String tableName, String partName, boolean deleteData,
+      EnvironmentContext envContext) throws NoSuchObjectException, MetaException,
+      TException {
+    if (!BackendConfig.INSTANCE.enableCatalogdHMSCache() ||
+        !BackendConfig.INSTANCE.enableSyncToLatestEventOnDdls()) {
+      return super.drop_partition_by_name_with_environment_context(dbName, tableName,
+          partName, deleteData, envContext);
+    }
+    String apiName =
+        HmsApiNameEnum.DROP_PARTITION_BY_NAME_WITH_ENVIRONMENT_CONTEXT.apiName();
+    org.apache.impala.catalog.Table tbl = getTableAndAcquireWriteLock(dbName,
+        tableName, apiName);
+    catalog_.getLock().writeLock().unlock();
+    boolean resp = super.drop_partition_by_name_with_environment_context(dbName,
+        tableName, partName, deleteData, envContext);
+    try {
+      syncToLatestEventId(tbl, apiName);
+    } finally {
+      catalogOpExecutor_.UnlockWriteLockIfErronouslyLocked();
+      tbl.releaseWriteLock();
+    }
+    return resp;
+  }
+
+  @Override
+  public DropPartitionsResult drop_partitions_req(
+      DropPartitionsRequest request)
+      throws NoSuchObjectException, MetaException, TException {
+    String dbName = request.getDbName();
+    String tableName = request.getTblName();
+    if (!BackendConfig.INSTANCE.enableCatalogdHMSCache() ||
+        !BackendConfig.INSTANCE.enableSyncToLatestEventOnDdls()) {
+      return super.drop_partitions_req(request);
+    }
+    String apiName =
+        HmsApiNameEnum.DROP_PARTITIONS_REQ.apiName();
+    org.apache.impala.catalog.Table tbl = getTableAndAcquireWriteLock(dbName,
+        tableName, apiName);
+    catalog_.getLock().writeLock().unlock();
+    DropPartitionsResult result = super.drop_partitions_req(request);
+    try {
+      syncToLatestEventId(tbl, apiName);
+    } finally {
+      catalogOpExecutor_.UnlockWriteLockIfErronouslyLocked();
+      tbl.releaseWriteLock();
+    }
+    return result;
+  }
+
+  @Override
+  public Partition exchange_partition(Map<String, String> partitionSpecMap,
+      String sourceDbWithCatalog, String sourceTbl, String destDbWithCatalog,
+      String destTbl) throws TException {
+    if (!BackendConfig.INSTANCE.enableCatalogdHMSCache() ||
+        !BackendConfig.INSTANCE.enableSyncToLatestEventOnDdls()) {
+      return super.exchange_partition(partitionSpecMap, sourceDbWithCatalog,
+          sourceTbl, destDbWithCatalog, destTbl);
+    }
+    // acquire lock on multiple tables at once
+    String apiName = HmsApiNameEnum.EXCHANGE_PARTITION.apiName();
+    org.apache.impala.catalog.Table srcTbl = null, destinationTbl = null;
+    Partition exchangedPartition = null;
+
+    try {
+      String sourceDb = MetaStoreUtils.parseDbName(sourceDbWithCatalog, serverConf_)[1];
+      String destDb = MetaStoreUtils.parseDbName(destDbWithCatalog, serverConf_)[1];
+      srcTbl = catalogOpExecutor_.getExistingTable(sourceDb, sourceTbl, apiName);
+      destinationTbl = catalogOpExecutor_.getExistingTable(destDb, destTbl, apiName);
+
+      if (!catalog_.tryWriteLock(
+          new org.apache.impala.catalog.Table[] {srcTbl, destinationTbl})) {
+        throw new CatalogException("Couldn't acquire lock on tables: "
+            + srcTbl.getFullName() + ", " + destinationTbl.getFullName());
+      }
+
+      exchangedPartition = super.exchange_partition(partitionSpecMap, sourceDbWithCatalog,
+          sourceTbl, destDbWithCatalog, destTbl);
+      syncToLatestEventId(srcTbl, apiName);
+      syncToLatestEventId(destinationTbl, apiName);
+    } catch (Exception e) {
+      rethrowException(e, apiName);
+    } finally {
+      catalogOpExecutor_.UnlockWriteLockIfErronouslyLocked();
+      if (srcTbl != null && srcTbl.isWriteLockedByCurrentThread()) {
+        srcTbl.releaseWriteLock();
+      }
+      if (destinationTbl != null && destinationTbl.isWriteLockedByCurrentThread()) {
+        destinationTbl.releaseWriteLock();
+      }
+    }
+    return exchangedPartition;
+  }
+
+  @Override
+  public List<Partition> exchange_partitions(Map<String, String> partitionSpecs,
+      String sourceDbWithCatalog, String sourceTbl, String destDbWithCatalog,
+      String destTbl) throws TException {
+    if (!BackendConfig.INSTANCE.enableCatalogdHMSCache() ||
+        !BackendConfig.INSTANCE.enableSyncToLatestEventOnDdls()) {
+      return super.exchange_partitions(partitionSpecs, sourceDbWithCatalog,
+          sourceTbl, destDbWithCatalog, destTbl);
+    }
+
+    // acquire lock on multiple tables at once
+    String apiName = HmsApiNameEnum.EXCHANGE_PARTITIONS.apiName();
+    org.apache.impala.catalog.Table srcTbl = null, destinationTbl = null;
+    List<Partition> exchangedPartitions = null;
+    try {
+      String sourceDb = MetaStoreUtils.parseDbName(sourceDbWithCatalog, serverConf_)[1];
+      String destDb = MetaStoreUtils.parseDbName(destDbWithCatalog, serverConf_)[1];
+      srcTbl = catalogOpExecutor_.getExistingTable(sourceDb, sourceTbl, apiName);
+      destinationTbl = catalogOpExecutor_.getExistingTable(destDb, destTbl, apiName);
+
+      if (!catalog_.tryWriteLock(
+          new org.apache.impala.catalog.Table[] {srcTbl, destinationTbl})) {
+        throw new CatalogException("Couldn't acquire lock on tables: "
+            + srcTbl.getFullName() + ", " + destinationTbl.getFullName());
+      }
+
+      exchangedPartitions = super.exchange_partitions(partitionSpecs, sourceDbWithCatalog,
+          sourceTbl, destDbWithCatalog, destTbl);
+      syncToLatestEventId(srcTbl, apiName);
+      syncToLatestEventId(destinationTbl, apiName);
+    } catch (Exception e) {
+      rethrowException(e, apiName);
+    } finally {
+      catalogOpExecutor_.UnlockWriteLockIfErronouslyLocked();
+      if (srcTbl != null && srcTbl.isWriteLockedByCurrentThread()) {
+        srcTbl.releaseWriteLock();
+      }
+      if (destinationTbl != null && destinationTbl.isWriteLockedByCurrentThread()) {
+        destinationTbl.releaseWriteLock();
+      }
+    }
+    return exchangedPartitions;
+  }
+
+  @Override
+  public void alter_partition(String dbName, String tblName, Partition partition)
+      throws InvalidOperationException, MetaException, TException {
+    if (!BackendConfig.INSTANCE.enableCatalogdHMSCache() ||
+        !BackendConfig.INSTANCE.enableSyncToLatestEventOnDdls()) {
+      super.alter_partition(dbName, tblName, partition);
+      return;
+    }
+    String apiName = HmsApiNameEnum.ALTER_PARTITION.apiName();
+    org.apache.impala.catalog.Table tbl = getTableAndAcquireWriteLock(dbName,
+        tblName, apiName);
+    catalog_.getLock().writeLock().unlock();
+    super.alter_partition(dbName, tblName, partition);
+    try {
+      syncToLatestEventId(tbl, apiName);
+    } finally {
+      catalogOpExecutor_.UnlockWriteLockIfErronouslyLocked();
+      tbl.releaseWriteLock();
+    }
+  }
+
+  @Override
+  public void alter_partitions(String dbName, String tblName, List<Partition> partitions)
+      throws InvalidOperationException, MetaException, TException {
+    if (!BackendConfig.INSTANCE.enableCatalogdHMSCache() ||
+        !BackendConfig.INSTANCE.enableSyncToLatestEventOnDdls()) {
+      super.alter_partitions(dbName, tblName, partitions);
+      return;
+    }
+    String apiName = HmsApiNameEnum.ALTER_PARTITIONS.apiName();
+    org.apache.impala.catalog.Table tbl = getTableAndAcquireWriteLock(dbName,
+        tblName, apiName);
+    catalog_.getLock().writeLock().unlock();
+    super.alter_partitions(dbName, tblName, partitions);
+    try {
+      syncToLatestEventId(tbl, apiName);
+    } finally {
+      catalogOpExecutor_.UnlockWriteLockIfErronouslyLocked();
+      tbl.releaseWriteLock();
+    }
+  }
+
+  @Override
+  public void alter_partitions_with_environment_context(String dbName, String tblName,
+      List<Partition> list, EnvironmentContext context) throws
+      InvalidOperationException, MetaException, TException {
+    if (!BackendConfig.INSTANCE.enableCatalogdHMSCache() ||
+        !BackendConfig.INSTANCE.enableSyncToLatestEventOnDdls()) {
+      super.alter_partitions_with_environment_context(dbName, tblName, list, context);
+      return;
+    }
+    String apiName = HmsApiNameEnum.ALTER_PARTITIONS_WITH_ENVIRONMENT_CONTEXT.apiName();
+    org.apache.impala.catalog.Table tbl = getTableAndAcquireWriteLock(dbName,
+        tblName, apiName);
+    catalog_.getLock().writeLock().unlock();
+    super.alter_partitions_with_environment_context(dbName, tblName, list, context);
+    try {
+      syncToLatestEventId(tbl, apiName);
+    } finally {
+      catalogOpExecutor_.UnlockWriteLockIfErronouslyLocked();
+      tbl.releaseWriteLock();
+    }
+  }
+
+  @Override
+  public AlterPartitionsResponse alter_partitions_req(
+      AlterPartitionsRequest alterPartitionsRequest)
+      throws InvalidOperationException, MetaException, TException {
+    if (!BackendConfig.INSTANCE.enableCatalogdHMSCache() ||
+        !BackendConfig.INSTANCE.enableSyncToLatestEventOnDdls()) {
+      return super.alter_partitions_req(alterPartitionsRequest);
+    }
+    String apiName = HmsApiNameEnum.ALTER_PARTITIONS_REQ.apiName();
+    org.apache.impala.catalog.Table tbl =
+        getTableAndAcquireWriteLock(alterPartitionsRequest.getDbName(),
+        alterPartitionsRequest.getTableName(), apiName);
+    catalog_.getLock().writeLock().unlock();
+    AlterPartitionsResponse response =
+        super.alter_partitions_req(alterPartitionsRequest);
+    try {
+      syncToLatestEventId(tbl, apiName);
+    } finally {
+      catalogOpExecutor_.UnlockWriteLockIfErronouslyLocked();
+      tbl.releaseWriteLock();
+    }
+    return response;
+  }
+
+  @Override
+  public void alter_partition_with_environment_context(String dbName, String tblName,
+      Partition partition, EnvironmentContext environmentContext)
+      throws InvalidOperationException, MetaException, TException {
+    if (!BackendConfig.INSTANCE.enableCatalogdHMSCache() ||
+        !BackendConfig.INSTANCE.enableSyncToLatestEventOnDdls()) {
+      super.alter_partition_with_environment_context(dbName, tblName,
+          partition, environmentContext);
+      return;
+    }
+    String apiName = HmsApiNameEnum.ALTER_PARTITION_WITH_ENVIRONMENT_CONTEXT.apiName();
+    org.apache.impala.catalog.Table tbl = getTableAndAcquireWriteLock(dbName,
+        tblName, apiName);
+    catalog_.getLock().writeLock().unlock();
+    super.alter_partition_with_environment_context(dbName, tblName,
+        partition, environmentContext);
+    try {
+      syncToLatestEventId(tbl, apiName);
+    } finally {
+      catalogOpExecutor_.UnlockWriteLockIfErronouslyLocked();
+      tbl.releaseWriteLock();
+    }
+  }
+
+  @Override
+  public void rename_partition(String dbName, String tblName, List<String> list,
+      Partition partition) throws InvalidOperationException, MetaException, TException {
+    if (!BackendConfig.INSTANCE.enableCatalogdHMSCache() ||
+        !BackendConfig.INSTANCE.enableSyncToLatestEventOnDdls()) {
+      super.rename_partition(dbName, tblName, list, partition);
+      return;
+    }
+    String apiName = HmsApiNameEnum.RENAME_PARTITION.apiName();
+    org.apache.impala.catalog.Table tbl = getTableAndAcquireWriteLock(dbName,
+        tblName, apiName);
+    catalog_.getLock().writeLock().unlock();
+    super.rename_partition(dbName, tblName, list, partition);
+    try {
+      syncToLatestEventId(tbl, apiName);
+    } finally {
+      catalogOpExecutor_.UnlockWriteLockIfErronouslyLocked();
+      tbl.releaseWriteLock();
+    }
+  }
+
+  @Override
+  public RenamePartitionResponse rename_partition_req(RenamePartitionRequest request)
+      throws InvalidOperationException, MetaException, TException {
+    if (!BackendConfig.INSTANCE.enableCatalogdHMSCache() ||
+        !BackendConfig.INSTANCE.enableSyncToLatestEventOnDdls()) {
+      return super.rename_partition_req(request);
+    }
+    String apiName = HmsApiNameEnum.RENAME_PARTITION_REQ.apiName();
+    org.apache.impala.catalog.Table tbl = getTableAndAcquireWriteLock(
+        request.getDbName(), request.getTableName(), apiName);
+    catalog_.getLock().writeLock().unlock();
+    RenamePartitionResponse response =
+        super.rename_partition_req(request);
+    try {
+      syncToLatestEventId(tbl, apiName);
+    } finally {
+      catalogOpExecutor_.UnlockWriteLockIfErronouslyLocked();
+      tbl.releaseWriteLock();
+    }
+    return response;
+  }
+
+  @Override
+  public void drop_table(String dbname, String tblname, boolean deleteData)
+      throws NoSuchObjectException, MetaException, TException {
+    if (!BackendConfig.INSTANCE.enableCatalogdHMSCache() ||
+        !BackendConfig.INSTANCE.enableSyncToLatestEventOnDdls()) {
+      super.drop_table(dbname, tblname, deleteData);
+      return;
+    }
+    org.apache.impala.catalog.Table tbl = null;
+    String apiName = HmsApiNameEnum.DROP_TABLE.apiName();
+    long currentEventId = -1;
+    catalogOpExecutor_.getMetastoreDdlLock().lock();
+    try {
+      try {
+        currentEventId = super.get_current_notificationEventId().getEventId();
+        super.drop_table(dbname, tblname, deleteData);
+      } catch (NoSuchObjectException e) {
+        LOG.debug("Table {}.{} does not exist in metastore, removing it from catalog " +
+                        "if exists", dbname, tblname);
+        if (catalog_.removeTable(dbname, tblname) != null) {
+          LOG.info("Table {}.{} does not exist in metastore, removed from catalog " +
+                  "as well", dbname, tblname);
+        }
+        throw e;
+      }
+      dropTableIfExists(currentEventId, dbname, tblname, apiName);
+    } finally {
+      catalogOpExecutor_.getMetastoreDdlLock().unlock();
+    }
+  }
+
+  @Override
+  public void drop_table_with_environment_context(String dbname, String tblname,
+      boolean deleteData, EnvironmentContext environmentContext)
+      throws NoSuchObjectException, MetaException, TException {
+    if (!BackendConfig.INSTANCE.enableCatalogdHMSCache() ||
+        !BackendConfig.INSTANCE.enableSyncToLatestEventOnDdls()) {
+      super.drop_table_with_environment_context(dbname, tblname,
+          deleteData, environmentContext);
+      return;
+    }
+    org.apache.impala.catalog.Table tbl = null;
+    String apiName =
+        HmsApiNameEnum.DROP_TABLE_WITH_ENVIRONMENT_CONTEXT.apiName();
+    long currentEventId = -1;
+    catalogOpExecutor_.getMetastoreDdlLock().lock();
+    try {
+      try {
+        currentEventId = super.get_current_notificationEventId().getEventId();
+        super.drop_table_with_environment_context(dbname, tblname,
+                deleteData, environmentContext);
+      } catch (NoSuchObjectException e) {
+        LOG.debug("Table {}.{} does not exist in metastore, removing it from catalog " +
+                "if exists", dbname, tblname);
+        if (catalog_.removeTable(dbname, tblname) != null) {
+          LOG.info("Table {}.{} does not exist in metastore, removed from catalog " +
+                  "as well", dbname, tblname);
+        }
+        throw e;
+      }
+      dropTableIfExists(currentEventId, dbname, tblname, apiName);
+    } finally {
+      catalogOpExecutor_.getMetastoreDdlLock().unlock();
+    }
+  }
+
+  @Override
+  public void truncate_table(String dbName, String tblName, List<String> partNames)
+      throws MetaException, TException {
+    if (!BackendConfig.INSTANCE.enableCatalogdHMSCache() ||
+        !BackendConfig.INSTANCE.enableSyncToLatestEventOnDdls()) {
+      super.truncate_table(dbName, tblName, partNames);
+      return;
+    }
+    String apiName = HmsApiNameEnum.TRUNCATE_TABLE.apiName();
+    org.apache.impala.catalog.Table tbl = getTableAndAcquireWriteLock(dbName,
+        tblName, apiName);
+    catalog_.getLock().writeLock().unlock();
+    super.truncate_table(dbName, tblName, partNames);
+    try {
+      syncToLatestEventId(tbl, apiName);
+    } finally {
+      catalogOpExecutor_.UnlockWriteLockIfErronouslyLocked();
+      tbl.releaseWriteLock();
+    }
+  }
+
+  @Override
+  public TruncateTableResponse truncate_table_req(TruncateTableRequest req)
+      throws MetaException, TException {
+    if (!BackendConfig.INSTANCE.enableCatalogdHMSCache() ||
+        !BackendConfig.INSTANCE.enableSyncToLatestEventOnDdls()) {
+      return super.truncate_table_req(req);
+    }
+    String apiName = HmsApiNameEnum.TRUNCATE_TABLE_REQ.apiName();
+    org.apache.impala.catalog.Table tbl = getTableAndAcquireWriteLock(
+        req.getDbName(), req.getTableName(), apiName);
+    catalog_.getLock().writeLock().unlock();
+    TruncateTableResponse response = super.truncate_table_req(req);
+    try {
+      syncToLatestEventId(tbl, apiName);
+    } finally {
+      catalogOpExecutor_.UnlockWriteLockIfErronouslyLocked();
+      tbl.releaseWriteLock();
+    }
+    return response;
+  }
+
+  private org.apache.impala.catalog.Table getTableAndAcquireWriteLock(
+      String dbNameWithCatalog, String tblName, String apiName) throws TException {
+    org.apache.impala.catalog.Table tbl = null;
+    try {
+      String dbName = MetaStoreUtils.parseDbName(dbNameWithCatalog, serverConf_)[1];
+      tbl = catalogOpExecutor_.getExistingTable(dbName, tblName, apiName);
+    } catch (Exception e) {
+      rethrowException(e, apiName);
+    }
+    if (!catalog_.tryWriteLock(tbl)) {
+      // should it be an internal exception?
+      CatalogException e =
+          new CatalogException("Could not acquire lock on table: " + tbl.getFullName());
+      rethrowException(e, apiName);
+    }
+    return tbl;
+  }
+
+  /**
+   * Get db from cache and acquire lock on it
+   * @param dbName
+   * @param apiName
+   * @return
+   * @throws TException if either db does not exist in cache or couldn't
+   *         acquire lock
+   */
+  private org.apache.impala.catalog.Db getDbAndAcquireLock(String dbName, String apiName)
+      throws TException {
+    Db db = catalog_.getDb(dbName);
+    if (db == null) {
+      rethrowException(new CatalogException("Database: " +
+          dbName + " does not exist in cache"), apiName);
+    }
+    if (!catalog_.tryLockDb(db)) {
+      rethrowException(
+          new CatalogException("Couldn't acquire write lock on db: " + db.getName()),
+          apiName);
+    }
+    return db;
+  }
+
+  /**
+   * Sync the table tbl to latest event id. Catches
+   * exceptions (if any) while syncing and convert them
+   * to TException
+   * @throws TException
+   */
+  private void syncToLatestEventId(org.apache.impala.catalog.Table tbl, String apiName)
+      throws TException {
+    Preconditions.checkState(
+        BackendConfig.INSTANCE.enableSyncToLatestEventOnDdls(),
+        "sync to latest event flag should be set to true");
+    Preconditions.checkState(tbl.isWriteLockedByCurrentThread(),
+        "Thread does not have write lock on table %s", tbl.getFullName());
+    try {
+      MetastoreEventsProcessor.syncToLatestEventId(catalog_, tbl,
+          metastoreEventFactory_, metastoreEventsMetrics_);
+    } catch (Exception e) {
+      String errMsg = String.format(SYNC_TABLE_LATEST_EVENT_ID_ERR_MSG,
+          tbl.getFullName(), apiName);
+      LOG.error("{}. Exception stacktrace: {} ", errMsg,
+          ExceptionUtils.getFullStackTrace(e));
+      rethrowException(e, apiName);
+    }
+  }
+
+  /**
+   * Sync db to latest event id. Catches
+   * exceptions (if any) while syncing and convert them
+   * to TException
+   * @throws TException
+   */
+  private void syncToLatestEventId(org.apache.impala.catalog.Db db, String apiName)
+      throws TException {
+    Preconditions.checkState(
+        BackendConfig.INSTANCE.enableSyncToLatestEventOnDdls(),
+            "sync to latest event flag should be set to true");
+    Preconditions.checkState(db.isLockHeldByCurrentThread(),
+        "Current thread does not hold lock on db: %s", db.getName());
+    try {
+      MetastoreEventsProcessor.syncToLatestEventId(catalog_, db,
+              metastoreEventFactory_, metastoreEventsMetrics_);
+    } catch (Exception e) {
+      String errMsg = String.format(SYNC_DB_LATEST_EVENT_ID_ERR_MSG,
+              db.getName(), apiName);
+      LOG.error("{}. Exception stacktrace: {} ", errMsg,
+          ExceptionUtils.getFullStackTrace(e));
+      rethrowException(e, apiName);
+    }
+  }
+
+  private <T extends Object> T alterTableCore(String dbname, String tblName,
+      org.apache.hadoop.hive.metastore.api.Table newTable, String apiName,
+      AlterTableTask<T> task) throws InvalidOperationException, MetaException,
+      TException {
+    boolean isRename = !dbname.equalsIgnoreCase(newTable.getDbName()) ||
+        !tblName.equalsIgnoreCase(newTable.getTableName());
+    org.apache.impala.catalog.Table tbl = getTableAndAcquireWriteLock(dbname, tblName,
+        apiName);
+    org.apache.impala.catalog.Table newTbl = null;
+    try {
+      // perform HMS operation
+      if (!isRename) {
+        // release lock if it is not rename. For rename operation the lock would
+        // get released in finally block
+        catalog_.getLock().writeLock().unlock();
+        T resp = task.execute();
+        syncToLatestEventId(tbl, apiName);
+        return resp;
+      }
+      long currentEventId = super.get_current_notificationEventId().getEventId();
+      T resp = task.execute();
+      // Rename scenario, remove old table and add new one
+      try {
+        List<NotificationEvent> events =
+            MetastoreEventsProcessor.getNextMetastoreEventsInBatches(catalog_,
+                currentEventId, event -> "ALTER_TABLE".equals(event.getEventType())
+                // the alter table event is generated on the renamed table
+                && newTable.getDbName().equalsIgnoreCase(event.getDbName())
+                && newTable.getTableName().equalsIgnoreCase(event.getTableName()));
+        Preconditions.checkState(events.size() == 1, String.format("For table %s.%s, "
+            + "from event id: %s, expected ALTER_TABLE events size to be 1 but is %s",
+            newTable.getDbName(), newTable.getTableName(), currentEventId,
+            events.size()));
+
+        MetastoreEvents.MetastoreEvent event = metastoreEventFactory_.get(events.get(0),
+            metastoreEventsMetrics_);
+        Preconditions.checkState(event instanceof MetastoreEvents.AlterTableEvent);
+
+        MetastoreEvents.AlterTableEvent alterEvent =
+            (MetastoreEvents.AlterTableEvent) event;
+
+        org.apache.hadoop.hive.metastore.api.Table oldMsTable =
+            alterEvent.getBeforeTable();
+        org.apache.hadoop.hive.metastore.api.Table newMsTable =
+            alterEvent.getAfterTable();
+
+        TTableName oldTTable = new TTableName(oldMsTable.getDbName(),
+            oldMsTable.getTableName());
+        TTableName newTTable = new TTableName(newMsTable.getDbName(),
+            newMsTable.getTableName());
+        // Rename the table in the Catalog and get the resulting catalog object.
+        // ALTER TABLE/VIEW RENAME is implemented as an ADD + DROP.
+        Pair<org.apache.impala.catalog.Table, org.apache.impala.catalog.Table> result =
+            catalog_.renameTable(oldTTable, newTTable);
+        if (result.first == null && result.second == null) {
+          throw new CatalogException("failed to rename table " + oldTTable + " to " +
+              newTTable + " for " + apiName);
+        }
+        // first set the last synced event id to the alter table's event id
+        result.second.setLastSyncedEventId(alterEvent.getEventId());
+        result.second.setCreateEventId(alterEvent.getEventId());
+        catalogOpExecutor_.addToDeleteEventLog(alterEvent.getEventId(),
+            DeleteEventLog.getTblKey(oldTTable.getDb_name(), oldTTable.getTable_name()));
+      } finally {
+        catalog_.getLock().writeLock().unlock();
+      }
+      // sync renamed table to latest event id
+      newTbl = getTableAndAcquireWriteLock(newTable.getDbName(), newTable.getTableName(),
+          apiName);
+      // release writelock
+      catalog_.getLock().writeLock().unlock();
+      syncToLatestEventId(newTbl, apiName);
+      return resp;
+    } catch (Exception e) {
+      rethrowException(e, apiName);
+    } finally {
+      catalogOpExecutor_.UnlockWriteLockIfErronouslyLocked();
+      if (tbl != null && tbl.isWriteLockedByCurrentThread()) {
+        tbl.releaseWriteLock();
+      }
+      if (newTbl != null && newTbl.isWriteLockedByCurrentThread()) {
+        newTbl.releaseWriteLock();
+      }
+    }
+    return null;
+  }
+
+  private void createTableCore(String dbNameWithCatalog, String tblName, String apiName,
+      CreateTableTask task) throws TException {
+    String dbName = MetaStoreUtils.parseDbName(dbNameWithCatalog, serverConf_)[1];
+    catalogOpExecutor_.getMetastoreDdlLock().lock();
+    org.apache.impala.catalog.Table tbl = null;
+    long fromEventId = -1;
+    try {
+      fromEventId = super.get_current_notificationEventId().getEventId();
+      try {
+        task.execute();
+      } catch (TException e) {
+        // rethrow if not AlreadyExistsException
+        if (!(e instanceof AlreadyExistsException)) {
+          throw e;
+        }
+        if (catalog_.getTableNoThrow(dbName, tblName) != null) {
+          LOG.debug("Table {}.{} already exists in metastore as well as catalog",
+              dbName, tblName);
+          throw e;
+        }
+        LOG.debug("Table {}.{} exists in metastore but not in catalog. Ignoring " +
+            "exception {} from metastore", dbName, tblName, e.getClass().getName());
+      }
+      List<NotificationEvent> events =
+          MetastoreEventsProcessor
+              .getNextMetastoreEventsInBatches(catalog_, fromEventId,
+                  event -> MetastoreEvents.CreateTableEvent.CREATE_TABLE_EVENT_TYPE
+                      .equals(event.getEventType())
+                      && dbName.equalsIgnoreCase(event.getDbName())
+                      && tblName.equalsIgnoreCase(event.getTableName()));
+
+      Preconditions.checkState(events.size() == 1,
+          "Table %s.%s was recreated in metastore since event id %s" +
+              "while the current table creation was in progress", dbName, tblName,
+          fromEventId);
+      long createEventId = events.get(0).getEventId();
+      catalog_.addIncompleteTable(dbName, tblName, createEventId);
+      LOG.info("Added incomplete table {}.{} with create event id: {}", dbName, tblName,
+          createEventId);
+      // sync to latest event ID
+      tbl = getTableAndAcquireWriteLock(dbName, tblName, apiName);
+      catalog_.getLock().writeLock().unlock();
+      syncToLatestEventId(tbl, apiName);
+    } catch (Exception e) {
+      rethrowException(e, apiName);
+    } finally {
+      catalogOpExecutor_.getMetastoreDdlLock().unlock();
+      catalogOpExecutor_.UnlockWriteLockIfErronouslyLocked();
+      if (tbl != null && tbl.isWriteLockedByCurrentThread()) {
+        tbl.releaseWriteLock();
+      }
+    }
+  }
+
+  private abstract class AlterTableTask<T> {
+    public abstract T execute() throws InvalidOperationException, MetaException,
+        TException;
+  }
+
+  private abstract class CreateTableTask<T> {
+    public abstract T execute() throws TException;
+  }
 }
diff --git a/fe/src/main/java/org/apache/impala/catalog/metastore/HmsApiNameEnum.java b/fe/src/main/java/org/apache/impala/catalog/metastore/HmsApiNameEnum.java
index eeb3357..b680134 100644
--- a/fe/src/main/java/org/apache/impala/catalog/metastore/HmsApiNameEnum.java
+++ b/fe/src/main/java/org/apache/impala/catalog/metastore/HmsApiNameEnum.java
@@ -24,7 +24,47 @@ package org.apache.impala.catalog.metastore;
 public enum HmsApiNameEnum {
   GET_TABLE_REQ("get_table_req"),
   GET_PARTITION_BY_EXPR("get_partitions_by_expr"),
-  GET_PARTITION_BY_NAMES("get_partitions_by_names_req");
+  GET_PARTITION_BY_NAMES("get_partitions_by_names_req"),
+  CREATE_DATABASE("create_database"),
+  DROP_DATABASE("drop_database"),
+  ALTER_DATABASE("alter_database"),
+  CREATE_TABLE("create_table"),
+  CREATE_TABLE_REQ("create_table_req"),
+  CREATE_TABLE_WITH_CONSTRAINTS("create_table_with_constraints"),
+  CREATE_TABLE_WITH_ENVIRONMENT_CONTEXT("create_table_with_environment_context"),
+  ALTER_TABLE("alter_table"),
+  ALTER_TABLE_WITH_ENVIRONMENT_CONTEXT("alter_table_with_environment_context"),
+  ALTER_TABLE_WITH_CASCADE("alter_table_with_cascade"),
+  ALTER_TABLE_REQ("alter_table_req"),
+  ADD_PARTITION("add_partition"),
+  ADD_PARTITION_WITH_ENVIRONMENT_CONTEXT("add_partition_with_environment_context"),
+  ADD_PARTITIONS("add_partitions"),
+  ADD_PARTITIONS_PSPEC("add_partitions_pspec"),
+  ADD_PARTITIONS_REQ("add_partitions_req"),
+  APPEND_PARTITION("append_partition"),
+  APPEND_PARTITION_WITH_ENVIRONMENT_CONTEXT("append_partition_with_environment_context"),
+  APPEND_PARTITION_BY_NAME("append_partition_by_name"),
+  APPEND_PARTITION_BY_NAME_WITH_ENVIRONMENT_CONTEXT(
+      "append_partition_by_name_with_environment_context"),
+  DROP_PARTITION("drop_partition"),
+  DROP_PARTITION_BY_NAME("drop_partition_by_name"),
+  DROP_PARTITION_WITH_ENVIRONMENT_CONTEXT("drop_partition_with_environment_context"),
+  DROP_PARTITION_BY_NAME_WITH_ENVIRONMENT_CONTEXT(
+      "drop_partition_by_name_with_environment_context"),
+  DROP_PARTITIONS_REQ("drop_partitions_req"),
+  ALTER_PARTITION("alter_partition"),
+  ALTER_PARTITIONS("alter_partitions"),
+  ALTER_PARTITIONS_WITH_ENVIRONMENT_CONTEXT("alter_partitions_with_environment_context"),
+  ALTER_PARTITIONS_REQ("alter_partitions_req"),
+  ALTER_PARTITION_WITH_ENVIRONMENT_CONTEXT("alter_partition_with_environment_context"),
+  RENAME_PARTITION("rename_partition"),
+  RENAME_PARTITION_REQ("rename_partition_req"),
+  EXCHANGE_PARTITION("exchange_partition"),
+  EXCHANGE_PARTITIONS("exchange_partitions"),
+  DROP_TABLE("drop_table"),
+  DROP_TABLE_WITH_ENVIRONMENT_CONTEXT("drop_table_with_environment_context"),
+  TRUNCATE_TABLE("truncate_table"),
+  TRUNCATE_TABLE_REQ("truncate_tale_req");
 
   private final String apiName;
 
diff --git a/fe/src/main/java/org/apache/impala/catalog/metastore/MetastoreServiceHandler.java b/fe/src/main/java/org/apache/impala/catalog/metastore/MetastoreServiceHandler.java
index f800362..48e6c3c 100644
--- a/fe/src/main/java/org/apache/impala/catalog/metastore/MetastoreServiceHandler.java
+++ b/fe/src/main/java/org/apache/impala/catalog/metastore/MetastoreServiceHandler.java
@@ -270,9 +270,13 @@ import org.apache.impala.catalog.DatabaseNotFoundException;
 import org.apache.impala.catalog.CatalogServiceCatalog;
 import org.apache.impala.catalog.IncompleteTable;
 import org.apache.impala.catalog.MetaStoreClientPool.MetaStoreClient;
+import org.apache.impala.catalog.events.EventFactory;
 import org.apache.impala.catalog.events.MetastoreEvents;
 import org.apache.impala.catalog.events.MetastoreEvents.DropTableEvent;
+import org.apache.impala.catalog.events.MetastoreEvents.MetastoreEvent;
+import org.apache.impala.catalog.events.MetastoreEvents.MetastoreEventFactory;
 import org.apache.impala.catalog.events.MetastoreEventsProcessor;
+import org.apache.impala.common.Metrics;
 import org.apache.impala.common.Reference;
 import org.apache.impala.common.Pair;
 import org.apache.impala.compat.MetastoreShim;
@@ -315,7 +319,11 @@ public abstract class MetastoreServiceHandler extends AbstractThriftHiveMetastor
   protected Configuration serverConf_;
   protected PartitionExpressionProxy expressionProxy_;
   protected final String defaultCatalogName_;
-  protected final boolean invalidateCacheOnDDLs_;
+
+  // Initializing metrics so that they can be tracked
+  // separately for events processed by this class
+  protected final Metrics metastoreEventsMetrics_ = new Metrics();
+  protected EventFactory metastoreEventFactory_;
 
   public MetastoreServiceHandler(CatalogOpExecutor catalogOpExecutor,
       boolean fallBackToHMSOnErrors) {
@@ -351,12 +359,26 @@ public abstract class MetastoreServiceHandler extends AbstractThriftHiveMetastor
     //  2. Access config directly from BackendConfig INSTANCE directly.
     //  For now, going with option #2
 
-    invalidateCacheOnDDLs_ =
+    boolean invalidateCacheOnDdlFlag =
         BackendConfig.INSTANCE.invalidateCatalogdHMSCacheOnDDLs();
-    LOG.info("Invalidate catalogd cache for DDLs on non transactional tables " +
-        "is set to {}",invalidateCacheOnDDLs_);
+    boolean syncToLatestEventIdFlag =
+        BackendConfig.INSTANCE.enableSyncToLatestEventOnDdls();
+    Preconditions.checkArgument(!(invalidateCacheOnDdlFlag && syncToLatestEventIdFlag),
+        "Configurations invalidate_hms_cache_on_ddls and "
+            + "enable_sync_to_latest_event_on_ddls can not be set to true "
+            + "at the same time");
+
+    if (BackendConfig.INSTANCE.invalidateCatalogdHMSCacheOnDDLs()) {
+      LOG.debug("Invalidate catalogd cache for DDLs on non transactional tables " +
+          "is set to true");
+    }
+    if (BackendConfig.INSTANCE.enableSyncToLatestEventOnDdls()) {
+      LOG.debug("catalogd cache sync to latest event id is set to true");
+    }
     // make sure catalogOpExecutor's metastoreHmsDdlLock is not null
     Preconditions.checkNotNull(catalogOpExecutor_.getMetastoreDdlLock());
+    initMetrics();
+    metastoreEventFactory_ = catalog_.getEventFactoryForSyncToLatestEvent();
   }
 
   @Override
@@ -461,7 +483,11 @@ public abstract class MetastoreServiceHandler extends AbstractThriftHiveMetastor
     } finally {
       catalogOpExecutor_.getMetastoreDdlLock().unlock();
     }
-    dropDbIfExists(databaseName, currentEventId, "drop_database");
+    if (!BackendConfig.INSTANCE.invalidateCatalogdHMSCacheOnDDLs() ||
+            !BackendConfig.INSTANCE.enableCatalogdHMSCache()) {
+      return;
+    }
+    dropDbIfExists(databaseName, ignoreUnknownDb, currentEventId, "drop_database");
   }
 
   @Override
@@ -676,10 +702,16 @@ public abstract class MetastoreServiceHandler extends AbstractThriftHiveMetastor
     long eventId = -1;
     try (MetaStoreClient client = catalog_.getMetaStoreClient()) {
       eventId = getCurrentEventId(client);
+      // TODO: Handle NoSuchObjectException
       client.getHiveClient().getThriftClient().drop_table(dbname,
-          tblname, deleteData);
-      removeTableIfExists(eventId, dbname, tblname,
-          "drop_table");
+              tblname, deleteData);
+      if (!BackendConfig.INSTANCE.invalidateCatalogdHMSCacheOnDDLs() ||
+              !BackendConfig.INSTANCE.enableCatalogdHMSCache()) {
+        return;
+      }
+      dropTableIfExists(eventId, dbname, tblname,
+              "drop_table");
+
     } finally {
       catalogOpExecutor_.getMetastoreDdlLock().unlock();
     }
@@ -697,7 +729,11 @@ public abstract class MetastoreServiceHandler extends AbstractThriftHiveMetastor
       client.getHiveClient().getThriftClient()
           .drop_table_with_environment_context(dbname, tblname, deleteData,
               environmentContext);
-      removeTableIfExists(eventId, dbname, tblname,
+      if (!BackendConfig.INSTANCE.invalidateCatalogdHMSCacheOnDDLs() ||
+              !BackendConfig.INSTANCE.enableCatalogdHMSCache()) {
+        return;
+      }
+      dropTableIfExists(eventId, dbname, tblname,
           "drop_table_with_environment_context");
     } finally {
       catalogOpExecutor_.getMetastoreDdlLock().unlock();
@@ -1432,6 +1468,11 @@ public abstract class MetastoreServiceHandler extends AbstractThriftHiveMetastor
       throws TException {
     LOG.debug("Received exception while executing {}", apiName, cause);
     if (fallBackToHMSOnErrors_) return;
+    rethrowException(cause, apiName);
+  }
+
+  protected void rethrowException(Exception cause, String apiName)
+      throws TException {
     if (cause instanceof TException) throw (TException) cause;
     // if this is not a TException we wrap it to a MetaException
     throw new MetaException(
@@ -2915,6 +2956,13 @@ public abstract class MetastoreServiceHandler extends AbstractThriftHiveMetastor
   }
 
   /**
+   * Gets the current event id from the hive metastore.
+   */
+  private long getCurrentEventId(MetaStoreClient msClient) throws TException {
+    return msClient.getHiveClient().getCurrentNotificationEventId().getEventId();
+  }
+
+  /**
    * For non transactional tables, invalidate the table from cache
    * if hms ddl apis are accessed from catalogd's metastore server.
    * Any subsequent get table request fetches the table from HMS and loads
@@ -2930,13 +2978,27 @@ public abstract class MetastoreServiceHandler extends AbstractThriftHiveMetastor
    */
   private void invalidateNonTransactionalTableIfExists(String dbNameWithCatalog,
       String tableName, String apiName) throws MetaException {
-    // return immediately if flag invalidateCacheOnDDLs_ is false
-    if (!invalidateCacheOnDDLs_) {
-      LOG.debug("Not invalidating table {}.{} from catalogd cache because " +
-              "invalidateCacheOnDDLs_ flag is set to {} ", dbNameWithCatalog,
-          tableName, invalidateCacheOnDDLs_);
+    // return immediately if
+    //     flag invalidateCacheOnDDLs_ is false
+    //                   or
+    //     catalogHMSCache is disabled
+    //                   or
+    //     syncToLatestEventId is true
+    if (BackendConfig.INSTANCE.enableSyncToLatestEventOnDdls()) {
+      LOG.debug("Skipping invalidation of table {}.{} due to metastore api {} because "
+          + "sync to latest event id is set to true", dbNameWithCatalog, tableName,
+          apiName);
       return;
     }
+
+    if (!BackendConfig.INSTANCE.invalidateCatalogdHMSCacheOnDDLs() ||
+        !BackendConfig.INSTANCE.enableCatalogdHMSCache()) {
+      LOG.debug("Skipping invalidation of table {}.{} due to metastore api {} because " +
+              "invalidate hms cache of ddl flag is set to false ", dbNameWithCatalog,
+          tableName, apiName);
+      return;
+    }
+
     String dbName = dbNameWithCatalog;
     try {
       // Parse db name. Throw error if parsing fails.
@@ -2983,42 +3045,20 @@ public abstract class MetastoreServiceHandler extends AbstractThriftHiveMetastor
   }
 
   /**
-   * Gets the current event id from the hive metastore.
-   */
-  private long getCurrentEventId(MetaStoreClient msClient) throws TException {
-    return msClient.getHiveClient().getCurrentNotificationEventId().getEventId();
-  }
-
-  /**
    * This method is identical to invalidateNonTransactionalTableIfExists()
    * except that it
    * 1.removes both transactional and non transactional tables
    * 2.removes(and not invalidates) table from the cache on
    * DDLs like drop_table
    */
-  private void removeTableIfExists(long beforeDropEventId, String dbNameWithCatalog,
+  protected void dropTableIfExists(long beforeDropEventId, String dbNameWithCatalog,
       String tableName, String apiName) throws MetaException {
-    // return immediately if flag invalidateCacheOnDDLs_ is false
-    if (!invalidateCacheOnDDLs_) {
-      LOG.debug("Not removing table {}.{} from catalogd cache because " +
-              "invalidateCacheOnDDLs_ flag is set to {} ", dbNameWithCatalog,
-          tableName, invalidateCacheOnDDLs_);
-      return;
-    }
     // Parse db name. Throw error if parsing fails.
     String dbName;
     String catName;
-    try {
-      String[] catAndDbName = MetaStoreUtils.parseDbName(dbNameWithCatalog, serverConf_);
-      catName = catAndDbName[0];
-      dbName = catAndDbName[1];
-    } catch (MetaException ex) {
-      LOG.error("Successfully executed metastpre api: {} but encountered error " +
-              "when parsing dbName {} to invalidate/remove table from cache " +
-              "with error message: {}", apiName, dbNameWithCatalog,
-          ex.getMessage());
-      throw ex;
-    }
+    String[] catAndDbName = MetaStoreUtils.parseDbName(dbNameWithCatalog, serverConf_);
+    catName = catAndDbName[0];
+    dbName = catAndDbName[1];
     try {
       List<NotificationEvent> events = MetastoreEventsProcessor
           .getNextMetastoreEventsInBatches(catalog_, beforeDropEventId,
@@ -3045,8 +3085,7 @@ public abstract class MetastoreServiceHandler extends AbstractThriftHiveMetastor
       }
     } catch (Exception e) {
       String msg =
-          "Unable to process the DROP table event for table " + dbName +
-              "." + tableName;
+          "Unable to process the DROP table event for table " + dbName + "." + tableName;
       LOG.error(msg, e);
       throw new MetaException(msg);
     }
@@ -3060,13 +3099,21 @@ public abstract class MetastoreServiceHandler extends AbstractThriftHiveMetastor
   private void renameNonTransactionalTableIfExists(String oldDbNameWithCatalog,
       String oldTableName, String newDbNameWithCatalog, String newTableName,
       String apiName) throws MetaException {
-    // return immediately if flag invalidateCacheOnDDLs_ is false
-    if (!invalidateCacheOnDDLs_) {
-      LOG.debug("invalidateCacheOnDDLs_ flag is false, skipping cache " +
-              "update for operation {} on table {}.{}", apiName,
-          oldDbNameWithCatalog, oldTableName);
+    if (BackendConfig.INSTANCE.enableSyncToLatestEventOnDdls()) {
+      LOG.debug("Skipping rename of table {}.{} due to metastore api {} because "
+          + "sync to latest event id is set to true", oldDbNameWithCatalog, oldTableName,
+          apiName);
+      return;
+    }
+
+    if (!BackendConfig.INSTANCE.invalidateCatalogdHMSCacheOnDDLs() ||
+        !BackendConfig.INSTANCE.enableCatalogdHMSCache()) {
+      LOG.debug("Skipping rename of table {}.{} due to metastore api {} because " +
+              "invalidate hms cache of ddl flag is set to false ", oldDbNameWithCatalog,
+          oldTableName, apiName);
       return;
     }
+
     String toParse = null, oldDbName, newDbName;
     // Parse old and new db names. Throw error if parsing fails
     try {
@@ -3098,14 +3145,8 @@ public abstract class MetastoreServiceHandler extends AbstractThriftHiveMetastor
   /*
   Drops db from the cache. Also add the drop event id to deleteEventLog
    */
-  private void dropDbIfExists(String databaseName, long beforeDropEventId, String apiName)
-      throws MetaException {
-    if (!invalidateCacheOnDDLs_) {
-      LOG.debug("invalidateCacheOnDDLs_ flag is false, skipping " +
-          "cache update for hms operation" + apiName + " on db: " +
-          databaseName);
-      return;
-    }
+  protected void dropDbIfExists(String databaseName, boolean ignoreUnknownDb,
+      long beforeDropEventId, String apiName) throws MetaException {
     // Parse db name. Throw error if parsing fails.
     String dbName;
     String catName;
@@ -3122,29 +3163,66 @@ public abstract class MetastoreServiceHandler extends AbstractThriftHiveMetastor
                       .DROP_DATABASE_EVENT_TYPE)
                   && catName.equalsIgnoreCase(event.getCatName())
                   && dbName.equalsIgnoreCase(event.getDbName()));
-      if (events.isEmpty()) {
-        throw new MetaException(
-            "Drop database event not received. Check if notification " +
-                "events are configured in hive metastore");
+
+      if (events.size() == 0) {
+        if (ignoreUnknownDb) {
+          LOG.debug("db {} does not exist in metastore. Removing it from catalog if "
+                  + "exists ", dbName);
+          if (catalog_.removeDb(dbName) != null) {
+            LOG.info("Db {} not known to metastore, removed it from catalog", dbName);
+          }
+          return;
+        } else {
+          throw new MetaException(
+              String.format("Drop database event not received for db: %s from event "
+                      + "id: %s. Check if notification events are configured in hive"
+                      + " metastore", dbName, beforeDropEventId));
+        }
       }
       Preconditions.checkState(events.size() == 1, "Expected drop_database count " +
-          "to be 1 for db %s", databaseName);
+          "to be 1 for db %s from event id %s", databaseName, beforeDropEventId);
       long dropEventId = events.get(0).getEventId();
       boolean isRemoved =
           catalogOpExecutor_.removeDbIfNotAddedLater(dropEventId,
               dbName);
       if (isRemoved) {
-        LOG.info("Removed database: " + databaseName +
-            " from cache due to metastore API: drop_database");
+        LOG.info("Removed database: " + databaseName + " from cache due to metastore "
+            + "api: {}", apiName);
         catalogOpExecutor_.addToDeleteEventLog(events);
       }
 
     } catch (Exception e) {
-      String errorMsg = "Unable to process Drop database event for db: " +
+      String errorMsg = "Unable to process drop database event for db: " +
           databaseName + " for metastore api: " + apiName;
       LOG.error(errorMsg, e);
       throw new MetaException(errorMsg);
     }
+  }
 
+  private void initMetrics() {
+    metastoreEventsMetrics_.addTimer(
+        MetastoreEventsProcessor.EVENTS_FETCH_DURATION_METRIC);
+    metastoreEventsMetrics_.addTimer(
+        MetastoreEventsProcessor.EVENTS_PROCESS_DURATION_METRIC);
+    metastoreEventsMetrics_.addMeter(
+        MetastoreEventsProcessor.EVENTS_RECEIVED_METRIC);
+    metastoreEventsMetrics_.addCounter(
+        MetastoreEventsProcessor.EVENTS_SKIPPED_METRIC);
+    metastoreEventsMetrics_.addCounter(
+        MetastoreEventsProcessor.NUMBER_OF_TABLE_REFRESHES);
+    metastoreEventsMetrics_.addCounter(
+        MetastoreEventsProcessor.NUMBER_OF_PARTITION_REFRESHES);
+    metastoreEventsMetrics_.addCounter(
+        MetastoreEventsProcessor.NUMBER_OF_TABLES_ADDED);
+    metastoreEventsMetrics_.addCounter(
+        MetastoreEventsProcessor.NUMBER_OF_TABLES_REMOVED);
+    metastoreEventsMetrics_.addCounter(
+        MetastoreEventsProcessor.NUMBER_OF_DATABASES_ADDED);
+    metastoreEventsMetrics_.addCounter(
+        MetastoreEventsProcessor.NUMBER_OF_DATABASES_REMOVED);
+    metastoreEventsMetrics_.addCounter(
+        MetastoreEventsProcessor.NUMBER_OF_PARTITIONS_ADDED);
+    metastoreEventsMetrics_.addCounter(
+        MetastoreEventsProcessor.NUMBER_OF_PARTITIONS_REMOVED);
   }
 }
diff --git a/fe/src/main/java/org/apache/impala/service/BackendConfig.java b/fe/src/main/java/org/apache/impala/service/BackendConfig.java
index 1befe4b..467faaa 100644
--- a/fe/src/main/java/org/apache/impala/service/BackendConfig.java
+++ b/fe/src/main/java/org/apache/impala/service/BackendConfig.java
@@ -345,4 +345,18 @@ public class BackendConfig {
   public boolean isAutoCheckCompaction() {
     return backendCfg_.auto_check_compaction;
   }
+
+  @VisibleForTesting
+  public void setInvalidateCatalogdHMSCacheOnDDLs(boolean flag) {
+    backendCfg_.invalidate_hms_cache_on_ddls = flag;
+  }
+
+  public boolean enableSyncToLatestEventOnDdls() {
+    return backendCfg_.enable_sync_to_latest_event_on_ddls;
+  }
+
+  @VisibleForTesting
+  public void setEnableSyncToLatestEventOnDdls(boolean flag) {
+    backendCfg_.enable_sync_to_latest_event_on_ddls = flag;
+  }
 }
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 ba13dbe..945da4c 100644
--- a/fe/src/main/java/org/apache/impala/service/CatalogOpExecutor.java
+++ b/fe/src/main/java/org/apache/impala/service/CatalogOpExecutor.java
@@ -141,6 +141,7 @@ import org.apache.impala.common.ImpalaException;
 import org.apache.impala.common.ImpalaRuntimeException;
 import org.apache.impala.common.InternalException;
 import org.apache.impala.common.JniUtil;
+import org.apache.impala.common.Metrics;
 import org.apache.impala.common.Pair;
 import org.apache.impala.common.Reference;
 import org.apache.impala.common.TransactionException;
@@ -663,7 +664,7 @@ public class CatalogOpExecutor {
    * This method checks if the write lock of 'catalog_' is unlocked. If it's still locked
    * then it logs an error and unlocks it.
    */
-  private void UnlockWriteLockIfErronouslyLocked() {
+  public void UnlockWriteLockIfErronouslyLocked() {
     if(catalog_.getLock().isWriteLockedByCurrentThread()) {
       LOG.error("Write lock should have been released.");
       catalog_.getLock().writeLock().unlock();
@@ -747,7 +748,12 @@ public class CatalogOpExecutor {
       Table existingTable = db.getTable(tblName);
       if (existingTable != null) {
         LOG.debug("EventId: {} Table {} was not added since "
-            + "it already exists in catalog", eventId, existingTable.getFullName());
+                + "it already exists in catalog.", eventId, existingTable.getFullName());
+        if (existingTable.getCreateEventId() != eventId) {
+          LOG.warn("Existing table {} create event Id: {} does not match the "
+                  + "event id: {}", existingTable.getFullName(),
+              existingTable.getCreateEventId(), eventId);
+        }
         return false;
       }
       // table does not exist in catalog. We must make sure that the table was
@@ -891,6 +897,53 @@ public class CatalogOpExecutor {
   }
 
   /**
+   * Updates the catalog db with alteredMsDb. To do so, first acquire lock on catalog db
+   * and then metastore db is updated. Also update the event id in the db.
+   * No update is done if the catalog db is already synced till this event id
+   * @param eventId: HMS event id for this alter db operation
+   * @param alteredMsDb: metastore db to update in catalogd
+   * @return: true if metastore db was updated in catalog's db
+   *          false otherwise
+   */
+  public boolean alterDbIfExists(long eventId,
+      org.apache.hadoop.hive.metastore.api.Database alteredMsDb) {
+    Preconditions.checkNotNull(alteredMsDb);
+    String dbName = alteredMsDb.getName();
+    Db dbToAlter = catalog_.getDb(dbName);
+    if (dbToAlter == null) {
+      LOG.debug("Event id: {}, not altering db {} since it does not exist in catalogd",
+          eventId, dbName);
+      return false;
+    }
+    boolean syncToLatestEventId =
+        BackendConfig.INSTANCE.enableSyncToLatestEventOnDdls();
+    boolean dbLocked = false;
+    try {
+      tryLock(dbToAlter, String.format("alter db from event id: %s", eventId));
+      catalog_.getLock().writeLock().unlock();
+      dbLocked = true;
+      if (syncToLatestEventId && dbToAlter.getLastSyncedEventId() >= eventId) {
+        LOG.debug("Not altering db {} from event id: {} since db is already synced "
+                + "till event id: {}", dbName, eventId, dbToAlter.getLastSyncedEventId());
+        return false;
+      }
+      boolean success = catalog_.updateDbIfExists(alteredMsDb);
+      if (success) {
+        dbToAlter.setLastSyncedEventId(eventId);
+      }
+      return success;
+    } catch (Exception e) {
+      LOG.error("Event id: {}, failed to alter db {}. Error message: {}", eventId, dbName,
+          e.getMessage());
+      return false;
+    } finally {
+      if (dbLocked) {
+        dbToAlter.getLock().unlock();
+      }
+    }
+  }
+
+  /**
    * Execute the ALTER TABLE command according to the TAlterTableParams and refresh the
    * table metadata, except for RENAME, ADD PARTITION and DROP PARTITION. This call is
    * thread-safe, i.e. concurrent operations on the same table are serialized.
@@ -1888,7 +1941,7 @@ public class CatalogOpExecutor {
     try {
       MetastoreEvent event = catalog_
           .getMetastoreEventProcessor().getEventsFactory()
-          .get(events.get(events.size() - 1));
+          .get(events.get(events.size() - 1), null);
       Preconditions.checkState(event instanceof CreateDatabaseEvent);
       return new Pair<>(events.get(0).getEventId(),
           ((CreateDatabaseEvent) event).getDatabase());
@@ -1920,7 +1973,7 @@ public class CatalogOpExecutor {
     try {
       MetastoreEvent event = catalog_
           .getMetastoreEventProcessor().getEventsFactory()
-          .get(events.get(events.size() - 1));
+          .get(events.get(events.size() - 1), null);
       Preconditions.checkState(event instanceof CreateTableEvent);
       return new Pair<>(events.get(0).getEventId(),
           ((CreateTableEvent) event).getTable());
@@ -1946,7 +1999,7 @@ public class CatalogOpExecutor {
     for (NotificationEvent notificationEvent : events) {
       try {
         MetastoreEvent event = catalog_
-            .getMetastoreEventProcessor().getEventsFactory().get(notificationEvent);
+            .getMetastoreEventProcessor().getEventsFactory().get(notificationEvent, null);
         Preconditions.checkState(event instanceof AlterTableEvent);
         AlterTableEvent alterEvent = (AlterTableEvent) event;
         if (!alterEvent.isRename()) continue;
@@ -1977,7 +2030,7 @@ public class CatalogOpExecutor {
     for (NotificationEvent event : events) {
       try {
         MetastoreEvent metastoreEvent = catalog_
-            .getMetastoreEventProcessor().getEventsFactory().get(event);
+            .getMetastoreEventProcessor().getEventsFactory().get(event, null);
         Preconditions.checkState(metastoreEvent instanceof AddPartitionEvent);
         Long eventId = metastoreEvent.getEventId();
         for (Partition part : ((AddPartitionEvent) metastoreEvent).getPartitions()) {
@@ -2009,7 +2062,7 @@ public class CatalogOpExecutor {
     for (NotificationEvent notificationEvent : events) {
       try {
         MetastoreEvent event = catalog_
-            .getMetastoreEventProcessor().getEventsFactory().get(notificationEvent);
+            .getMetastoreEventProcessor().getEventsFactory().get(notificationEvent, null);
         Preconditions.checkState(event instanceof DropPartitionEvent);
         Long eventId = notificationEvent.getEventId();
         List<Map<String, String>> droppedPartitions = ((DropPartitionEvent) event)
@@ -3931,13 +3984,22 @@ public class CatalogOpExecutor {
       throw new CatalogException(
           "Partition event " + eventId + " received on a non-hdfs table");
     }
+    boolean syncToLatestEventId =
+        BackendConfig.INSTANCE.enableSyncToLatestEventOnDdls();
     try {
       tryWriteLock(table, reason);
+      if (syncToLatestEventId && table.getLastSyncedEventId() >= eventId) {
+        LOG.info("Not adding partitions from event id: {} since table {} is already "
+                + "synced till event id {}", eventId, table.getFullName(),
+            table.getLastSyncedEventId());
+        return 0;
+      }
       long newCatalogVersion = catalog_.incrementAndGetCatalogVersion();
       catalog_.getLock().writeLock().unlock();
       HdfsTable hdfsTable = (HdfsTable) table;
       List<Partition> partitionsToAdd = filterPartitionsToAddFromEvent(eventId, hdfsTable,
           partitions);
+      int partitionsAdded = 0;
       if (!partitionsToAdd.isEmpty()) {
         LOG.debug("Found {}/{} partitions to add in table {} from event {}",
             partitionsToAdd.size(), partitions.size(), table.getFullName(), eventId);
@@ -3950,15 +4012,20 @@ public class CatalogOpExecutor {
           addHdfsPartitions(metaStoreClient, table, partitionsToAdd, partToEventId);
         }
         table.setCatalogVersion(newCatalogVersion);
-        return partitionsToAdd.size();
+        partitionsAdded = partitionsToAdd.size();
       }
-      return 0;
+      if (syncToLatestEventId) {
+        table.setLastSyncedEventId(eventId);
+      }
+      return partitionsAdded;
     } catch (InternalException | UnsupportedEncodingException e) {
       throw new CatalogException(
           "Unable to add partition for table " + table.getFullName(), e);
     } finally {
       UnlockWriteLockIfErronouslyLocked();
-      table.releaseWriteLock();
+      if (table.isWriteLockedByCurrentThread()) {
+        table.releaseWriteLock();
+      }
     }
   }
 
@@ -4056,8 +4123,18 @@ public class CatalogOpExecutor {
     if (!(table instanceof HdfsTable)) {
       throw new CatalogException("Partition event received on a non-hdfs table");
     }
+    boolean syncToLatestEventId =
+        BackendConfig.INSTANCE.enableSyncToLatestEventOnDdls();
+
+    boolean errorOccured = false;
     try {
       tryWriteLock(table, reason);
+      if (syncToLatestEventId && table.getLastSyncedEventId() >= eventId) {
+        LOG.info("Not dropping partitions from event id: {} since table {} is already "
+                + "synced till event id {}", eventId, table.getFullName(),
+            table.getLastSyncedEventId());
+        return 0;
+      }
       long newCatalogVersion = catalog_.incrementAndGetCatalogVersion();
       catalog_.getLock().writeLock().unlock();
       HdfsTable hdfsTable = (HdfsTable) table;
@@ -4111,9 +4188,16 @@ public class CatalogOpExecutor {
       table.setCatalogVersion(newCatalogVersion);
       return allTPartKeyVals.size();
     } catch (InternalException e) {
+      errorOccured = true;
       throw new CatalogException(
           "Unable to add partition for table " + table.getFullName(), e);
     } finally {
+      //  set table's last sycned event id  if no error occurred and
+      //  table's last synced event id < current event id
+      if (!errorOccured && syncToLatestEventId &&
+          table.getLastSyncedEventId() < eventId) {
+        table.setLastSyncedEventId(eventId);
+      }
       UnlockWriteLockIfErronouslyLocked();
       table.releaseWriteLock();
     }
@@ -4182,10 +4266,20 @@ public class CatalogOpExecutor {
     if (!(table instanceof HdfsTable)) {
       throw new CatalogException("Partition event received on a non-hdfs table");
     }
+    boolean syncToLatestEventId =
+        BackendConfig.INSTANCE.enableSyncToLatestEventOnDdls();
+
+    boolean errorOccured = false;
     try {
       tryWriteLock(table, reason);
       long newCatalogVersion = catalog_.incrementAndGetCatalogVersion();
       catalog_.getLock().writeLock().unlock();
+      if (syncToLatestEventId && table.getLastSyncedEventId() >= eventId) {
+        LOG.info("Not reloading partition from event id: {} since table {} is already "
+                + "synced till event id {}", eventId, table.getFullName(),
+            table.getLastSyncedEventId());
+        return 0;
+      }
       HdfsTable hdfsTable = (HdfsTable) table;
       // some partitions from the event or the table itself
       // may not exist in HMS anymore. Hence, we collect the names here and re-fetch
@@ -4206,9 +4300,16 @@ public class CatalogOpExecutor {
       LOG.info("Could not reload {} partitions of table {}", partsFromEvent.size(),
           table.getFullName(), e);
     } catch (InternalException e) {
+      errorOccured = true;
       throw new CatalogException(
           "Could not acquire lock on the table " + table.getFullName(), e);
     } finally {
+      //  set table's last sycned event id  if no error occurred and
+      //  table's last synced event id < current event id
+      if (!errorOccured && syncToLatestEventId &&
+          table.getLastSyncedEventId() < eventId) {
+        table.setLastSyncedEventId(eventId);
+      }
       UnlockWriteLockIfErronouslyLocked();
       table.releaseWriteLock();
     }
@@ -4323,6 +4424,14 @@ public class CatalogOpExecutor {
       tryWriteLock(table, reason);
       long newCatalogVersion = catalog_.incrementAndGetCatalogVersion();
       catalog_.getLock().writeLock().unlock();
+      boolean syncToLatestEvent =
+          BackendConfig.INSTANCE.enableSyncToLatestEventOnDdls();
+      if (hdfsTable.getLastSyncedEventId() > eventId) {
+        LOG.info("EventId: {}, Skipping addition of committed writeIds and partitions"
+            + " reload for table {} since it is already synced till eventId: {}",
+            eventId, hdfsTable.getFullName(), hdfsTable.getLastSyncedEventId());
+        return 0;
+      }
       Preconditions.checkState(previousWriteIdList != null,
           "Write id list of table %s should not be null", table.getFullName());
       // get a copy of previous write id list
@@ -4356,6 +4465,9 @@ public class CatalogOpExecutor {
       if (partsToRefresh.isEmpty()) {
         LOG.info("Not reloading partitions of table {} for event {} since the cache is "
             + "already up-to-date", table.getFullName(), eventId);
+        if (syncToLatestEvent) {
+          hdfsTable.setLastSyncedEventId(eventId);
+        }
         return 0;
       }
       // set write id as committed before reload the partitions so that we can get
@@ -4368,6 +4480,9 @@ public class CatalogOpExecutor {
             metaStoreClient.getHiveClient(), partsToRefresh, true, reason);
       }
       hdfsTable.setCatalogVersion(newCatalogVersion);
+      if (syncToLatestEvent) {
+        hdfsTable.setLastSyncedEventId(eventId);
+      }
       return numOfPartsReloaded;
     } catch (InternalException e) {
       throw new CatalogException(
@@ -4781,7 +4896,7 @@ public class CatalogOpExecutor {
    * to make sure that we don't keep adding events when they are not being garbage
    * collected.
    */
-  private void addToDeleteEventLog(long eventId, String objectKey) {
+  public void addToDeleteEventLog(long eventId, String objectKey) {
     if (!catalog_.isEventProcessingActive()) {
       LOG.trace("Not adding event {}:{} since events processing is not active", eventId,
           objectKey);
@@ -5942,7 +6057,8 @@ public class CatalogOpExecutor {
               //     ACID tables, there is a Jira to cover this: HIVE-22062.
               //   2: If no need for a full table reload then fetch partition level
               //     writeIds and reload only the ones that changed.
-              updatedThriftTable = catalog_.reloadTable(tbl, req, resultType, cmdString);
+              updatedThriftTable = catalog_.reloadTable(tbl, req, resultType, cmdString,
+                  -1);
             }
           } else {
             // Table was loaded from scratch, so it's already "refreshed".
@@ -6460,7 +6576,7 @@ public class CatalogOpExecutor {
    * TODO: Track object IDs to
    * know when a table has been dropped and re-created with the same name.
    */
-  private Table getExistingTable(String dbName, String tblName, String reason)
+  public Table getExistingTable(String dbName, String tblName, String reason)
       throws CatalogException {
     // passing null validWriteIdList makes sure that we return the table if it is
     // already loaded.
diff --git a/fe/src/main/java/org/apache/impala/service/JniCatalog.java b/fe/src/main/java/org/apache/impala/service/JniCatalog.java
index bd0a479..179f0e1 100644
--- a/fe/src/main/java/org/apache/impala/service/JniCatalog.java
+++ b/fe/src/main/java/org/apache/impala/service/JniCatalog.java
@@ -38,6 +38,8 @@ import org.apache.impala.catalog.Function;
 import org.apache.impala.catalog.MetaStoreClientPool;
 import org.apache.impala.catalog.MetaStoreClientPool.MetaStoreClient;
 import org.apache.impala.catalog.events.ExternalEventsProcessor;
+import org.apache.impala.catalog.events.MetastoreEvents.EventFactoryForSyncToLatestEvent;
+import org.apache.impala.catalog.events.MetastoreEvents.MetastoreEventFactory;
 import org.apache.impala.catalog.events.MetastoreEventsProcessor;
 import org.apache.impala.catalog.events.NoOpEventProcessor;
 import org.apache.impala.catalog.metastore.CatalogMetastoreServer;
@@ -148,6 +150,9 @@ public class JniCatalog {
     authzManager_ = authzFactory.newAuthorizationManager(catalog_);
     catalog_.setAuthzManager(authzManager_);
     catalogOpExecutor_ = new CatalogOpExecutor(catalog_, authzConfig, authzManager_);
+    MetastoreEventFactory eventFactory =
+        new EventFactoryForSyncToLatestEvent(catalogOpExecutor_);
+    catalog_.setEventFactoryForSyncToLatestEvent(eventFactory);
     ExternalEventsProcessor eventsProcessor = getEventsProcessor(metaStoreClientPool,
         catalogOpExecutor_);
     catalog_.setMetastoreEventProcessor(eventsProcessor);
@@ -155,6 +160,7 @@ public class JniCatalog {
     catalogMetastoreServer_ = getCatalogMetastoreServer(catalogOpExecutor_);
     catalog_.setCatalogMetastoreServer(catalogMetastoreServer_);
     catalogMetastoreServer_.start();
+
     try {
       catalog_.reset();
     } catch (CatalogException e) {
diff --git a/fe/src/test/java/org/apache/impala/catalog/AlterDatabaseTest.java b/fe/src/test/java/org/apache/impala/catalog/AlterDatabaseTest.java
index 80f1991..b187c4a 100644
--- a/fe/src/test/java/org/apache/impala/catalog/AlterDatabaseTest.java
+++ b/fe/src/test/java/org/apache/impala/catalog/AlterDatabaseTest.java
@@ -86,11 +86,9 @@ public class AlterDatabaseTest {
    */
   @BeforeClass
   public static void setUpTest() throws ImpalaException {
-    catalog_ = new ImpaladTestCatalog(CatalogServiceTestCatalog.create());
-    catalogOpExecutor_ =
-        new CatalogOpExecutor(catalog_.getSrcCatalog(),
-            new NoopAuthorizationFactory().getAuthorizationConfig(),
-            new NoopAuthorizationManager());
+    CatalogServiceTestCatalog testSrcCatalog = CatalogServiceTestCatalog.create();
+    catalog_ = new ImpaladTestCatalog(testSrcCatalog);
+    catalogOpExecutor_ = testSrcCatalog.getCatalogOpExecutor();
   }
 
   /**
diff --git a/fe/src/test/java/org/apache/impala/catalog/MetastoreApiTestUtils.java b/fe/src/test/java/org/apache/impala/catalog/MetastoreApiTestUtils.java
new file mode 100644
index 0000000..ae98d9f
--- /dev/null
+++ b/fe/src/test/java/org/apache/impala/catalog/MetastoreApiTestUtils.java
@@ -0,0 +1,169 @@
+// 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;
+
+import static org.junit.Assert.assertFalse;
+
+import com.google.common.base.Preconditions;
+import com.google.common.collect.Lists;
+import java.util.ArrayList;
+import java.util.List;
+import java.util.Map;
+
+import org.apache.hadoop.hive.metastore.api.Catalog;
+import org.apache.hadoop.hive.metastore.api.CurrentNotificationEventId;
+import org.apache.hadoop.hive.metastore.api.Database;
+import org.apache.hadoop.hive.metastore.api.FieldSchema;
+import org.apache.hadoop.hive.metastore.api.InsertEventRequestData;
+import org.apache.hadoop.hive.metastore.api.MetaException;
+import org.apache.hadoop.hive.metastore.api.NotificationEvent;
+import org.apache.hadoop.hive.metastore.api.Partition;
+import org.apache.hadoop.hive.metastore.api.SerDeInfo;
+import org.apache.hadoop.hive.metastore.api.StorageDescriptor;
+import org.apache.hadoop.hive.metastore.api.PrincipalType;
+import org.apache.hadoop.hive.metastore.messaging.AlterTableMessage;
+import org.apache.impala.catalog.MetaStoreClientPool.MetaStoreClient;
+import org.apache.thrift.TException;
+
+/**
+ * Utils class to create/get objects in HMS
+ */
+public class MetastoreApiTestUtils {
+
+  public static Database createHmsDatabaseObject(String catName,
+      String dbName, Map<String, String> params) {
+    Database database = new Database();
+    if (catName != null) database.setCatalogName(catName);
+    database.setName(dbName);
+    database.setDescription("Notification test database");
+    database.setOwnerName("NotificationOwner");
+    database.setOwnerType(PrincipalType.USER);
+    if (params != null && !params.isEmpty()) {
+      database.setParameters(params);
+    }
+    return database;
+  }
+
+  public static void addDatabaseParametersInHms(MetaStoreClient msClient, String dbName,
+      String key, String val) throws TException {
+    Database msDb = msClient.getHiveClient().getDatabase(dbName);
+    assertFalse(key + " already exists in the database parameters",
+        msDb.getParameters().containsKey(key));
+    msDb.putToParameters(key, val);
+    msClient.getHiveClient().alterDatabase(dbName, msDb);
+  }
+
+  /*
+  Get a metastore external table object
+   */
+  public static org.apache.hadoop.hive.metastore.api.Table getTestTable(String catName,
+      String dbName, String tblName, Map<String, String> params, boolean isPartitioned)
+      throws MetaException {
+    return getTestTable(catName, dbName, tblName, params, isPartitioned,
+        org.apache.hadoop.hive.metastore.TableType.EXTERNAL_TABLE.toString());
+  }
+
+  /*
+  Get a metastore table object. If tableType is non-null, its valid values
+  are: EXTERNAL_TABLE, MANAGED_TABLE
+  */
+  public static org.apache.hadoop.hive.metastore.api.Table getTestTable(String catName,
+      String dbName, String tblName, Map<String, String> params, boolean isPartitioned,
+      String tableType) throws MetaException {
+    org.apache.hadoop.hive.metastore.api.Table tbl =
+        new org.apache.hadoop.hive.metastore.api.Table();
+    if (catName != null) tbl.setCatName(catName);
+    tbl.setDbName(dbName);
+    tbl.setTableName(tblName);
+    tbl.putToParameters("tblParamKey", "tblParamValue");
+    List<FieldSchema> cols = Lists.newArrayList(
+        new FieldSchema("c1","string","c1 description"),
+        new FieldSchema("c2", "string","c2 description"));
+
+    StorageDescriptor sd = new StorageDescriptor();
+    sd.setCols(cols);
+    sd.setInputFormat(HdfsFileFormat.PARQUET.inputFormat());
+    sd.setOutputFormat(HdfsFileFormat.PARQUET.outputFormat());
+
+    SerDeInfo serDeInfo = new SerDeInfo();
+    serDeInfo.setSerializationLib(HdfsFileFormat.PARQUET.serializationLib());
+    sd.setSerdeInfo(serDeInfo);
+    tbl.setSd(sd);
+
+    if (params != null && !params.isEmpty()) {
+      tbl.setParameters(params);
+    }
+    if (isPartitioned) {
+      List<FieldSchema> pcols = Lists.newArrayList(
+          new FieldSchema("p1","string","partition p1 description"));
+      tbl.setPartitionKeys(pcols);
+    }
+    if (tableType != null) {
+      Preconditions.checkArgument(tableType.equals(
+          org.apache.hadoop.hive.metastore.TableType.MANAGED_TABLE.toString()) ||
+          tableType.equals(org.apache.hadoop.hive.metastore.TableType.EXTERNAL_TABLE
+              .toString()), "Invalid table type " + tableType);
+      tbl.setTableType(tableType);
+    }
+    return tbl;
+  }
+
+  public static void createDatabase(MetaStoreClient msClient, String catName,
+      String dbName, Map<String, String> params) throws TException {
+    Database database = createHmsDatabaseObject(catName, dbName, params);
+    msClient.getHiveClient().createDatabase(database);
+  }
+
+  /*
+   Creates an external table since tableType is null
+   */
+  public static void createTable(MetaStoreClient msClient, String catName, String dbName,
+      String tblName, Map<String, String> params, boolean isPartitioned)
+      throws TException {
+    createTable(msClient, catName, dbName, tblName, params, isPartitioned, null);
+  }
+
+  /*
+  Creates a table of type tableType. Valid values of tableType are: EXTERNAL_TABLE,
+  MANAGED_TABLE
+   */
+  public static void createTable(MetaStoreClient msClient, String catName,
+      String dbName, String tblName, Map<String, String> params,
+      boolean isPartitioned, String tableType) throws TException {
+    org.apache.hadoop.hive.metastore.api.Table tbl =
+        getTestTable(catName, dbName, tblName, params, isPartitioned, tableType);
+    msClient.getHiveClient().createTable(tbl);
+  }
+
+  public static void addPartitions(MetaStoreClient msClient, String dbName,
+      String tblName, List<List<String>> partitionValues) throws TException {
+    // int i = 0;
+    List<Partition> partitions = new ArrayList(partitionValues.size());
+    org.apache.hadoop.hive.metastore.api.Table msTable =
+        msClient.getHiveClient().getTable(dbName, tblName);
+    for (List<String> partVals : partitionValues) {
+      Partition partition = new Partition();
+      partition.setDbName(msTable.getDbName());
+      partition.setTableName(msTable.getTableName());
+      partition.setSd(msTable.getSd().deepCopy());
+      partition.setValues(partVals);
+      partitions.add(partition);
+    }
+    msClient.getHiveClient().add_partitions(partitions);
+  }
+}
diff --git a/fe/src/test/java/org/apache/impala/catalog/events/EventsProcessorStressTest.java b/fe/src/test/java/org/apache/impala/catalog/events/EventsProcessorStressTest.java
index 6b4fccc..da3f9c9 100644
--- a/fe/src/test/java/org/apache/impala/catalog/events/EventsProcessorStressTest.java
+++ b/fe/src/test/java/org/apache/impala/catalog/events/EventsProcessorStressTest.java
@@ -58,7 +58,7 @@ public class EventsProcessorStressTest {
   // use a fixed seed value to make the test repeatable
   private static final Random random = new Random(117);
 
-  private static CatalogServiceCatalog catalog_;
+  private static CatalogServiceTestCatalog catalog_;
   private static MetastoreEventsProcessor eventsProcessor_;
   private static final String testDbPrefix_ = "events_stress_db_";
   private static final String testTblPrefix_ = "stress_test_tbl_";
@@ -107,9 +107,7 @@ public class EventsProcessorStressTest {
   @BeforeClass
   public static void setupTestEnv() throws Exception {
     catalog_ = CatalogServiceTestCatalog.create();
-    CatalogOpExecutor catalogOpExecutor = new CatalogOpExecutor(catalog_,
-        new NoopAuthorizationFactory().getAuthorizationConfig(),
-        new NoopAuthorizationManager());
+    CatalogOpExecutor catalogOpExecutor = catalog_.getCatalogOpExecutor();
     try (MetaStoreClient metaStoreClient = catalog_.getMetaStoreClient()) {
       CurrentNotificationEventId currentNotificationId =
           metaStoreClient.getHiveClient().getCurrentNotificationEventId();
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 baad1a0..96c84fa 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
@@ -78,12 +78,12 @@ import org.apache.impala.catalog.DatabaseNotFoundException;
 import org.apache.impala.catalog.Db;
 import org.apache.impala.catalog.FeCatalogUtils;
 import org.apache.impala.catalog.FeFsPartition;
-import org.apache.impala.catalog.HdfsFileFormat;
 import org.apache.impala.catalog.HdfsPartition;
 import org.apache.impala.catalog.HdfsTable;
 import org.apache.impala.catalog.IncompleteTable;
 import org.apache.impala.catalog.MetaStoreClientPool;
 import org.apache.impala.catalog.MetaStoreClientPool.MetaStoreClient;
+import org.apache.impala.catalog.MetastoreApiTestUtils;
 import org.apache.impala.catalog.ScalarFunction;
 import org.apache.impala.catalog.Table;
 import org.apache.impala.catalog.Type;
@@ -183,16 +183,14 @@ import org.mockito.Mockito;
 public class MetastoreEventsProcessorTest {
   private static final String TEST_DB_NAME = "events_test_db";
 
-  private static CatalogServiceCatalog catalog_;
+  private static CatalogServiceTestCatalog catalog_;
   private static CatalogOpExecutor catalogOpExecutor_;
   private static MetastoreEventsProcessor eventsProcessor_;
 
   @BeforeClass
   public static void setUpTestEnvironment() throws TException, ImpalaException {
     catalog_ = CatalogServiceTestCatalog.create();
-    catalogOpExecutor_ = new CatalogOpExecutor(catalog_,
-        new NoopAuthorizationFactory().getAuthorizationConfig(),
-        new NoopAuthorizationManager());
+    catalogOpExecutor_ = catalog_.getCatalogOpExecutor();
     try (MetaStoreClient metaStoreClient = catalog_.getMetaStoreClient()) {
       CurrentNotificationEventId currentNotificationId =
           metaStoreClient.getHiveClient().getCurrentNotificationEventId();
@@ -431,6 +429,7 @@ public class MetastoreEventsProcessorTest {
     String tblToBeDropped = "tbl_to_be_dropped";
     createTable(tblToBeDropped, true);
     createTable("tbl_to_be_dropped_unpartitioned", false);
+
     // create 2 partitions
     List<List<String>> partVals = new ArrayList<>(2);
     partVals.add(Arrays.asList("1"));
@@ -526,6 +525,13 @@ public class MetastoreEventsProcessorTest {
     assertNotNull("Db should have been found after create database statement",
         catalog_.getDb(TEST_DB_NAME));
     eventsProcessor_.processEvents();
+    Db db = catalog_.getDb(TEST_DB_NAME);
+    long createEventId = db.getCreateEventId();
+    long beforeLastSyncedEventId = db.getLastSyncedEventId();
+    if (BackendConfig.INSTANCE.enableSyncToLatestEventOnDdls()) {
+      assertEquals(createEventId, beforeLastSyncedEventId);
+    }
+
     long numberOfSelfEventsBefore =
         eventsProcessor_.getMetrics()
             .getCounter(MetastoreEventsProcessor.EVENTS_SKIPPED_METRIC)
@@ -539,11 +545,23 @@ public class MetastoreEventsProcessorTest {
     assertEquals(
         newOwnerUser, catalog_.getDb(TEST_DB_NAME).getMetaStoreDb().getOwnerName());
 
+    if (BackendConfig.INSTANCE.enableSyncToLatestEventOnDdls()) {
+      // self events should reset last synced event id
+      assertTrue(catalog_.getDb(TEST_DB_NAME).getLastSyncedEventId() >
+          beforeLastSyncedEventId);
+    }
+    beforeLastSyncedEventId = db.getLastSyncedEventId();
+
     alterDbSetOwnerFromImpala(TEST_DB_NAME, newOwnerRole, TOwnerType.ROLE);
     eventsProcessor_.processEvents();
     assertEquals(
         newOwnerRole, catalog_.getDb(TEST_DB_NAME).getMetaStoreDb().getOwnerName());
 
+    if (BackendConfig.INSTANCE.enableSyncToLatestEventOnDdls()) {
+      assertTrue(catalog_.getDb(TEST_DB_NAME).getLastSyncedEventId() >
+          beforeLastSyncedEventId);
+    }
+
     long selfEventsCountAfter =
         eventsProcessor_.getMetrics()
             .getCounter(MetastoreEventsProcessor.EVENTS_SKIPPED_METRIC)
@@ -607,6 +625,7 @@ public class MetastoreEventsProcessorTest {
         catalog_.getTable(TEST_DB_NAME, testTblName));
     // create a non-partitioned table
     createTable(testTblName, false);
+
     eventsProcessor_.processEvents();
     assertNotNull("Catalog should have a incomplete instance of table after CREATE_TABLE "
             + "event is received",
@@ -617,6 +636,7 @@ public class MetastoreEventsProcessorTest {
     // test partitioned table case
     final String testPartitionedTbl = "testCreateTableEventPartitioned";
     createTable(testPartitionedTbl, true);
+
     eventsProcessor_.processEvents();
     assertNotNull("Catalog should have create a incomplete table after receiving "
             + "CREATE_TABLE event",
@@ -631,6 +651,7 @@ public class MetastoreEventsProcessorTest {
     createDatabaseFromImpala(TEST_DB_NAME, null);
     eventsProcessor_.processEvents();
     createTable("createondroppeddb", false);
+
     dropDatabaseCascadeFromImpala(TEST_DB_NAME);
     eventsProcessor_.processEvents();
     assertEquals(EventProcessorStatus.ACTIVE, eventsProcessor_.getStatus());
@@ -733,6 +754,66 @@ public class MetastoreEventsProcessorTest {
   }
 
   /**
+   * Tests insert events into partitioned table and asserts that
+   * last synced event id is advanced after the insert
+   */
+  @Test
+  public void testPartitionedTblInsertSyncToLatestEvent() throws Exception {
+    String tblName = "test_insert_into_part_tbl_sync_to_latest_event";
+    testTblInsertsSyncToLatestEvent(TEST_DB_NAME, tblName, true);
+  }
+
+  /**
+   * Tests insert events into non partitioned table and asserts that
+   * last synced event id is advanced after the insert
+   */
+  @Test
+  public void testNonPartitionedTblInsertSyncToLatestEvent() throws Exception {
+    String tblName = "test_insert_into_non_part_tbl_sync_to_latest_event";
+    testTblInsertsSyncToLatestEvent(TEST_DB_NAME, tblName, false);
+  }
+
+  private void testTblInsertsSyncToLatestEvent(String dbName, String tblName,
+      boolean isPartitioned) throws Exception {
+    // Test insert into partition
+    boolean prevFlagVal = BackendConfig.INSTANCE.enableSyncToLatestEventOnDdls();
+    boolean invalidateHMSFlag = BackendConfig.INSTANCE.invalidateCatalogdHMSCacheOnDDLs();
+    try {
+      BackendConfig.INSTANCE.setEnableSyncToLatestEventOnDdls(true);
+      BackendConfig.INSTANCE.setInvalidateCatalogdHMSCacheOnDDLs(false);
+      createDatabase(dbName, null);
+      createTable(dbName, tblName, isPartitioned);
+      eventsProcessor_.processEvents();
+      assertNotNull("Table " + tblName + " not present in catalog",
+          catalog_.getTableNoThrow(dbName, tblName));
+      long lastSyncedEventIdBefore =
+          catalog_.getTable(dbName, tblName).getLastSyncedEventId();
+      assertTrue("expected lastSyncedEventIdBefore to be > 0",
+          lastSyncedEventIdBefore > 0);
+      testInsertEvents(dbName, tblName, isPartitioned);
+
+      Table tbl = catalog_.getTable(dbName, tblName);
+      long lastSyncedEventIdAfter = tbl.getLastSyncedEventId();
+      long currentEventIdHms = -1;
+      try (MetaStoreClient metaStoreClient = catalog_.getMetaStoreClient()) {
+        currentEventIdHms =
+            metaStoreClient.getHiveClient().getCurrentNotificationEventId().getEventId();
+      }
+      assertTrue(String.format("for table %s, expected lastSyncedEventIdBefore %s to be "
+              + "less than lastSyncedEventIdAfter %s", tblName, lastSyncedEventIdBefore,
+          lastSyncedEventIdAfter), lastSyncedEventIdBefore < lastSyncedEventIdAfter);
+
+      assertTrue(String.format("for table %s, expected lastSyncedEventIdAfter %s to be "
+                  + "less than equal to currentEventIdHms %s", tblName,
+              lastSyncedEventIdAfter, currentEventIdHms),
+          lastSyncedEventIdAfter <= currentEventIdHms);
+    } finally {
+      BackendConfig.INSTANCE.setEnableSyncToLatestEventOnDdls(prevFlagVal);
+      BackendConfig.INSTANCE.setInvalidateCatalogdHMSCacheOnDDLs(invalidateHMSFlag);
+    }
+  }
+
+  /**
    * Test insert from impala. Insert into table and partition from impala
    * should be treated as self-event.
    */
@@ -831,6 +912,10 @@ public class MetastoreEventsProcessorTest {
         .getOrLoadTable(TEST_DB_NAME, tableToInsertPart, "test", null);
     HdfsTable multiInsertTbl = (HdfsTable) catalog_
         .getOrLoadTable(TEST_DB_NAME, tableToInsertMulPart, "test", null);
+    // get lastSyncedEvent id of tables before inserting from Impala
+    long insertTblLatestEventIdBefore = insertTbl.getLastSyncedEventId();
+    long multiInsertTblLatestEventIdBefore = multiInsertTbl.getLastSyncedEventId();
+
     // we copy files from the src tbl and then issue a insert catalogOp to simulate a
     // insert operation
     List<String> tbl1Part1Files = copyFiles(allTypes.getFileSystem(),
@@ -876,6 +961,16 @@ public class MetastoreEventsProcessorTest {
     assertEquals(tbl2Part2Files, getFilesFromEvent(events.get(3)));
     eventsProcessor_.processEvents();
 
+    if (BackendConfig.INSTANCE.enableSyncToLatestEventOnDdls()) {
+      // when processing self events, eventProcessor should update last synced event id
+      // for the table
+      assertTrue(
+          catalog_.getTable(TEST_DB_NAME, tableToInsertPart).getLastSyncedEventId() >
+              insertTblLatestEventIdBefore);
+      assertTrue(
+          catalog_.getTable(TEST_DB_NAME, tableToInsertMulPart).getLastSyncedEventId() >
+              multiInsertTblLatestEventIdBefore);
+    }
     // Test insert into table
     String unpartitionedTbl = "tbl_to_insert";
     // create table self-event 5
@@ -1157,11 +1252,19 @@ public class MetastoreEventsProcessorTest {
 
     // check refresh after alter table add parameter
     loadTable(testTblName);
+    Table testTbl = catalog_.getTable(TEST_DB_NAME, testTblName);
+    long lastSyncedEventIdBefore = testTbl.getLastSyncedEventId();
     alterTableAddParameter(testTblName, "somekey", "someval");
     eventsProcessor_.processEvents();
     assertFalse("Table should have been refreshed after alter table add parameter",
         catalog_.getTable(TEST_DB_NAME, testTblName)
                 instanceof IncompleteTable);
+    if (BackendConfig.INSTANCE.enableSyncToLatestEventOnDdls()) {
+      assertTrue("Table's last synced event id should have been advanced after"
+              + " processing alter table event",
+          catalog_.getTable(TEST_DB_NAME, testTblName).getLastSyncedEventId() >
+              lastSyncedEventIdBefore);
+    }
     // check refresh after alter table add col
     loadTable(testTblName);
     alterTableAddCol(testTblName, "newCol", "int", "null");
@@ -1335,10 +1438,8 @@ public class MetastoreEventsProcessorTest {
    */
   @Test
   public void testEventProcessorFetchAfterHMSRestart() throws ImpalaException {
-    CatalogServiceCatalog catalog = CatalogServiceTestCatalog.create();
-    CatalogOpExecutor catalogOpExecutor = new CatalogOpExecutor(catalog,
-        new NoopAuthorizationFactory().getAuthorizationConfig(),
-        new NoopAuthorizationManager());
+    CatalogServiceTestCatalog catalog = CatalogServiceTestCatalog.create();
+    CatalogOpExecutor catalogOpExecutor = catalog.getCatalogOpExecutor();
     MetastoreEventsProcessor fetchProcessor =
         new HMSFetchNotificationsEventProcessor(catalogOpExecutor,
             eventsProcessor_.getCurrentEventId(), 2L);
@@ -1605,7 +1706,12 @@ public class MetastoreEventsProcessorTest {
             MetastoreEventPropertyKey.DISABLE_EVENT_HMS_SYNC.getKey(),
             tblTransition.second);
         eventsProcessor_.processEvents();
-        assertEquals(EventProcessorStatus.NEEDS_INVALIDATE, eventsProcessor_.getStatus());
+        // if sync to latest event id is enabled, then the event is skipped
+        // since table does not exist in cache
+        if (!BackendConfig.INSTANCE.enableSyncToLatestEventOnDdls()) {
+          assertEquals(EventProcessorStatus.NEEDS_INVALIDATE,
+              eventsProcessor_.getStatus());
+        }
         // issue a catalog reset to make sure that table comes back again and event
         // processing is active
         catalog_.reset();
@@ -1724,10 +1830,9 @@ public class MetastoreEventsProcessorTest {
     }
 
     org.apache.hadoop.hive.metastore.api.Table tableBefore =
-        getTestTable(null, dbName, tblName, beforeParams, false, null);
+        MetastoreApiTestUtils.getTestTable(null, dbName, tblName, beforeParams, false);
     org.apache.hadoop.hive.metastore.api.Table tableAfter =
-        getTestTable(null, dbName, tblName, afterParams, false, null);
-
+        MetastoreApiTestUtils.getTestTable(null, dbName, tblName, afterParams, false);
     Map<String, String> dbParams = new HashMap<>(1);
     if (dbFlag != null) {
       dbParams.put(MetastoreEventPropertyKey.DISABLE_EVENT_HMS_SYNC.getKey(), dbFlag);
@@ -1750,7 +1855,7 @@ public class MetastoreEventsProcessorTest {
     // issue a dummy alter table by adding a param
     afterParams.put("dummy", "value");
     org.apache.hadoop.hive.metastore.api.Table nextTable =
-        getTestTable(null, dbName, tblName, afterParams, false, null);
+        MetastoreApiTestUtils.getTestTable(null, dbName, tblName, afterParams, false);
     NotificationEvent nextNotification =
         createFakeAlterTableNotification(dbName, tblName, tableAfter, nextTable);
     alterTableEvent =
@@ -1875,7 +1980,7 @@ public class MetastoreEventsProcessorTest {
    */
   @Test
   public void testEventMetricsWhenNotConfigured() {
-    CatalogServiceCatalog testCatalog = CatalogServiceTestCatalog.create();
+    CatalogServiceTestCatalog testCatalog = CatalogServiceTestCatalog.create();
     assertTrue("Events processed is not expected to be configured for this test",
         testCatalog.getMetastoreEventProcessor() instanceof NoOpEventProcessor);
     TEventProcessorMetrics response = testCatalog.getEventProcessorMetrics();
@@ -2184,14 +2289,24 @@ public class MetastoreEventsProcessorTest {
     eventsProcessor_.processEvents();
     final String testTblName = "testSelfEventsForTable";
     createTableFromImpala(TEST_DB_NAME, testTblName, true);
+    Table testTbl = catalog_.getTable(TEST_DB_NAME, testTblName);
+    assertTrue(testTbl.getCreateEventId() == testTbl.getLastSyncedEventId());
     eventsProcessor_.processEvents();
     long numberOfSelfEventsBefore = eventsProcessor_.getMetrics()
         .getCounter(MetastoreEventsProcessor.EVENTS_SKIPPED_METRIC).getCount();
 
+    long lastSyncedEventIdBefore = testTbl.getLastSyncedEventId();
     alterTableSetTblPropertiesFromImpala(testTblName);
     eventsProcessor_.processEvents();
     confirmTableIsLoaded(TEST_DB_NAME, testTblName);
 
+    if (BackendConfig.INSTANCE.enableSyncToLatestEventOnDdls()) {
+      // event processor when processing a self event should update
+      // last synced event id as well
+      assertTrue(catalog_.getTable(TEST_DB_NAME, testTblName).getLastSyncedEventId() >
+          lastSyncedEventIdBefore);
+    }
+
     // add a col
     alterTableAddColsFromImpala(
         TEST_DB_NAME, testTblName, "newCol", TPrimitiveType.STRING);
@@ -2239,6 +2354,8 @@ public class MetastoreEventsProcessorTest {
     eventsProcessor_.processEvents();
     confirmTableIsLoaded(TEST_DB_NAME, newTblName);
 
+    Table newTbl = catalog_.getTable(TEST_DB_NAME, newTblName);
+    lastSyncedEventIdBefore = newTbl.getLastSyncedEventId();
     org.apache.hadoop.hive.metastore.api.Table hmsTbl = catalog_.getTable(TEST_DB_NAME,
         newTblName).getMetaStoreTable();
     assertNotNull("Location is expected to be set to proceed forward in the test",
@@ -2249,6 +2366,11 @@ public class MetastoreEventsProcessorTest {
     eventsProcessor_.processEvents();
     confirmTableIsLoaded(TEST_DB_NAME, newTblName);
 
+    if (BackendConfig.INSTANCE.enableSyncToLatestEventOnDdls()) {
+      // self event processing by event processor should update last synced event id
+      assertTrue(catalog_.getTable(TEST_DB_NAME, newTblName).getLastSyncedEventId() >
+          lastSyncedEventIdBefore);
+    }
     //TODO add test for alterview
     //add test for alterCommentOnTableOrView
 
@@ -2327,7 +2449,8 @@ public class MetastoreEventsProcessorTest {
       List<MetastoreEvent> mockEvents = createMockEvents(100, 10,
           eventType, TEST_DB_NAME, testTblName, eventMessage);
       MetastoreEventFactory eventFactory = eventsProcessor_.getEventsFactory();
-      List<MetastoreEvent> batch = eventFactory.createBatchEvents(mockEvents);
+      List<MetastoreEvent> batch = eventFactory.createBatchEvents(mockEvents,
+          eventsProcessor_.getMetrics());
       assertEquals(1, batch.size());
       assertTrue(batch.get(0) instanceof BatchPartitionEvent);
       BatchPartitionEvent batchEvent = (BatchPartitionEvent) batch.get(0);
@@ -2347,7 +2470,7 @@ public class MetastoreEventsProcessorTest {
       mockEvents.addAll(createMockEvents(22, 3,
           eventType, TEST_DB_NAME, testTblName, eventMessage));
 
-      batch = eventFactory.createBatchEvents(mockEvents);
+      batch = eventFactory.createBatchEvents(mockEvents, eventsProcessor_.getMetrics());
       assertEquals(4, batch.size());
       MetastoreEvent batch1 = batch.get(0);
       assertEquals(3, ((BatchPartitionEvent) batch1).getBatchEvents().size());
@@ -2368,7 +2491,8 @@ public class MetastoreEventsProcessorTest {
       mockEvents.addAll(createMockEvents(101, 1, eventType, "db1",
           testTblName, eventMessage));
 
-      List<MetastoreEvent> batchEvents = eventFactory.createBatchEvents(mockEvents);
+      List<MetastoreEvent> batchEvents = eventFactory.createBatchEvents(mockEvents,
+          eventsProcessor_.getMetrics());
       assertEquals(2, batchEvents.size());
       for (MetastoreEvent event : batchEvents) {
         if (eventType.equalsIgnoreCase("ALTER_PARTITION")) {
@@ -2383,7 +2507,8 @@ public class MetastoreEventsProcessorTest {
           testTblName, eventMessage);
       mockEvents.addAll(createMockEvents(101, 1, eventType, TEST_DB_NAME,
           "testtbl", eventMessage));
-      batchEvents = eventFactory.createBatchEvents(mockEvents);
+      batchEvents = eventFactory.createBatchEvents(mockEvents,
+          eventsProcessor_.getMetrics());
       assertEquals(2, batchEvents.size());
       for (MetastoreEvent event : batchEvents) {
         if (eventType.equalsIgnoreCase("ALTER_PARTITION")) {
@@ -2421,7 +2546,7 @@ public class MetastoreEventsProcessorTest {
         createMockEvents(startEventId + mockEvents.size(), 5, "ALTER_PARTITION",
             TEST_DB_NAME, testTblName, eventTypeToMessage.get("ALTER_PARTITION")));
     List<MetastoreEvent> batchedEvents = eventsProcessor_.getEventsFactory()
-        .createBatchEvents(mockEvents);
+        .createBatchEvents(mockEvents, eventsProcessor_.getMetrics());
     assertEquals(7, batchedEvents.size());
     // batch 1 should contain 3 AlterPartitionEvent
     BatchPartitionEvent<AlterPartitionEvent> batch1 = (BatchPartitionEvent
@@ -2468,7 +2593,8 @@ public class MetastoreEventsProcessorTest {
     }
     List<MetastoreEvent> metastoreEvents = new ArrayList<>();
     for (NotificationEvent notificationEvent : mockEvents) {
-      metastoreEvents.add(eventsProcessor_.getEventsFactory().get(notificationEvent));
+      metastoreEvents.add(eventsProcessor_.getEventsFactory().get(notificationEvent,
+          eventsProcessor_.getMetrics()));
     }
     return metastoreEvents;
   }
@@ -2480,6 +2606,8 @@ public class MetastoreEventsProcessorTest {
     try {
       final TBackendGflags stubCfg = origCfg.deepCopy();
       stubCfg.setHms_event_incremental_refresh_transactional_table(true);
+      // also enable sync to latest event on ddls
+      stubCfg.setEnable_sync_to_latest_event_on_ddls(true);
       BackendConfig.create(stubCfg);
 
       createDatabase(TEST_DB_NAME, null);
@@ -2498,6 +2626,8 @@ public class MetastoreEventsProcessorTest {
     try {
       final TBackendGflags stubCfg = origCfg.deepCopy();
       stubCfg.setHms_event_incremental_refresh_transactional_table(true);
+      // also enable sync to latest event on ddls
+      stubCfg.setEnable_sync_to_latest_event_on_ddls(true);
       BackendConfig.create(stubCfg);
 
       createDatabase(TEST_DB_NAME, null);
@@ -2520,13 +2650,23 @@ public class MetastoreEventsProcessorTest {
     }
     eventsProcessor_.processEvents();
     loadTable(tblName);
-    Table tbl = catalog_.getTable(TEST_DB_NAME, tblName);
+    HdfsTable tbl = (HdfsTable)catalog_.getTable(TEST_DB_NAME, tblName);
+    long lastSyncedEventId = tbl.getLastSyncedEventId();
 
     try (MetaStoreClient client = catalog_.getMetaStoreClient()) {
       long txnId = MetastoreShim.openTransaction(client.getHiveClient());
       long writeId = MetastoreShim.allocateTableWriteId(client.getHiveClient(),
           txnId, TEST_DB_NAME, tblName);
       eventsProcessor_.processEvents();
+
+      if (isPartitioned) {
+        // open and alloc write id events should be processed by event processor
+        // these events are ignored for non-partitioned tables
+        assertTrue(String.format("Expected last synced event id: %s for table %s to be "
+                + "greater than %s", tbl.getLastSyncedEventId(), tbl.getFullName(),
+            lastSyncedEventId), tbl.getLastSyncedEventId() > lastSyncedEventId);
+      }
+      lastSyncedEventId = tbl.getLastSyncedEventId();
       ValidWriteIdList writeIdList = tbl.getValidWriteIds();
       assertFalse(writeIdList.isWriteIdValid(writeId));
       assertFalse(writeIdList.isWriteIdAborted(writeId));
@@ -2554,9 +2694,18 @@ public class MetastoreEventsProcessorTest {
         // to reduce memory footprint so the writeIdList won't be up-to-date until next
         // table reloading.
         assertEquals(0, numFiles);
+        if (isPartitioned) {
+          assertTrue(String.format("Expected last synced event id: %s for table %s to be "
+                  + "greater than %s", tbl.getLastSyncedEventId(), tbl.getFullName(),
+              lastSyncedEventId), tbl.getLastSyncedEventId() > lastSyncedEventId);
+        }
       } else {
         assertTrue(writeIdList.isWriteIdValid(writeId));
         assertEquals(1, numFiles);
+        assertTrue(String.format("Expected last synced event id: %s for table %s to be "
+                + "greater than %s", tbl.getLastSyncedEventId(), tbl.getFullName(),
+            lastSyncedEventId), tbl.getLastSyncedEventId() > lastSyncedEventId);
+
       }
     }
   }
@@ -2771,8 +2920,10 @@ public class MetastoreEventsProcessorTest {
     partVals.add(Arrays.asList("2"));
     addPartitions(TEST_DB_NAME, testTblName, partVals);
     eventsProcessor_.processEvents();
+    Table catalogTbl = catalog_.getTable(TEST_DB_NAME, testTblName);
     List<TPartitionKeyValue> partKeyVals = new ArrayList<>();
     partKeyVals.add(new TPartitionKeyValue("p1", "1"));
+    long lastSyncedEventIdBefore = catalogTbl.getLastSyncedEventId();
     alterTableSetPartitionPropertiesFromImpala(testTblName, partKeyVals);
     HdfsPartition hdfsPartition =
         catalog_.getHdfsPartition(TEST_DB_NAME, testTblName, partKeyVals);
@@ -2780,7 +2931,13 @@ public class MetastoreEventsProcessorTest {
     assertEquals("dummyValue1", hdfsPartition.getParameters().get("dummyKey1"));
 
     eventsProcessor_.processEvents();
-    Table catalogTbl = catalog_.getTable(TEST_DB_NAME, testTblName);
+    if (BackendConfig.INSTANCE.enableSyncToLatestEventOnDdls()) {
+      // event processor when processing a self event should update last synced
+      // event id to this event id
+      assertTrue(catalog_.getTable(TEST_DB_NAME, testTblName).getLastSyncedEventId()
+          > lastSyncedEventIdBefore);
+    }
+
     confirmTableIsLoaded(TEST_DB_NAME, testTblName);
     // we check for the object hash of the HDFSPartition to make sure that it was not
     // refresh
@@ -2818,25 +2975,16 @@ public class MetastoreEventsProcessorTest {
         catalog_.getHdfsPartition(TEST_DB_NAME, testTblName, partKeyVals2));
   }
 
-  private void createDatabase(String dbName, Map<String, String> params)
-      throws TException {
-    createDatabase(null, dbName, params);
+  private void createDatabase(String catName, String dbName,
+      Map<String, String> params) throws TException {
+    try(MetaStoreClient msClient = catalog_.getMetaStoreClient()) {
+      MetastoreApiTestUtils.createDatabase(msClient, catName, dbName, params);
+    }
   }
 
-  private void createDatabase(String catName, String dbName, Map<String, String> params)
+  private void createDatabase(String dbName, Map<String, String> params)
       throws TException {
-    Database database = new Database();
-    if (catName != null) database.setCatalogName(catName);
-    database.setName(dbName);
-    database.setDescription("Notification test database");
-    database.setOwnerName("NotificationOwner");
-    database.setOwnerType(PrincipalType.USER);
-    if (params != null && !params.isEmpty()) {
-      database.setParameters(params);
-    }
-    try (MetaStoreClient msClient = catalog_.getMetaStoreClient()) {
-      msClient.getHiveClient().createDatabase(database);
-    }
+    createDatabase(null, dbName, params);
   }
 
   private void createHiveCatalog(String catName) throws TException {
@@ -2862,11 +3010,7 @@ public class MetastoreEventsProcessorTest {
 
   private void addDatabaseParameters(String key, String val) throws TException {
     try (MetaStoreClient msClient = catalog_.getMetaStoreClient()) {
-      Database msDb = msClient.getHiveClient().getDatabase(TEST_DB_NAME);
-      assertFalse(key + " already exists in the database parameters",
-          msDb.getParameters().containsKey(key));
-      msDb.putToParameters(key, val);
-      msClient.getHiveClient().alterDatabase(TEST_DB_NAME, msDb);
+      MetastoreApiTestUtils.addDatabaseParametersInHms(msClient, TEST_DB_NAME, key, val);
     }
   }
 
@@ -2877,63 +3021,22 @@ public class MetastoreEventsProcessorTest {
     }
   }
 
-  private void createTransactionalTable(
-      String dbName, String tblName, boolean isPartitioned) throws TException {
+
+  private void createTransactionalTable(String dbName, String tblName,
+      boolean isPartitioned) throws TException {
     Map<String, String> params = new HashMap<>();
     params.put("transactional", "true");
     params.put("transactional_properties", "insert_only");
     createTable(null, dbName, tblName, params, isPartitioned, "MANAGED_TABLE");
   }
 
-  private void createTable(String catName, String dbName, String tblName,
-      Map<String, String> params, boolean isPartitioned, String tableType)
-      throws TException {
-    org.apache.hadoop.hive.metastore.api.Table
-        tbl = getTestTable(catName, dbName, tblName, params, isPartitioned, tableType);
-
-    try (MetaStoreClient msClient = catalog_.getMetaStoreClient()) {
-      msClient.getHiveClient().createTable(tbl);
-    }
-  }
-
-  private org.apache.hadoop.hive.metastore.api.Table getTestTable(String catName,
-      String dbName, String tblName, Map<String, String> params, boolean isPartitioned,
-      String tableType) throws MetaException {
-    org.apache.hadoop.hive.metastore.api.Table tbl =
-        new org.apache.hadoop.hive.metastore.api.Table();
-    if (catName != null) tbl.setCatName(catName);
-    tbl.setDbName(dbName);
-    tbl.setTableName(tblName);
-    tbl.putToParameters("tblParamKey", "tblParamValue");
-    List<FieldSchema> cols = Lists.newArrayList(
-        new FieldSchema("c1","string","c1 description"),
-        new FieldSchema("c2", "string","c2 description"));
-
-    StorageDescriptor sd = new StorageDescriptor();
-    sd.setCols(cols);
-    sd.setInputFormat(HdfsFileFormat.PARQUET.inputFormat());
-    sd.setOutputFormat(HdfsFileFormat.PARQUET.outputFormat());
-
-    SerDeInfo serDeInfo = new SerDeInfo();
-    serDeInfo.setSerializationLib(HdfsFileFormat.PARQUET.serializationLib());
-    sd.setSerdeInfo(serDeInfo);
-    tbl.setSd(sd);
-
-    if (params != null && !params.isEmpty()) {
-      tbl.setParameters(params);
-    }
-    if (isPartitioned) {
-      List<FieldSchema> pcols = Lists.newArrayList(
-          new FieldSchema("p1","string","partition p1 description"));
-      tbl.setPartitionKeys(pcols);
-    }
-    if (tableType != null) {
-      Preconditions.checkArgument(tableType.equals(TableType.MANAGED_TABLE.toString())
-              || tableType.equals(TableType.EXTERNAL_TABLE.toString()),
-          "Invalid table type " + tableType);
-      tbl.setTableType(tableType);
+  private void createTable(String catName, String dbName,
+      String tblName, Map<String, String> params, boolean isPartitioned,
+      String tableType) throws TException {
+    try(MetaStoreClient msClient = catalog_.getMetaStoreClient()) {
+      MetastoreApiTestUtils.createTable(msClient, catName, dbName, tblName, params,
+          isPartitioned, tableType);
     }
-    return tbl;
   }
 
   /**
@@ -2952,15 +3055,20 @@ public class MetastoreEventsProcessorTest {
   /**
    * Creates db from Impala
    */
-  private void createDatabaseFromImpala(String dbName, String desc)
-      throws ImpalaException {
+  public static void createDatabaseFromImpala(CatalogOpExecutor catalogOpExecutor,
+      String dbName, String desc) throws ImpalaException {
     TDdlExecRequest req = new TDdlExecRequest();
     req.setDdl_type(TDdlType.CREATE_DATABASE);
     TCreateDbParams createDbParams = new TCreateDbParams();
     createDbParams.setDb(dbName);
     createDbParams.setComment(desc);
     req.setCreate_db_params(createDbParams);
-    catalogOpExecutor_.execDdlRequest(req);
+    catalogOpExecutor.execDdlRequest(req);
+  }
+
+  private void createDatabaseFromImpala(String dbName, String desc)
+      throws ImpalaException {
+    createDatabaseFromImpala(catalogOpExecutor_, dbName, desc);
   }
 
   /**
@@ -3016,8 +3124,8 @@ public class MetastoreEventsProcessorTest {
    * Creates a table using CatalogOpExecutor to simulate a DDL operation from Impala
    * client
    */
-  private void createTableFromImpala(String dbName, String tblName,
-      Map<String, String> tblParams, boolean isPartitioned)
+  public static void createTableFromImpala(CatalogOpExecutor opExecutor, String dbName,
+      String tblName, Map<String, String> tblParams, boolean isPartitioned)
       throws ImpalaException {
     TDdlExecRequest req = new TDdlExecRequest();
     req.setDdl_type(TDdlType.CREATE_TABLE);
@@ -3041,7 +3149,12 @@ public class MetastoreEventsProcessorTest {
       createTableParams.setPartition_columns(partitionColumns);
     }
     req.setCreate_table_params(createTableParams);
-    catalogOpExecutor_.execDdlRequest(req);
+    opExecutor.execDdlRequest(req);
+  }
+
+  private void createTableFromImpala(String dbName, String tblName,
+      Map<String, String> tblParams, boolean isPartitioned) throws ImpalaException {
+    createTableFromImpala(catalogOpExecutor_, dbName, tblName, tblParams, isPartitioned);
   }
 
   /**
@@ -3409,7 +3522,7 @@ public class MetastoreEventsProcessorTest {
     return tUpdateCatalogRequest;
   }
 
-  private TColumn getScalarColumn(String colName, TPrimitiveType type) {
+  private static TColumn getScalarColumn(String colName, TPrimitiveType type) {
     TTypeNode tTypeNode = new TTypeNode(TTypeNodeType.SCALAR);
     tTypeNode.setScalar_type(new TScalarType(type));
     TColumnType columnType = new TColumnType(Arrays.asList(tTypeNode));
@@ -3623,22 +3736,8 @@ public class MetastoreEventsProcessorTest {
   private void addPartitions(String dbName, String tblName,
       List<List<String>> partitionValues)
       throws TException {
-    int i = 0;
-    List<Partition> partitions = new ArrayList<>(partitionValues.size());
-    try (MetaStoreClient msClient = catalog_.getMetaStoreClient()) {
-      org.apache.hadoop.hive.metastore.api.Table msTable =
-          msClient.getHiveClient().getTable(dbName, tblName);
-      for (List<String> partVals : partitionValues) {
-        Partition partition = new Partition();
-        partition.setDbName(msTable.getDbName());
-        partition.setTableName(msTable.getTableName());
-        partition.setSd(msTable.getSd().deepCopy());
-        partition.setValues(partVals);
-        partitions.add(partition);
-      }
-    }
-    try (MetaStoreClient metaStoreClient = catalog_.getMetaStoreClient()) {
-      metaStoreClient.getHiveClient().add_partitions(partitions);
+    try(MetaStoreClient client = catalog_.getMetaStoreClient()) {
+      MetastoreApiTestUtils.addPartitions(client, dbName, tblName, partitionValues);
     }
   }
 
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 511150e..a823981 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,16 +17,12 @@
 
 package org.apache.impala.catalog.events;
 
-import java.util.Collections;
-import java.util.List;
 import org.apache.hadoop.hive.metastore.api.CurrentNotificationEventId;
 import org.apache.hadoop.hive.metastore.api.NotificationEvent;
 import org.apache.hadoop.hive.metastore.api.NotificationEventResponse;
 import org.apache.impala.catalog.CatalogException;
-import org.apache.impala.catalog.CatalogServiceCatalog;
-import org.apache.impala.catalog.MetaStoreClientPool.MetaStoreClient;
+import org.apache.impala.common.Metrics;
 import org.apache.impala.service.CatalogOpExecutor;
-import org.apache.thrift.TException;
 
 /**
  * A test MetastoreEventProcessor which executes in the same thread. Useful for testing
@@ -34,13 +30,18 @@ import org.apache.thrift.TException;
  */
 public class SynchronousHMSEventProcessorForTests extends MetastoreEventsProcessor {
 
-  SynchronousHMSEventProcessorForTests(
+  public SynchronousHMSEventProcessorForTests(
       CatalogOpExecutor catalogOpExecutor, long startSyncFromId,
       long pollingFrequencyInSec) throws CatalogException {
     super(catalogOpExecutor, startSyncFromId, pollingFrequencyInSec);
   }
 
   @Override
+  public Metrics getMetrics() {
+    return super.getMetrics();
+  }
+
+  @Override
   public void startScheduler() {
     // nothing to do here; there is no background thread for this processor
   }
diff --git a/fe/src/test/java/org/apache/impala/catalog/metastore/AbstractCatalogMetastoreTest.java b/fe/src/test/java/org/apache/impala/catalog/metastore/AbstractCatalogMetastoreTest.java
index 5ac0199..0dc150d 100644
--- a/fe/src/test/java/org/apache/impala/catalog/metastore/AbstractCatalogMetastoreTest.java
+++ b/fe/src/test/java/org/apache/impala/catalog/metastore/AbstractCatalogMetastoreTest.java
@@ -44,7 +44,7 @@ import org.junit.Test;
  * server endpoint at a random available port.
  */
 public abstract class AbstractCatalogMetastoreTest {
-  protected static CatalogServiceCatalog catalog_;
+  protected static CatalogServiceTestCatalog catalog_;
   protected static CatalogOpExecutor catalogOpExecutor_;
   protected static CatalogMetastoreServer catalogMetastoreServer_;
   protected static HiveMetaStoreClient catalogHmsClient_;
@@ -53,9 +53,7 @@ public abstract class AbstractCatalogMetastoreTest {
   @BeforeClass
   public static void setup() throws Exception {
     catalog_ = CatalogServiceTestCatalog.create();
-    catalogOpExecutor_ = new CatalogOpExecutor(catalog_,
-        new NoopAuthorizationFactory().getAuthorizationConfig(),
-        new NoopAuthorizationManager());
+    catalogOpExecutor_ = catalog_.getCatalogOpExecutor();
     catalogMetastoreServer_ = new CatalogTestMetastoreServer(
         catalogOpExecutor_);
     catalog_.setCatalogMetastoreServer(catalogMetastoreServer_);
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
new file mode 100644
index 0000000..33eeea1
--- /dev/null
+++ b/fe/src/test/java/org/apache/impala/catalog/metastore/CatalogHmsSyncToLatestEventIdTest.java
@@ -0,0 +1,777 @@
+// 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.metastore;
+
+import com.google.common.collect.Lists;
+import java.util.HashMap;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.hive.metastore.HiveMetaStoreClient;
+import org.apache.hadoop.hive.metastore.conf.MetastoreConf;
+import org.apache.hadoop.hive.metastore.conf.MetastoreConf.ConfVars;
+import org.apache.hadoop.hive.metastore.api.Database;
+import org.apache.hadoop.hive.metastore.api.Partition;
+import org.apache.hadoop.hive.metastore.api.NoSuchObjectException;
+
+import org.apache.impala.catalog.*;
+import org.apache.impala.catalog.MetaStoreClientPool.MetaStoreClient;
+import org.apache.impala.catalog.events.ExternalEventsProcessor;
+import org.apache.impala.catalog.events.MetastoreEventsProcessor;
+import org.apache.impala.catalog.MetastoreApiTestUtils;
+import org.apache.impala.catalog.events.NoOpEventProcessor;
+import org.apache.impala.catalog.events.SynchronousHMSEventProcessorForTests;
+import org.apache.impala.service.BackendConfig;
+import org.apache.impala.service.CatalogOpExecutor;
+import org.apache.impala.testutil.CatalogServiceTestCatalog;
+import org.apache.impala.testutil.CatalogTestMetastoreServer;
+import org.apache.thrift.TException;
+import org.junit.After;
+import org.junit.AfterClass;
+import org.junit.Assume;
+import org.junit.Before;
+import org.junit.BeforeClass;
+import org.junit.Ignore;
+import org.junit.Test;
+import org.junit.runner.RunWith;
+import org.junit.runners.Parameterized;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import org.apache.hadoop.hive.metastore.api.FieldSchema;
+import org.apache.hadoop.hive.metastore.api.StorageDescriptor;
+import org.apache.hadoop.hive.metastore.api.CurrentNotificationEventId;
+import org.apache.impala.catalog.CatalogException;
+import org.apache.impala.compat.MetastoreShim;
+
+import static org.junit.Assert.assertFalse;
+import static org.junit.Assert.assertTrue;
+
+
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.List;
+import java.util.Map;
+
+/**
+ * This class mainly tests ddl operations from catalogHmsClient and asserts that
+ * db/table is synced to the latest event id. It also processes the same events
+ * from MetastoreEventProcessor and asserts that those events are skipped since
+ * they have already been processed by catalogHmsClient
+ */
+@RunWith(Parameterized.class)
+public class CatalogHmsSyncToLatestEventIdTest extends AbstractCatalogMetastoreTest {
+    private static String TEST_DB_NAME = "sync_to_latest_events_test_db";
+    private static Logger LOG =
+            LoggerFactory.getLogger(CatalogHmsSyncToLatestEventIdTest.class);
+    protected static CatalogServiceTestCatalog catalog_;
+    protected static CatalogOpExecutor catalogOpExecutor_;
+    protected static CatalogMetastoreServer catalogMetastoreServer_;
+    protected static HiveMetaStoreClient catalogHmsClient_;
+    private static SynchronousHMSEventProcessorForTests eventsProcessor_;
+    protected static final Configuration CONF = MetastoreConf.newMetastoreConf();
+    private String tableType_;
+    private static String managedTableType =
+        org.apache.hadoop.hive.metastore.TableType.MANAGED_TABLE.toString();
+    private static String externalTableType =
+        org.apache.hadoop.hive.metastore.TableType.EXTERNAL_TABLE.toString();
+    private static boolean flagEnableCatalogCache ,flagInvalidateCache,
+        flagSyncToLatestEventId;
+
+    @BeforeClass
+    public static void setup() throws Exception {
+        catalog_ = CatalogServiceTestCatalog.create();
+        catalogOpExecutor_ = catalog_.getCatalogOpExecutor();
+        catalogMetastoreServer_ = new CatalogTestMetastoreServer(
+                catalogOpExecutor_);
+        catalog_.setCatalogMetastoreServer(catalogMetastoreServer_);
+        try (MetaStoreClient metaStoreClient = catalog_.getMetaStoreClient()) {
+            CurrentNotificationEventId currentNotificationId =
+                    metaStoreClient.getHiveClient().getCurrentNotificationEventId();
+            eventsProcessor_ = new SynchronousHMSEventProcessorForTests(
+                    catalogOpExecutor_, currentNotificationId.getEventId(), 10L);
+            eventsProcessor_.start();
+        }
+        // Don't set event processor in catalog because
+        // sync to latest event id should work even if event processor
+        // is disabled
+        catalogMetastoreServer_.start();
+        MetastoreConf.setVar(CONF, ConfVars.THRIFT_URIS,
+                "thrift://localhost:" + catalogMetastoreServer_.getPort());
+        // metastore clients which connect to catalogd's HMS endpoint need this
+        // configuration set since the forwarded HMS call use catalogd's HMS client
+        // not the end-user's UGI.
+        CONF.set("hive.metastore.execute.setugi", "false");
+        catalogHmsClient_ = new HiveMetaStoreClient(CONF);
+        assertTrue("Event processor should not be set",
+            catalog_.getMetastoreEventProcessor() instanceof NoOpEventProcessor);
+        // get previous values of flag to be set in cleanup
+        flagEnableCatalogCache = BackendConfig.INSTANCE.enableCatalogdHMSCache();
+        flagInvalidateCache = BackendConfig.INSTANCE.invalidateCatalogdHMSCacheOnDDLs();
+        flagSyncToLatestEventId = BackendConfig.INSTANCE.enableSyncToLatestEventOnDdls();
+    }
+
+    @AfterClass
+    public static void cleanUp() throws Exception {
+        // in cleanup, set flag's values to previous value
+        BackendConfig.INSTANCE.setEnableCatalogdHMSCache(flagEnableCatalogCache);
+        BackendConfig.INSTANCE.setEnableSyncToLatestEventOnDdls(flagInvalidateCache);
+        BackendConfig.INSTANCE.setInvalidateCatalogdHMSCacheOnDDLs(
+            flagSyncToLatestEventId);
+        if (eventsProcessor_ != null) {
+            eventsProcessor_.shutdown();
+        }
+        catalogMetastoreServer_.stop();
+        catalog_.close();
+
+    }
+
+    @After
+    public void afterTest() throws TException, CatalogException {
+        String dbName = TEST_DB_NAME;
+        try {
+            try (MetaStoreClient msClient = catalog_.getMetaStoreClient()) {
+                msClient.getHiveClient().dropDatabase(dbName, true, true, true);
+            }
+            catalog_.removeDb(dbName);
+        } catch (NoSuchObjectException e) {
+            LOG.error("database {} does not exist in catalogd", dbName);
+            catalog_.removeDb(dbName);
+        }
+    }
+
+    @Before
+    public void beforeTest() throws Exception {
+        BackendConfig.INSTANCE.setEnableCatalogdHMSCache(true);
+        BackendConfig.INSTANCE.setEnableSyncToLatestEventOnDdls(true);
+        BackendConfig.INSTANCE.setInvalidateCatalogdHMSCacheOnDDLs(false);
+    }
+
+    public CatalogHmsSyncToLatestEventIdTest(String tableType) {
+        tableType_ = tableType;
+    }
+
+    @Parameterized.Parameters
+    public static String[] createTableTypes() {
+        return new String[] {managedTableType, externalTableType};
+    }
+    @Test
+    public void testCreateDatabase() throws Exception {
+        LOG.info("Executing testCreateDatabase");
+        String dbName = "test_create_database";
+        try {
+            Database msDb = MetastoreApiTestUtils
+                .createHmsDatabaseObject(null, dbName, null);
+            catalogHmsClient_.createDatabase(msDb);
+            Db db = catalog_.getDb(dbName);
+            assertTrue(db != null);
+            assertTrue(db.getLastSyncedEventId() != -1);
+            assertTrue(db.getLastSyncedEventId() == db.getCreateEventId());
+        } finally {
+            catalogHmsClient_.dropDatabase(dbName, true, true, true);
+            assertTrue("db " + dbName + " should not be present in catalogd",
+                catalog_.getDb(dbName) == null);
+        }
+    }
+
+    @Test
+    public void testAlterDatabase() throws Exception {
+        LOG.info("Executing testAlterDatabase");
+        String dbName = "test_alter_database";
+        try {
+            createDatabaseInCatalog(dbName);
+            eventsProcessor_.processEvents();
+            long lastSkippedEventsCount = eventsProcessor_.getMetrics()
+                .getCounter(MetastoreEventsProcessor.EVENTS_SKIPPED_METRIC).getCount();
+            Db catalogDb = catalog_.getDb(dbName);
+            long prevSyncedEventId = catalogDb.getLastSyncedEventId();
+
+            try (MetaStoreClient msClient = catalog_.getMetaStoreClient()) {
+                MetastoreApiTestUtils.addDatabaseParametersInHms(msClient, dbName,
+                    "key1", "val1");
+            }
+            String newOwner = "new_owner";
+            Database alteredMsDb = getDatabaseInHms(dbName);
+            alteredMsDb.setOwnerName(newOwner);
+            // alter db via catalogHmsClient
+            catalogHmsClient_.alterDatabase(dbName, alteredMsDb);
+
+            catalogDb = catalog_.getDb(dbName);
+            assertTrue(catalogDb.getOwnerUser().equals(newOwner));
+            assertTrue(catalogDb.getMetaStoreDb().getParameters()
+                .get("key1").equals("val1"));
+            assertTrue(catalogDb.getLastSyncedEventId() >
+                prevSyncedEventId);
+            eventsProcessor_.processEvents();
+            long currentSkippedEventsCount = eventsProcessor_.getMetrics()
+                .getCounter(MetastoreEventsProcessor.EVENTS_SKIPPED_METRIC).getCount();
+            assertTrue(lastSkippedEventsCount + 2 == currentSkippedEventsCount);
+        } finally {
+            catalogHmsClient_.dropDatabase(dbName, true, true, true);
+        }
+    }
+
+    @Test
+    public void testAddDropAlterPartitions() throws Exception {
+        LOG.info("Executing testAddDropAlterPartitions");
+        String tblName = "test_add_drop_alter_partitions_" + tableType_ + "_tbl" ;
+        try {
+            createDatabaseInCatalog(TEST_DB_NAME);
+            try {
+                catalogHmsClient_.dropTable(TEST_DB_NAME, tblName, true, true);
+            } catch (Exception e) {
+                LOG.error("Failed to drop table {} from HMS", tblName);
+            }
+            catalogHmsClient_
+                .createTable(MetastoreApiTestUtils.getTestTable(null,
+                TEST_DB_NAME, tblName, null, true, tableType_));
+
+            HdfsTable tbl = getCatalogHdfsTable(TEST_DB_NAME, tblName);
+
+            assertTrue(tbl != null);
+            assertTrue("table's last synced id should not be -1",
+                tbl.getLastSyncedEventId() != -1);
+            assertTrue(tbl.getLastSyncedEventId() == tbl.getCreateEventId());
+
+            eventsProcessor_.processEvents();
+            long lastSkippedCount = eventsProcessor_.getMetrics()
+                .getCounter(MetastoreEventsProcessor.EVENTS_SKIPPED_METRIC).getCount();
+            long prevSyncedEventId = tbl.getLastSyncedEventId();
+            List<List<String> > partVals = new ArrayList<>();
+            partVals.add(Arrays.asList("1"));
+            partVals.add(Arrays.asList("2"));
+            partVals.add(Arrays.asList("3"));
+            addPartitionsInHms(TEST_DB_NAME, tblName, partVals);
+            // added partitions should not reflect in table
+            // stored in catalog cache
+            assertTrue(tbl.getPartitions().size() == 0);
+
+            // alter partition 2 directly in HMS
+            Partition partitionToAlter =
+                getPartitionInHms(TEST_DB_NAME, tblName, Arrays.asList("2"));
+
+            String newLocation = "/path/to/newLocation/";
+            partitionToAlter.getSd().setLocation(newLocation);
+            alterPartitionInHms(TEST_DB_NAME, tblName, partitionToAlter);
+
+            // when dropping partitions from catalogHmsClient, sync to latest
+            // event id adds 3 partitions and drops 1
+            catalogHmsClient_.dropPartition(TEST_DB_NAME, tblName,
+                Arrays.asList("3"), true);
+            tbl = getCatalogHdfsTable(TEST_DB_NAME, tblName);
+            assertTrue("Table should have 2 partitions after dropping 1 "
+                    + "out of 3 partitions", tbl.getPartitions().size() == 2);
+
+            // assert that  partition with new location from cached table
+            // exists
+            FeFsPartition modifiedPartition = null;
+            for (FeFsPartition part : FeCatalogUtils.loadAllPartitions(tbl)) {
+                if (part.getLocation().equals(newLocation)) {
+                    modifiedPartition = part;
+                    break;
+                }
+            }
+            assertTrue(modifiedPartition != null);
+            assertTrue(tbl.getLastSyncedEventId() > prevSyncedEventId);
+            // test that events processor skipped all events
+            // since last synced event
+            eventsProcessor_.processEvents();
+            long currentSkippedCount = eventsProcessor_.getMetrics()
+                .getCounter(MetastoreEventsProcessor.EVENTS_SKIPPED_METRIC).getCount();
+
+            assertTrue( String.format("CurrentSkippedCount %s differs from "
+                        + "lastSkippedCount + 3 %s", currentSkippedCount,
+                    lastSkippedCount),
+                currentSkippedCount == lastSkippedCount + 3);
+        } finally {
+            catalogHmsClient_.dropTable(TEST_DB_NAME, tblName, true, true);
+        }
+    }
+
+    @Test
+    public void testExchangePartition() throws Exception {
+        // run this test only for managed table
+        Assume.assumeTrue(tableType_.equals(managedTableType));
+        LOG.info("Executing testExchangePartition");
+        String srcTblName = "test_exchange_partition_src_" + tableType_ + "_tbl";
+        String destTblName = "test_exchange_partition_dest_" + tableType_ + "_tbl";
+        try {
+            createDatabaseInCatalog(TEST_DB_NAME);
+            // drop tables if already exist
+            catalogHmsClient_.dropTable(TEST_DB_NAME, srcTblName, true, true);
+            catalogHmsClient_.dropTable(TEST_DB_NAME, destTblName, true, true);
+            org.apache.hadoop.hive.metastore.api.Table srcMsTable =
+                MetastoreApiTestUtils.getTestTable(null, TEST_DB_NAME,
+                    srcTblName, null, true, managedTableType);
+            catalogHmsClient_.createTable(srcMsTable);
+            // add 3 partitions but only in HMS
+            List<List<String> > srcPartVals = new ArrayList<>();
+            srcPartVals.add(Arrays.asList("1"));
+            srcPartVals.add(Arrays.asList("2"));
+            srcPartVals.add(Arrays.asList("3"));
+            addPartitionsInHms(TEST_DB_NAME, srcTblName, srcPartVals);
+            Map<String, String> partitionSpec =
+                getPartitionSpec(srcMsTable, Arrays.asList("1"));
+            org.apache.hadoop.hive.metastore.api.Table destMsTable =
+                MetastoreApiTestUtils.getTestTable(null,
+                TEST_DB_NAME, destTblName, null, true, managedTableType);
+            catalogHmsClient_.createTable(destMsTable);
+            // add partition with val 4 but only in HMS
+            List<List<String> > destPartVals = new ArrayList<>();
+            destPartVals.add(Arrays.asList("4"));
+            addPartitionsInHms(TEST_DB_NAME, destTblName, destPartVals);
+            long eventIdBeforeExchange = getLatestEventIdFromHMS();
+
+            catalogHmsClient_.exchange_partition(partitionSpec,TEST_DB_NAME, srcTblName,
+                TEST_DB_NAME, destTblName);
+            HdfsTable srcCatalogTbl = getCatalogHdfsTable(TEST_DB_NAME, srcTblName);
+            HdfsTable destCatalogTbl = getCatalogHdfsTable(TEST_DB_NAME, destTblName);
+            assertTrue(srcCatalogTbl.getPartitions().size() == 2);
+            assertTrue(destCatalogTbl.getPartitions().size() == 2);
+
+            // assert that part with val 1 does not exist in src table
+            for (FeFsPartition srcPartition :
+                FeCatalogUtils.loadAllPartitions(srcCatalogTbl)) {
+                List<String> partVals =
+                    srcPartition.getPartitionValuesAsStrings(false);
+                assertFalse(partVals.equals(Arrays.asList("1")));
+            }
+            // it is enough to assert that last synced event id of Hdfs table >
+            // event id before exchange partition api
+            assertTrue(srcCatalogTbl.getLastSyncedEventId() > eventIdBeforeExchange);
+            assertTrue(destCatalogTbl.getLastSyncedEventId() > eventIdBeforeExchange);
+        } finally {
+            catalogHmsClient_.dropTable(TEST_DB_NAME, srcTblName, true, true);
+            catalogHmsClient_.dropTable(TEST_DB_NAME, destTblName, true, true);
+        }
+    }
+
+    @Test
+    public void testTableCreateDropCreate() throws Exception {
+        LOG.info("Executing testTableCreateDropCreate");
+        String tblName = "test_create_drop_create_" + tableType_ + "_tbl";
+        String tblNameLowerCase = tblName.toLowerCase();
+        try {
+            createDatabaseInCatalog(TEST_DB_NAME);
+            catalogHmsClient_.dropTable(TEST_DB_NAME, tblName, true, true);
+            catalogHmsClient_.createTable(MetastoreApiTestUtils.getTestTable(null,
+                TEST_DB_NAME, tblName, null, true, tableType_));
+            HdfsTable tbl = (HdfsTable) catalog_.getTable(TEST_DB_NAME, tblNameLowerCase);
+            assertTrue(tbl.isPartitioned());
+            // last synced event id is same as create event id
+            long prevCreateEventId = tbl.getLastSyncedEventId();
+            // drop table from HMS skipping catalog metastore
+            try (MetaStoreClient msClient = catalog_.getMetaStoreClient()) {
+                msClient.getHiveClient().dropTable(TEST_DB_NAME, tblName, true, false);
+            }
+            // recreate table with same name but unpartitioned to distinguish it
+            // from previous table
+            catalogHmsClient_.createTable(MetastoreApiTestUtils.getTestTable(null,
+                TEST_DB_NAME, tblName, null, false, tableType_));
+            HdfsTable currentTbl = (HdfsTable) catalog_.getTable(TEST_DB_NAME,
+                tblNameLowerCase);
+            assertTrue(currentTbl.getLastSyncedEventId() != prevCreateEventId);
+            assertTrue(!currentTbl.isPartitioned());
+        } finally {
+            catalogHmsClient_.dropTable(TEST_DB_NAME, tblName, true, true);
+        }
+    }
+
+    @Test
+    public void testAlterTableNoRename() throws Exception {
+        LOG.info("Executing testAlterTableNoRename");
+        String tblName = "test_alter_table_" + tableType_ + "_tbl";
+        try {
+            createDatabaseInCatalog(TEST_DB_NAME);
+            // drop table if it already exists
+            catalogHmsClient_.dropTable(TEST_DB_NAME, tblName, true, true);
+            catalogHmsClient_.createTable(MetastoreApiTestUtils.getTestTable(null,
+                TEST_DB_NAME, tblName, null, true, tableType_));
+            HdfsTable tbl = getCatalogHdfsTable(TEST_DB_NAME, tblName);
+            eventsProcessor_.processEvents();
+            long lastSkippedEventsCount = eventsProcessor_.getMetrics()
+                .getCounter(MetastoreEventsProcessor.EVENTS_SKIPPED_METRIC).getCount();
+
+            org.apache.hadoop.hive.metastore.api.Table newMsTable =
+                tbl.getMetaStoreTable().deepCopy();
+            List<FieldSchema> cols = Lists.newArrayList(
+                new FieldSchema("c1","string","c1 description"));
+
+            org.apache.hadoop.hive.metastore.api.StorageDescriptor updatedSd =
+                newMsTable.getSd();
+            updatedSd.setCols(cols);
+            newMsTable.setSd(updatedSd);
+
+            // alter table but not from catalogHMSClient so that it is
+            // synced up later
+            try (MetaStoreClient msClient = catalog_.getMetaStoreClient()) {
+                msClient.getHiveClient().alter_table_with_environmentContext(TEST_DB_NAME,
+                    tblName, newMsTable, null);
+            }
+            // assert the cached table's SD is not changed
+            tbl = getCatalogHdfsTable(TEST_DB_NAME, tblName);
+            long prevSyncedEventId = tbl.getLastSyncedEventId();
+            org.apache.hadoop.hive.metastore.api.StorageDescriptor oldSd =
+                tbl.getMetaStoreTable().getSd();
+            assertFalse(oldSd.equals(updatedSd));
+
+            // get the latest table from metastore and alter it via catalogHmsClient
+            org.apache.hadoop.hive.metastore.api.Table latestMsTable =
+                getHmsTable(TEST_DB_NAME, tblName);
+
+            String newOwner = "newOwnerForTestAlterTable";
+            latestMsTable.setOwner(newOwner);
+
+            // alter latest table via catalogHMSClient
+            catalogHmsClient_.alter_table_with_environmentContext(TEST_DB_NAME, tblName,
+                latestMsTable, null);
+
+            // get latest table from the cache
+            HdfsTable updatedTbl = getCatalogHdfsTable(TEST_DB_NAME, tblName);
+            assertTrue(updatedTbl.getOwnerUser().equals(newOwner));
+            assertTrue(updatedTbl.getMetaStoreTable().getSd().equals(updatedSd));
+            assertTrue(
+                updatedTbl.getLastSyncedEventId() > prevSyncedEventId);
+            // assert that alter table events are skipped by event processor
+            eventsProcessor_.processEvents();
+            long currentSkippedEventsCount = eventsProcessor_.getMetrics()
+                .getCounter(MetastoreEventsProcessor.EVENTS_SKIPPED_METRIC).getCount();
+            assertTrue(lastSkippedEventsCount + 2 == currentSkippedEventsCount);
+
+        } finally {
+            catalogHmsClient_.dropTable(TEST_DB_NAME, tblName, true, true);
+        }
+    }
+
+    @Test
+    public void testAlterTableRename() throws Exception {
+        LOG.info("Executing testALterTableRename");
+        String tblName = "test_alter_table_rename_" + tableType_ + "_tbl";
+        String newTblName = tblName + "_new";
+        try {
+            createDatabaseInCatalog(TEST_DB_NAME);
+            catalogHmsClient_.dropTable(TEST_DB_NAME, tblName, true, true);
+            catalogHmsClient_.dropTable(TEST_DB_NAME, newTblName, true, true);
+            catalogHmsClient_.createTable(MetastoreApiTestUtils.getTestTable(null,
+                TEST_DB_NAME, tblName, null, true, tableType_));
+            HdfsTable oldTbl = getCatalogHdfsTable(TEST_DB_NAME, tblName);
+            eventsProcessor_.processEvents();
+            long lastSkippedEventsCount = eventsProcessor_.getMetrics()
+                .getCounter(MetastoreEventsProcessor.EVENTS_SKIPPED_METRIC).getCount();
+            org.apache.hadoop.hive.metastore.api.Table newMsTable =
+                oldTbl.getMetaStoreTable().deepCopy();
+            newMsTable.setTableName(newTblName);
+
+            catalogHmsClient_.alter_table_with_environmentContext(TEST_DB_NAME, tblName,
+                newMsTable, null);
+            // check that old table does not exist in cache
+            assertTrue(catalog_.getTableNoThrow(TEST_DB_NAME, tblName) == null);
+            HdfsTable newTbl = getCatalogHdfsTable(TEST_DB_NAME, newTblName);
+            assertTrue(newTbl != null);
+            assertTrue(newTbl.getLastSyncedEventId() > -1);
+            eventsProcessor_.processEvents();
+            long currentSkippedEventsCount = eventsProcessor_.getMetrics()
+                .getCounter(MetastoreEventsProcessor.EVENTS_SKIPPED_METRIC).getCount();
+            assertTrue( lastSkippedEventsCount + 1 == currentSkippedEventsCount);
+        } finally {
+            catalogHmsClient_.dropTable(TEST_DB_NAME, tblName, true, true);
+            catalogHmsClient_.dropTable(TEST_DB_NAME, newTblName, true, true);
+        }
+    }
+
+    @Test
+    public void testSyncToLatestEventIdFlag() throws Exception {
+        String tblName = "test_sync_to_latest_event_id_flag_" + tableType_ + "_tbl";
+        LOG.info("Executing testSyncToLatestEventIdFlag");
+        boolean prevFlag =
+            BackendConfig.INSTANCE.enableSyncToLatestEventOnDdls();
+        try {
+            createDatabaseInCatalog(TEST_DB_NAME);
+            catalogHmsClient_.dropTable(TEST_DB_NAME, tblName, true, true);
+            catalogHmsClient_
+                .createTable(MetastoreApiTestUtils.getTestTable(null,
+                    TEST_DB_NAME, tblName, null, true, tableType_));
+
+            HdfsTable tbl = getCatalogHdfsTable(TEST_DB_NAME, tblName);
+            long lastSyncedEventId = tbl.getLastSyncedEventId();
+            // set sync to latest event id flag to false so that further
+            // table is not synced for HMS operations
+            BackendConfig.INSTANCE.setEnableSyncToLatestEventOnDdls(false);
+            List<List<String> > partVals = new ArrayList<>();
+            partVals.add(Arrays.asList("1"));
+            partVals.add(Arrays.asList("2"));
+            partVals.add(Arrays.asList("3"));
+            addPartitionsInHms(TEST_DB_NAME, tblName, partVals);
+
+            catalogHmsClient_.dropPartition(TEST_DB_NAME, tblName,
+                Arrays.asList("3"), true);
+            tbl = getCatalogHdfsTable(TEST_DB_NAME, tblName);
+            // with sync to latest event id flag set to false
+            // last synced event id for a table should not change
+            assertTrue(lastSyncedEventId == tbl.getLastSyncedEventId());
+
+        } finally {
+            BackendConfig.INSTANCE.setEnableSyncToLatestEventOnDdls(prevFlag);
+            catalogHmsClient_.dropTable(TEST_DB_NAME, tblName, true, true);
+        }
+    }
+
+    @Test
+    public void testFullTableReload() throws Exception {
+        LOG.info("Executing testFullTableReload");
+        String tblName = "full_table_reload_test_"+ tableType_ + "_tbl";
+        try {
+            createDatabaseInCatalog(TEST_DB_NAME);
+            catalogHmsClient_.dropTable(TEST_DB_NAME, tblName, true, true);
+            // create a table in HMS and add it as incomplete table
+            // so that full table refresh can reload it
+            createTableInHms(TEST_DB_NAME, tblName, true);
+            IncompleteTable tbl =
+                IncompleteTable.createUninitializedTable(catalog_.getDb(TEST_DB_NAME),
+                    tblName);
+            tbl.setCreateEventId(getLatestEventIdFromHMS());
+            catalog_.addTable(catalog_.getDb(TEST_DB_NAME), tbl);
+            long prevLastSyncedEventId =
+                catalog_.getTable(TEST_DB_NAME, tblName).getLastSyncedEventId();
+            // add partitions but only in HMS so that
+            // request for full table reload syncs table to latest
+            // event id
+            List<List<String> > partVals = new ArrayList<>();
+            partVals.add(Arrays.asList("1"));
+            partVals.add(Arrays.asList("2"));
+            partVals.add(Arrays.asList("3"));
+            addPartitionsInHms(TEST_DB_NAME, tblName, partVals);
+            Table refreshedTbl = catalog_.getOrLoadTable(TEST_DB_NAME, tblName,
+                "testing table syncing to latest event id", null);
+
+            assertTrue(
+                refreshedTbl.getLastSyncedEventId() > refreshedTbl.getCreateEventId());
+            assertTrue(refreshedTbl.getLastSyncedEventId() > prevLastSyncedEventId);
+            assertTrue(refreshedTbl instanceof HdfsTable);
+            HdfsTable hdfsTable = (HdfsTable) refreshedTbl;
+            assertTrue(hdfsTable.getPartitions().size() == 3);
+
+        } finally {
+            catalogHmsClient_.dropTable(TEST_DB_NAME, tblName, true, true);
+        }
+    }
+
+    @Test
+    public void testTableEventsProcessedByEventProcessor() throws Exception {
+        // TODO: Move this to new file and add more tests
+        // that cover more events MetastoreEvents
+        LOG.info("Executing testEventsProcessedByEventProcessor");
+        String tblName = "test_table_events_processed_by_event_processor_" +
+            tableType_ + "_tbl";
+        ExternalEventsProcessor prevEventProcessor =
+            catalog_.getMetastoreEventProcessor();
+        try {
+            catalog_.setMetastoreEventProcessor(eventsProcessor_);
+            eventsProcessor_.processEvents();
+            createDatabaseInCatalog(TEST_DB_NAME);
+            catalogHmsClient_.dropTable(TEST_DB_NAME, tblName, true, true);
+            catalogHmsClient_
+                .createTable(MetastoreApiTestUtils.getTestTable(null,
+                    TEST_DB_NAME, tblName, null, true, tableType_));
+            HdfsTable tbl = getCatalogHdfsTable(TEST_DB_NAME, tblName);
+            assertTrue(tbl != null);
+            assertTrue("table's last synced id should not be -1",
+                tbl.getLastSyncedEventId() != -1);
+            assertTrue(tbl.getLastSyncedEventId() == tbl.getCreateEventId());
+            long prevSyncedEventId = tbl.getLastSyncedEventId();
+            eventsProcessor_.processEvents();
+            List<List<String> > partVals = new ArrayList<>();
+            partVals.add(Arrays.asList("1"));
+            partVals.add(Arrays.asList("2"));
+            partVals.add(Arrays.asList("3"));
+            addPartitionsInHms(TEST_DB_NAME, tblName, partVals);
+            // added partitions should not reflect in table
+            // stored in catalog cache
+            assertTrue(tbl.getPartitions().size() == 0);
+            eventsProcessor_.processEvents();
+            tbl = getCatalogHdfsTable(TEST_DB_NAME, tblName);
+            LOG.info("After add partititons, table last synced event id {}, latest "
+                    + "event id in HMS {}", tbl.getLastSyncedEventId(),
+                getLatestEventIdFromHMS());
+            assertTrue(tbl.getLastSyncedEventId() > prevSyncedEventId);
+            prevSyncedEventId = tbl.getLastSyncedEventId();
+
+            // alter partition 2 directly in HMS
+            Partition partitionToAlter =
+                getPartitionInHms(TEST_DB_NAME, tblName, Arrays.asList("2"));
+
+            String newLocation = "/path/to/newLocation/";
+            partitionToAlter.getSd().setLocation(newLocation);
+            alterPartitionInHms(TEST_DB_NAME, tblName, partitionToAlter);
+            eventsProcessor_.processEvents();
+            LOG.info("After alter partititons, table last synced event id {}, latest "
+                    + "event id in HMS {}", tbl.getLastSyncedEventId(),
+                getLatestEventIdFromHMS());
+            assertTrue(tbl.getLastSyncedEventId() > prevSyncedEventId);
+            prevSyncedEventId = tbl.getLastSyncedEventId();
+            dropPartitionInHms(TEST_DB_NAME, tblName, Arrays.asList("3"), true);
+            eventsProcessor_.processEvents();
+            LOG.info("After drop partitions, table last synced event id {}, latest "
+                    + "event id in HMS {}", tbl.getLastSyncedEventId(),
+                getLatestEventIdFromHMS());
+            assertTrue(tbl.getLastSyncedEventId() > prevSyncedEventId);
+
+            tbl = getCatalogHdfsTable(TEST_DB_NAME, tblName);
+            assertTrue("Table should have 2 partitions after dropping 1 "
+                + "out of 3 partitions", tbl.getPartitions().size() == 2);
+            // assert that  partition with new location from cached table
+            // exists
+            FeFsPartition modifiedPartition = null;
+            for (FeFsPartition part : FeCatalogUtils.loadAllPartitions(tbl)) {
+                if (part.getLocation().equals(newLocation)) {
+                    modifiedPartition = part;
+                    break;
+                }
+            }
+            assertTrue(modifiedPartition != null);
+        } finally {
+            catalog_.setMetastoreEventProcessor(prevEventProcessor);
+            eventsProcessor_.processEvents();
+            catalogHmsClient_.dropTable(TEST_DB_NAME, tblName, true, true);
+        }
+    }
+
+    @Test
+    public void testDbEventProcessedByEventProcessor() throws Exception {
+        LOG.info("Executing testDbEventProcessedByEventProcessor");
+        String dbName = "test_db_event_processed_by_event_processor_db";
+        ExternalEventsProcessor prevEventProcessor =
+            catalog_.getMetastoreEventProcessor();
+        try {
+            catalog_.setMetastoreEventProcessor(eventsProcessor_);
+            createDatabaseInCatalog(dbName);
+            long prevSyncedEventId = catalog_.getDb(dbName).getLastSyncedEventId();
+            try (MetaStoreClient msClient = catalog_.getMetaStoreClient()) {
+                MetastoreApiTestUtils.addDatabaseParametersInHms(msClient, dbName,
+                    "key1", "val1");
+            }
+            // assert that db's parameters are null
+            assertTrue(
+                catalog_.getDb(dbName).getMetaStoreDb().getParameters() == null);
+
+            eventsProcessor_.processEvents();
+            // after processing event, key1 should reflect in msDb parameters
+            assertTrue(catalog_.getDb(dbName).getMetaStoreDb().getParameters()
+                .get("key1").equals("val1"));
+            assertTrue(
+                catalog_.getDb(dbName).getLastSyncedEventId() > prevSyncedEventId);
+        } finally {
+            catalog_.setMetastoreEventProcessor(prevEventProcessor);
+            catalogHmsClient_.dropDatabase(dbName, true, true, true);
+        }
+    }
+
+    private void createDatabaseInHms(String catName, String dbName,
+        Map<String, String> params) throws TException {
+        try (MetaStoreClient msClient = catalog_.getMetaStoreClient()) {
+            MetastoreApiTestUtils.createDatabase(msClient,
+                null, TEST_DB_NAME, null);
+        }
+    }
+
+    private void createTableInHms(String dbName, String tblName, boolean isPartitioned)
+        throws TException {
+        try (MetaStoreClient msClient = catalog_.getMetaStoreClient()) {
+            MetastoreApiTestUtils.createTable(msClient, null,
+                dbName, tblName, null, isPartitioned);
+        }
+    }
+
+    private void addPartitionsInHms(String dbName, String tblName,
+        List<List<String>> partitionValues) throws TException {
+        try(MetaStoreClient msClient = catalog_.getMetaStoreClient()) {
+            MetastoreApiTestUtils.addPartitions(msClient, dbName,
+                tblName, partitionValues);
+        }
+    }
+
+    private long getLatestEventIdFromHMS() throws TException {
+        long lastEventId = -1;
+        try(MetaStoreClient client = catalog_.getMetaStoreClient()) {
+            lastEventId =
+                client.getHiveClient().getCurrentNotificationEventId().getEventId();
+        }
+        return lastEventId;
+    }
+
+    private org.apache.hadoop.hive.metastore.api.Table getHmsTable(String dbName,
+        String tblName) throws TException {
+        try (MetaStoreClient msClient = catalog_.getMetaStoreClient()) {
+            return msClient.getHiveClient().getTable(dbName, tblName);
+        }
+    }
+
+    private org.apache.impala.catalog.HdfsTable getCatalogHdfsTable(String dbName,
+        String tblName) throws CatalogException{
+        org.apache.impala.catalog.Table tbl =  catalog_.getTable(dbName, tblName);
+        if (tbl instanceof HdfsTable) {
+            return (HdfsTable) tbl;
+        } else {
+            return null;
+        }
+    }
+
+    private Partition getPartitionInHms(String dbName, String tblName,
+        List<String> partVal) throws TException {
+        try (MetaStoreClient msClient = catalog_.getMetaStoreClient()) {
+            return msClient.getHiveClient().getPartition(dbName,
+                tblName, partVal);
+        }
+    }
+
+    private void alterPartitionInHms(String dbName, String tblName, Partition partition)
+        throws TException {
+        try(MetaStoreClient msClient = catalog_.getMetaStoreClient()) {
+            msClient.getHiveClient().alter_partition(dbName, tblName, partition);
+        }
+    }
+
+    private void dropPartitionInHms(String dbName, String tblName, List<String> partVals,
+        boolean deleteData) throws TException {
+        try(MetaStoreClient msClient = catalog_.getMetaStoreClient()) {
+            msClient.getHiveClient().dropPartition(dbName, tblName, partVals, deleteData);
+        }
+    }
+
+    private Database getDatabaseInHms(String dbName) throws TException {
+        try (MetaStoreClient msClient = catalog_.getMetaStoreClient()) {
+            return  msClient.getHiveClient().getDatabase(dbName);
+        }
+    }
+
+    private Map<String, String> getPartitionSpec(
+        org.apache.hadoop.hive.metastore.api.Table tbl, List<String> vals) {
+        Map<String, String> partitionSpec = new HashMap<>();
+        for(int i = 0; i < tbl.getPartitionKeys().size(); i++) {
+            FieldSchema partCol = tbl.getPartitionKeys().get(i);
+            partitionSpec.put(partCol.getName(), vals.get(i));
+        }
+        return partitionSpec;
+    }
+
+    private void createDatabaseInCatalog(String dbName) throws TException {
+        Database msDb = MetastoreApiTestUtils
+            .createHmsDatabaseObject(null, dbName, null);
+        catalogHmsClient_.createDatabase(msDb);
+        assertTrue("db " + dbName + " not present in catalogd",
+            catalog_.getDb(dbName) != null);
+    }
+}
diff --git a/fe/src/test/java/org/apache/impala/testutil/CatalogServiceTestCatalog.java b/fe/src/test/java/org/apache/impala/testutil/CatalogServiceTestCatalog.java
index b8436ee..573e551 100644
--- a/fe/src/test/java/org/apache/impala/testutil/CatalogServiceTestCatalog.java
+++ b/fe/src/test/java/org/apache/impala/testutil/CatalogServiceTestCatalog.java
@@ -17,6 +17,7 @@
 
 package org.apache.impala.testutil;
 
+import com.google.common.base.Preconditions;
 import org.apache.impala.authorization.AuthorizationFactory;
 import org.apache.impala.authorization.NoopAuthorizationFactory;
 import org.apache.impala.authorization.AuthorizationPolicy;
@@ -24,11 +25,14 @@ import org.apache.impala.authorization.NoopAuthorizationFactory.NoopAuthorizatio
 import org.apache.impala.catalog.CatalogException;
 import org.apache.impala.catalog.CatalogServiceCatalog;
 import org.apache.impala.catalog.MetaStoreClientPool;
+import org.apache.impala.catalog.TableLoadingMgr;
+import org.apache.impala.catalog.events.MetastoreEvents.EventFactoryForSyncToLatestEvent;
 import org.apache.impala.catalog.metastore.CatalogMetastoreServer;
 import org.apache.impala.catalog.events.NoOpEventProcessor;
 import org.apache.impala.catalog.metastore.NoOpCatalogMetastoreServer;
 import org.apache.impala.compat.MetastoreShim;
 import org.apache.impala.common.ImpalaException;
+import org.apache.impala.service.CatalogOpExecutor;
 import org.apache.impala.service.FeSupport;
 import org.apache.impala.thrift.TUniqueId;
 
@@ -41,6 +45,7 @@ import java.util.UUID;
  * for testing.
  */
 public class CatalogServiceTestCatalog extends CatalogServiceCatalog {
+  private CatalogOpExecutor opExecutor_;
   private CatalogServiceTestCatalog(boolean loadInBackground, int numLoadingThreads,
       TUniqueId catalogServiceId, MetaStoreClientPool metaStoreClientPool)
       throws ImpalaException {
@@ -54,7 +59,7 @@ public class CatalogServiceTestCatalog extends CatalogServiceCatalog {
     rd.run();
   }
 
-  public static CatalogServiceCatalog create() {
+  public static CatalogServiceTestCatalog create() {
     return createWithAuth(new NoopAuthorizationFactory());
   }
 
@@ -62,18 +67,23 @@ public class CatalogServiceTestCatalog extends CatalogServiceCatalog {
    * Creates a catalog server that reads authorization policy metadata from the
    * authorization config.
    */
-  public static CatalogServiceCatalog createWithAuth(AuthorizationFactory authzFactory) {
+  public static CatalogServiceTestCatalog createWithAuth(AuthorizationFactory factory) {
     FeSupport.loadLibrary();
-    CatalogServiceCatalog cs;
+    CatalogServiceTestCatalog cs;
     try {
       if (MetastoreShim.getMajorVersion() > 2) {
         MetastoreShim.setHiveClientCapabilities();
       }
       cs = new CatalogServiceTestCatalog(false, 16, new TUniqueId(),
           new MetaStoreClientPool(0, 0));
-      cs.setAuthzManager(authzFactory.newAuthorizationManager(cs));
+      cs.setAuthzManager(factory.newAuthorizationManager(cs));
       cs.setMetastoreEventProcessor(NoOpEventProcessor.getInstance());
       cs.setCatalogMetastoreServer(NoOpCatalogMetastoreServer.INSTANCE);
+      cs.setCatalogOpExecutor(new CatalogOpExecutor(cs,
+          new NoopAuthorizationFactory().getAuthorizationConfig(),
+          new NoopAuthorizationFactory.NoopAuthorizationManager()));
+      cs.setEventFactoryForSyncToLatestEvent(new EventFactoryForSyncToLatestEvent(
+          cs.getCatalogOpExecutor()));
       cs.reset();
     } catch (ImpalaException e) {
       throw new IllegalStateException(e.getMessage(), e);
@@ -94,14 +104,28 @@ public class CatalogServiceTestCatalog extends CatalogServiceCatalog {
     if (MetastoreShim.getMajorVersion() > 2) {
       MetastoreShim.setHiveClientCapabilities();
     }
-    CatalogServiceCatalog cs = new CatalogServiceTestCatalog(false, 16,
+    CatalogServiceTestCatalog cs = new CatalogServiceTestCatalog(false, 16,
         new TUniqueId(), new EmbeddedMetastoreClientPool(0, derbyPath));
     cs.setAuthzManager(new NoopAuthorizationManager());
     cs.setMetastoreEventProcessor(NoOpEventProcessor.getInstance());
+    cs.setCatalogOpExecutor(new CatalogOpExecutor(cs,
+        new NoopAuthorizationFactory().getAuthorizationConfig(),
+        new NoopAuthorizationFactory.NoopAuthorizationManager()));
+    cs.setEventFactoryForSyncToLatestEvent(
+        new EventFactoryForSyncToLatestEvent(cs.getCatalogOpExecutor()));
     cs.reset();
     return cs;
   }
 
   @Override
   public AuthorizationPolicy getAuthPolicy() { return authPolicy_; }
+
+  private void setCatalogOpExecutor(CatalogOpExecutor opExecutor) {
+    opExecutor_ = opExecutor;
+  }
+
+  public CatalogOpExecutor getCatalogOpExecutor() {
+    Preconditions.checkNotNull(opExecutor_, "returning null opExecutor_");
+    return opExecutor_;
+  }
 }
diff --git a/tests/custom_cluster/test_metastore_service.py b/tests/custom_cluster/test_metastore_service.py
index 4569816..0611d63 100644
--- a/tests/custom_cluster/test_metastore_service.py
+++ b/tests/custom_cluster/test_metastore_service.py
@@ -122,7 +122,8 @@ class TestMetastoreService(CustomClusterTestSuite):
         impalad_args="--use_local_catalog=true",
         catalogd_args="--catalog_topic_mode=minimal "
                       "--start_hms_server=true "
-                      "--hms_port=5899"
+                      "--hms_port=5899 "
+                      "--enable_sync_to_latest_event_on_ddls=false"
     )
     def test_get_table_req_with_fallback(self):
       """
@@ -224,7 +225,8 @@ class TestMetastoreService(CustomClusterTestSuite):
                       "--start_hms_server=true "
                       "--hms_port=5899 "
                       "--fallback_to_hms_on_errors=false "
-                      "--hms_event_polling_interval_s=0"
+                      "--hms_event_polling_interval_s=0 "
+                      "--enable_sync_to_latest_event_on_ddls=false"
     )
     def test_get_table_req_without_fallback(self):
       """
@@ -409,7 +411,8 @@ class TestMetastoreService(CustomClusterTestSuite):
                       "--start_hms_server=true "
                       "--hms_port=5899 "
                       "--fallback_to_hms_on_errors=true "
-                      "--invalidate_hms_cache_on_ddls=true"
+                      "--invalidate_hms_cache_on_ddls=true "
+                      "--enable_sync_to_latest_event_on_ddls=false"
     )
     def test_cache_invalidated_on_nontransactional_table_ddls(self):
         db_name = ImpalaTestSuite.get_random_name(
@@ -425,7 +428,8 @@ class TestMetastoreService(CustomClusterTestSuite):
                       "--start_hms_server=true "
                       "--hms_port=5899 "
                       "--fallback_to_hms_on_errors=true "
-                      "--invalidate_hms_cache_on_ddls=false"
+                      "--invalidate_hms_cache_on_ddls=false "
+                      "--enable_sync_to_latest_event_on_ddls=false"
     )
     def test_cache_valid_on_nontransactional_table_ddls(self):
         db_name = ImpalaTestSuite.get_random_name(
@@ -663,6 +667,7 @@ class TestMetastoreService(CustomClusterTestSuite):
                       "--hms_port=5899 "
                       "--fallback_to_hms_on_errors=true "
                       "--invalidate_hms_cache_on_ddls=true "
+                      "--enable_sync_to_latest_event_on_ddls=false "
                       "--hms_event_polling_interval_s=1"
     )
     def test_cache_invalidate_incomplete_table(self):
@@ -770,6 +775,7 @@ class TestMetastoreService(CustomClusterTestSuite):
                       "--hms_port=5899 "
                       "--fallback_to_hms_on_errors=true "
                       "--invalidate_hms_cache_on_ddls=true "
+                      "--enable_sync_to_latest_event_on_ddls=false "
                       "--hms_event_polling_interval_s=5"
     )
     def test_table_create_drop_seq(self):
@@ -886,6 +892,7 @@ class TestMetastoreService(CustomClusterTestSuite):
                       "--hms_port=5899 "
                       "--fallback_to_hms_on_errors=true "
                       "--invalidate_hms_cache_on_ddls=true "
+                      "--enable_sync_to_latest_event_on_ddls=false "
                       "--hms_event_polling_interval_s=5"
     )
     def test_database_create_drop_seq(self):