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

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


The following commit(s) were added to refs/heads/master by this push:
     new 381d8bfd6a HIVE-26265: Option to filter out Txn events during 
replication. (Francis Pang reviewed by Peter Vary) (#3365)
381d8bfd6a is described below

commit 381d8bfd6a2a4f3715eca1cce1b689d3237c4142
Author: cmunkey <49877369+cmun...@users.noreply.github.com>
AuthorDate: Tue Jun 28 00:48:22 2022 -0700

    HIVE-26265: Option to filter out Txn events during replication. (Francis 
Pang reviewed by Peter Vary) (#3365)
    
    Purpose: Currently, all Txn events OpenTxn, CommitTxn, and RollbackTxn are 
included in the REPL DUMP, even when the transcation does not involve the 
database being dumped
    (replicated). These events are unnecessary and result is excessive space 
required for the dump, as well as increasing work that results from these 
events being replayed
    during REPL LOAD.
    
    Solution proposed: To reduce this unnecessary space and work, added the 
hive.repl.filter.transactions configuration property. When set to "true", extra 
Txn events
    will be filtered out as follows: CommitTxn and RollbackTxn are included in 
the REPL DUMP only if the transaction referenced had a corresponding 
ALLOCATE_WRITE_ID event
    that was dumped. OpenTxn is never dumped, and the OpenTxn event will be 
implcitly Opened when REPL LOAD processes the ALLOC_WRITE_ID event, since the 
ALLOC_WRITE_ID
    contains the open transaction ids. The default setting is "false".
    
    Co-authored-by: Francis Pang <fp...@cloudera.com>
---
 .../java/org/apache/hadoop/hive/conf/HiveConf.java |   4 +
 .../hcatalog/listener/DbNotificationListener.java  |   2 +-
 .../hadoop/hive/ql/parse/ReplicationTestUtils.java |  50 +-
 .../parse/TestReplicationFilterTransactions.java   | 514 +++++++++++++++++++++
 .../apache/hadoop/hive/ql/exec/ReplTxnTask.java    |  15 +
 .../hadoop/hive/ql/exec/repl/util/ReplUtils.java   |   5 +
 .../ql/parse/repl/dump/events/AbortTxnHandler.java |  18 +
 .../parse/repl/dump/events/CommitTxnHandler.java   |   7 +
 .../ql/parse/repl/dump/events/OpenTxnHandler.java  |   5 +
 .../apache/hadoop/hive/metastore/HMSHandler.java   |   4 +
 .../hive/metastore/events/AbortTxnEvent.java       |  23 +-
 .../hive/metastore/messaging/AbortTxnMessage.java  |   3 +
 .../hive/metastore/messaging/MessageBuilder.java   |   4 +-
 .../messaging/json/JSONAbortTxnMessage.java        |  13 +-
 .../hadoop/hive/metastore/txn/TxnHandler.java      |  40 +-
 .../events/TestAbortTxnEventDbsUpdated.java        |  68 +++
 16 files changed, 764 insertions(+), 11 deletions(-)

diff --git a/common/src/java/org/apache/hadoop/hive/conf/HiveConf.java 
b/common/src/java/org/apache/hadoop/hive/conf/HiveConf.java
index 2a68674e59..67cfef75a3 100644
--- a/common/src/java/org/apache/hadoop/hive/conf/HiveConf.java
+++ b/common/src/java/org/apache/hadoop/hive/conf/HiveConf.java
@@ -3652,6 +3652,10 @@ public class HiveConf extends Configuration {
         "org.apache.hive.hcatalog.api.repl.exim.EximReplicationTaskFactory",
         "Parameter that can be used to override which ReplicationTaskFactory 
will be\n" +
         "used to instantiate ReplicationTask events. Override for third party 
repl plugins"),
+    REPL_FILTER_TRANSACTIONS("hive.repl.filter.transactions", false,
+            "Enable transaction event filtering to save dump space.\n" +
+                    "When true, transactions are implicitly opened during REPL 
DUMP.\n" +
+                    "The default setting is false"),
     
HIVE_MAPPER_CANNOT_SPAN_MULTIPLE_PARTITIONS("hive.mapper.cannot.span.multiple.partitions",
 false, ""),
     HIVE_REWORK_MAPREDWORK("hive.rework.mapredwork", false,
         "should rework the mapred work or not.\n" +
diff --git 
a/hcatalog/server-extensions/src/main/java/org/apache/hive/hcatalog/listener/DbNotificationListener.java
 
b/hcatalog/server-extensions/src/main/java/org/apache/hive/hcatalog/listener/DbNotificationListener.java
index 85a94a7fc4..d66add1591 100644
--- 
a/hcatalog/server-extensions/src/main/java/org/apache/hive/hcatalog/listener/DbNotificationListener.java
+++ 
b/hcatalog/server-extensions/src/main/java/org/apache/hive/hcatalog/listener/DbNotificationListener.java
@@ -666,7 +666,7 @@ public class DbNotificationListener extends 
TransactionalMetaStoreEventListener
       return;
     }
     AbortTxnMessage msg =
-        
MessageBuilder.getInstance().buildAbortTxnMessage(abortTxnEvent.getTxnId());
+        
MessageBuilder.getInstance().buildAbortTxnMessage(abortTxnEvent.getTxnId(), 
abortTxnEvent.getDbsUpdated());
     NotificationEvent event =
         new NotificationEvent(0, now(), EventType.ABORT_TXN.toString(),
             msgEncoder.getSerializer().serialize(msg));
diff --git 
a/itests/hive-unit/src/test/java/org/apache/hadoop/hive/ql/parse/ReplicationTestUtils.java
 
b/itests/hive-unit/src/test/java/org/apache/hadoop/hive/ql/parse/ReplicationTestUtils.java
index 0859218573..d9040299b1 100644
--- 
a/itests/hive-unit/src/test/java/org/apache/hadoop/hive/ql/parse/ReplicationTestUtils.java
+++ 
b/itests/hive-unit/src/test/java/org/apache/hadoop/hive/ql/parse/ReplicationTestUtils.java
@@ -17,21 +17,26 @@
  */
 package org.apache.hadoop.hive.ql.parse;
 
+import org.apache.hadoop.fs.FSDataInputStream;
+import org.apache.hadoop.fs.FileSystem;
+import org.apache.hadoop.fs.LocatedFileStatus;
 import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.fs.RemoteIterator;
 import org.apache.hadoop.hdfs.DistributedFileSystem;
 import org.apache.hadoop.hive.conf.HiveConf;
 import org.apache.hadoop.hive.ql.exec.repl.DirCopyWork;
-import org.apache.hadoop.hive.ql.parse.repl.PathBuilder;
 import org.apache.hadoop.hive.ql.exec.repl.util.ReplUtils;
 import org.junit.Assert;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
 
 import java.io.BufferedReader;
+import java.io.File;
 import java.io.IOException;
 import java.io.InputStream;
 import java.io.InputStreamReader;
 import java.util.ArrayList;
 import java.util.List;
-import java.util.Arrays;
 import java.util.Set;
 import java.util.HashSet;
 
@@ -39,6 +44,7 @@ import java.util.HashSet;
  * ReplicationTestUtils - static helper functions for replication test
  */
 public class ReplicationTestUtils {
+  private static final Logger LOG = 
LoggerFactory.getLogger(ReplicationTestUtils.class);
 
   public enum OperationType {
     REPL_TEST_ACID_INSERT, REPL_TEST_ACID_INSERT_SELECT, REPL_TEST_ACID_CTAS,
@@ -555,4 +561,44 @@ public class ReplicationTestUtils {
     Assert.assertTrue(tableNames.containsAll(expected));
     reader.close();
   }
+
+  public static void findTxnsFromDump(WarehouseInstance.Tuple tuple, HiveConf 
conf,
+                                      List<Path> openTxns, List<Path> 
commitTxns, List<Path> abortTxns) throws IOException {
+    Path dumpRoot = new Path(tuple.dumpLocation);
+    FileSystem fs = FileSystem.get(dumpRoot.toUri(), conf);
+
+    LOG.info("Scanning for event files: " + dumpRoot.toString());
+    RemoteIterator<LocatedFileStatus> files = fs.listFiles(dumpRoot, true);
+    while(files.hasNext()) {
+      LocatedFileStatus status = files.next();
+
+      if (!status.getPath().getName().equals("_dumpmetadata")) {
+        continue;
+      }
+
+      String event = getEvent(fs, status.getPath());
+      if (event.equals("EVENT_OPEN_TXN")) {
+        openTxns.add(status.getPath());
+      } else if (event.equals("EVENT_COMMIT_TXN")) {
+        commitTxns.add(status.getPath());
+      } else if (event.equals("EVENT_ABORT_TXN")) {
+        abortTxns.add(status.getPath());
+      }
+    }
+  }
+
+  private static String getEvent(FileSystem fs, Path path) throws IOException {
+    try (FSDataInputStream fdis = fs.open(path);
+         BufferedReader br = new BufferedReader(new InputStreamReader(fdis))) {
+      // Assumes event is at least on first line.
+      String line = br.readLine();
+      Assert.assertNotNull(line);
+      // Assumes event is present.
+      int index = line.indexOf("\t");
+      Assert.assertNotEquals(-1, index);
+      String event = line.substring(0, index);
+      LOG.info("Reading event file: " + path.toString() + " : " + event + ", 
raw: " + line);
+      return event;
+    }
+  }
 }
diff --git 
a/itests/hive-unit/src/test/java/org/apache/hadoop/hive/ql/parse/TestReplicationFilterTransactions.java
 
b/itests/hive-unit/src/test/java/org/apache/hadoop/hive/ql/parse/TestReplicationFilterTransactions.java
new file mode 100644
index 0000000000..bb64f17c0b
--- /dev/null
+++ 
b/itests/hive-unit/src/test/java/org/apache/hadoop/hive/ql/parse/TestReplicationFilterTransactions.java
@@ -0,0 +1,514 @@
+/*
+ * 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.hadoop.hive.ql.parse;
+
+import org.apache.commons.io.FileUtils;
+import java.sql.Connection;
+import java.sql.ResultSet;
+import java.sql.Statement;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.FileSystem;
+import org.apache.hadoop.fs.LocatedFileStatus;
+import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.fs.RemoteIterator;
+import org.apache.hadoop.fs.permission.AclStatus;
+import org.apache.hadoop.fs.permission.FsPermission;
+import org.apache.hadoop.hdfs.DistributedFileSystem;
+import org.apache.hadoop.hdfs.MiniDFSCluster;
+import org.apache.hadoop.hive.conf.HiveConf;
+import org.apache.hadoop.hive.metastore.MetaStoreEventListener;
+import org.apache.hadoop.hive.metastore.api.MetaException;
+import org.apache.hadoop.hive.metastore.conf.MetastoreConf;
+import org.apache.hadoop.hive.metastore.events.*;
+import 
org.apache.hadoop.hive.metastore.messaging.json.gzip.GzipJSONMessageEncoder;
+import org.apache.hadoop.hive.metastore.tools.SQLGenerator;
+import org.apache.hadoop.hive.metastore.utils.TestTxnDbUtil;
+import org.apache.hadoop.hive.ql.parse.repl.PathBuilder;
+import org.apache.hadoop.hive.ql.processors.CommandProcessorException;
+import org.apache.hadoop.hive.shims.Utils;
+import org.junit.After;
+import org.junit.Assert;
+import org.junit.Before;
+import org.junit.Rule;
+import org.junit.Test;
+import org.junit.rules.TemporaryFolder;
+import org.junit.rules.TestName;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.io.File;
+import java.io.IOException;
+import java.nio.charset.Charset;
+import java.nio.charset.StandardCharsets;
+import java.sql.Connection;
+import java.util.ArrayList;
+import java.util.Collections;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+import java.util.concurrent.atomic.AtomicInteger;
+import java.util.concurrent.ConcurrentHashMap;
+
+import static 
org.apache.hadoop.hive.common.repl.ReplConst.SOURCE_OF_REPLICATION;
+
+/**
+ * TestTxnReplicationOptimizations - Test transaction replication.
+ */
+public class TestReplicationFilterTransactions {
+  static final private Logger LOG = 
LoggerFactory.getLogger(TestReplicationFilterTransactions.class);
+
+  private final static String tid =
+          
TestReplicationFilterTransactions.class.getCanonicalName().toLowerCase().replace('.','_')
 + "_" + System.currentTimeMillis();
+  private final static String TEST_PATH =
+          System.getProperty("test.warehouse.dir", "/tmp") + Path.SEPARATOR + 
tid;
+
+  @Rule
+  public TemporaryFolder tempFolder= new TemporaryFolder();
+
+  // Event listener for the primary, mainly counts txn events.
+  // Count totals are saved to static fields so they can be accessed
+  // after dump/load.
+  static public class PrimaryEventListenerTestImpl extends 
MetaStoreEventListener {
+    public PrimaryEventListenerTestImpl(Configuration conf) {
+      super(conf);
+    }
+
+    private static final AtomicInteger countOpenTxn = new AtomicInteger(0);
+    private static final AtomicInteger countCommitTxn = new AtomicInteger (0);
+    private static final AtomicInteger countAbortTxn = new AtomicInteger (0);
+
+    @Override
+    public void onOpenTxn(OpenTxnEvent openTxnEvent, Connection dbConn, 
SQLGenerator sqlGenerator) throws MetaException {
+      super.onOpenTxn(openTxnEvent, dbConn, sqlGenerator);
+      countOpenTxn.getAndIncrement();
+    }
+
+    @Override
+    public void onCommitTxn(CommitTxnEvent commitTxnEvent, Connection dbConn, 
SQLGenerator sqlGenerator) throws MetaException {
+      super.onCommitTxn(commitTxnEvent, dbConn, sqlGenerator);
+      countCommitTxn.getAndIncrement();
+    }
+
+    @Override
+    public void onAbortTxn(AbortTxnEvent abortTxnEvent, Connection dbConn, 
SQLGenerator sqlGenerator) throws MetaException {
+      super.onAbortTxn(abortTxnEvent, dbConn, sqlGenerator);
+      countAbortTxn.getAndIncrement();
+    }
+
+    public static int getCountOpenTxn() {
+      return countOpenTxn.get();
+    }
+
+    public static int getCountCommitTxn() {
+      return countCommitTxn.get();
+    }
+
+    public static int getCountAbortTxn() {
+      return countAbortTxn.get();
+    }
+
+    public static void reset() {
+      countOpenTxn.set(0);
+      countCommitTxn.set(0);
+      countAbortTxn.set(0);
+    }
+  }
+
+  // Event listener for the replica, mainly counts txn events.
+  // Count totals are saved to static fields so they can be accessed
+  // after dump/load.
+  static public class ReplicaEventListenerTestImpl extends 
MetaStoreEventListener {
+    public ReplicaEventListenerTestImpl(Configuration conf) {
+      super(conf);
+    }
+
+    private static final AtomicInteger countOpenTxn = new AtomicInteger(0);
+    private static final AtomicInteger countCommitTxn = new AtomicInteger (0);
+    private static final AtomicInteger countAbortTxn = new AtomicInteger (0);
+
+    private static final Map<Long, Long> txnMapping = new 
ConcurrentHashMap<Long, Long>();
+    
+    @Override
+    public void onOpenTxn(OpenTxnEvent openTxnEvent, Connection dbConn, 
SQLGenerator sqlGenerator) throws MetaException {
+      super.onOpenTxn(openTxnEvent, dbConn, sqlGenerator);
+      countOpenTxn.getAndIncrement();
+
+      // Following code reads REPL_TXN_MAP, so we can check later test that 
primary to replica TxnId
+      // mapping was done.
+      try {
+        TestReplicationFilterTransactions.updateTxnMapping(txnMapping);
+      } catch (Exception e) {
+        throw new RuntimeException(e);
+      }
+    }
+
+    @Override
+    public void onCommitTxn(CommitTxnEvent commitTxnEvent, Connection dbConn, 
SQLGenerator sqlGenerator) throws MetaException {
+      super.onCommitTxn(commitTxnEvent, dbConn, sqlGenerator);
+      countCommitTxn.getAndIncrement();
+    }
+
+    @Override
+    public void onAbortTxn(AbortTxnEvent abortTxnEvent, Connection dbConn, 
SQLGenerator sqlGenerator) throws MetaException {
+      super.onAbortTxn(abortTxnEvent, dbConn, sqlGenerator);
+      countAbortTxn.getAndIncrement();
+    }
+
+    public static int getCountOpenTxn() {
+      return countOpenTxn.get();
+    }
+
+    public static int getCountCommitTxn() {
+      return countCommitTxn.get();
+    }
+
+    public static int getCountAbortTxn() {
+      return countAbortTxn.get();
+    }
+
+    public static Map<Long, Long> getTxnMapping() { return new 
HashMap(txnMapping); }
+    
+    public static void reset() {
+      countOpenTxn.set(0);
+      countCommitTxn.set(0);
+      countAbortTxn.set(0);
+      txnMapping.clear();
+    }
+  }
+
+  static class EventCount {
+    int countOpenTxn;
+    int countCommitTxn;
+    int countAbortTxn;
+
+    public EventCount(int countOpenTxn, int countCommitTxn, int countAbortTxn) 
{
+      this.countOpenTxn = countOpenTxn;
+      this.countCommitTxn = countCommitTxn;
+      this.countAbortTxn = countAbortTxn;
+    }
+
+    public int getCountOpenTxn() {
+      return countOpenTxn;
+    }
+
+    public int getCountCommitTxn() {
+      return countCommitTxn;
+    }
+
+    public int getCountAbortTxn() {
+      return countAbortTxn;
+    }
+  }
+
+  @Rule
+  public final TestName testName = new TestName();
+
+  static WarehouseInstance primary;
+  static WarehouseInstance replica;
+
+  static HiveConf dfsConf;
+
+  String primaryDbName, replicatedDbName, otherDbName;
+
+  EventCount expected;
+  int txnOffset;
+
+
+  private Map<String, String> setupConf(String dfsUri, String 
listenerClassName) {
+    Map<String, String> confMap = new HashMap<String, String>();
+    confMap.put("fs.defaultFS", dfsUri);
+    confMap.put("hive.support.concurrency", "true");
+    confMap.put("hive.txn.manager", 
"org.apache.hadoop.hive.ql.lockmgr.DbTxnManager");
+    confMap.put("hive.metastore.client.capability.check", "false");
+    confMap.put("hive.repl.bootstrap.dump.open.txn.timeout", "1s");
+    confMap.put("hive.strict.checks.bucketing", "false");
+    confMap.put("hive.mapred.mode", "nonstrict");
+    confMap.put("mapred.input.dir.recursive", "true");
+    confMap.put("hive.metastore.disallow.incompatible.col.type.changes", 
"false");
+    confMap.put("hive.stats.autogather", "false");
+    confMap.put("hive.in.repl.test", "true");
+
+    // Primary and replica have different listeners so that we know exactly 
what
+    // happened on primary and replica respectively.
+    confMap.put(MetastoreConf.ConfVars.EVENT_LISTENERS.getVarname(), 
listenerClassName);
+    confMap.put(MetastoreConf.ConfVars.EVENT_MESSAGE_FACTORY.getHiveName(),
+            GzipJSONMessageEncoder.class.getCanonicalName());
+    return confMap;
+  }
+
+  @Before
+  public void setup() throws Throwable {
+    TestReplicationFilterTransactions.dfsConf = new 
HiveConf(TestReplicationFilterTransactions.class);
+    
TestReplicationFilterTransactions.dfsConf.set("dfs.client.use.datanode.hostname",
 "true");
+    TestReplicationFilterTransactions.dfsConf.set("hadoop.proxyuser." + 
Utils.getUGI().getShortUserName() + ".hosts", "*");
+    TestReplicationFilterTransactions.dfsConf.set("dfs.namenode.acls.enabled", 
"true");
+
+    MiniDFSCluster miniDFSCluster =
+            new 
MiniDFSCluster.Builder(TestReplicationFilterTransactions.dfsConf).numDataNodes(2).format(true).build();
+
+    Map<String, String> conf = 
setupConf(miniDFSCluster.getFileSystem().getUri().toString(),
+            PrimaryEventListenerTestImpl.class.getName());
+    primary = new WarehouseInstance(LOG, miniDFSCluster, conf);
+
+    conf = setupConf(miniDFSCluster.getFileSystem().getUri().toString(),
+            ReplicaEventListenerTestImpl.class.getName());
+    conf.put(MetastoreConf.ConfVars.REPLDIR.getHiveName(), primary.repldDir);
+
+    replica = new WarehouseInstance(LOG, miniDFSCluster, conf);
+
+    primaryDbName = testName.getMethodName() + "_" + 
System.currentTimeMillis();
+    replicatedDbName = "replicated_" + primaryDbName;
+    primary.run("create database " + primaryDbName + " WITH DBPROPERTIES ( '" +
+            SOURCE_OF_REPLICATION + "' = '1,2,3')");
+    otherDbName = testName.getMethodName() + "_other_" + 
System.currentTimeMillis();
+    primary.run("create database " + otherDbName);
+
+    PrimaryEventListenerTestImpl.reset();
+    ReplicaEventListenerTestImpl.reset();
+
+    // Each test always has 8 openTxns, 6 commitTxn, and 2 abortTxns.
+    // Note that this is the number that was done on the primary,
+    // and some are done on non-replicated database.
+    expected = new EventCount(8, 6, 2);
+  }
+
+  static void updateTxnMapping(Map<Long, Long> map) throws Exception {
+    // Poll REPL_TXN_MAP and add to map.
+    // Do dirty read.
+    try (Connection conn = TestTxnDbUtil.getConnection(replica.hiveConf)) {
+      conn.setTransactionIsolation(Connection.TRANSACTION_READ_UNCOMMITTED);
+      try (Statement stmt = conn.createStatement()) {
+        try (ResultSet rs = stmt.executeQuery("SELECT \"RTM_TARGET_TXN_ID\", 
\"RTM_SRC_TXN_ID\" FROM \"REPL_TXN_MAP\"")) {
+          while(rs.next()) {
+            Long srcTxnId = rs.getLong(1);
+            Long tgtTxnId = rs.getLong(2);
+            map.put(srcTxnId, tgtTxnId);
+          }
+        }
+      }
+    }
+  }
+
+  @After
+  public void tearDown() throws Throwable {
+    primary.run("drop database if exists " + primaryDbName + " cascade");
+    primary.run("drop database if exists " + otherDbName + " cascade");
+    replica.run("drop database if exists " + replicatedDbName + " cascade");
+    primary.close();
+    replica.close();
+  }
+
+  private void prepareBootstrapData() throws Throwable {
+
+    // Burn some txnids on the source. This will ensure that the primary 
TxnIds will be
+    // greater than the replica TxnIds to make sure that TxnId mapping is 
occurring.
+    // This will cause the primary transaction numbering to be 10 greater than 
the
+    // replica transaction numbering.
+    primary.run("use " + primaryDbName)
+            .run("create table t999 (id int) clustered by(id) into 3 buckets 
stored as orc " +
+                    "tblproperties (\"transactional\"=\"true\")")
+            .run("insert into t999 values (99901)")
+            .run("insert into t999 values (99902)")
+            .run("insert into t999 values (99903)")
+            .run("insert into t999 values (99904)")
+            .run("insert into t999 values (99905)")
+            .run("insert into t999 values (99906)")
+            .run("insert into t999 values (99907)")
+            .run("insert into t999 values (99908)")
+            .run("insert into t999 values (99909)")
+            .run("insert into t999 values (99910)")
+            .run("drop table t999");
+    txnOffset = 10;
+
+    // primaryDbName is replicated, t2 and t2 are ACID tables with initial 
data.
+    // t3 is an ACID table with 2 initial rows, later t3 will be locked to 
force aborted transaction.
+    primary.run("use " + primaryDbName)
+            .run("create table t1 (id int) clustered by(id) into 3 buckets 
stored as orc " +
+                  "tblproperties (\"transactional\"=\"true\")")
+            .run("insert into t1 values(1)")
+            .run("create table t2 (place string) partitioned by (country 
string) clustered by(place) " +
+                    "into 3 buckets stored as orc tblproperties 
(\"transactional\"=\"true\")")
+            .run("insert into t2 partition(country='india') values 
('bangalore')")
+            .run("create table t3 (id int) stored as orc tblproperties 
(\"transactional\"=\"true\")")
+            .run("insert into t3 values(111), (222)");
+
+    // otherDbName is not replicated, but contains ACID tables.
+    primary.run("use " + otherDbName)
+            .run("create table t1 (id int) clustered by(id) into 3 buckets 
stored as orc " +
+                    "tblproperties (\"transactional\"=\"true\")")
+            .run("insert into t1 values(100)")
+            .run("create table t2 (place string) partitioned by (country 
string) clustered by(place) " +
+                    "into 3 buckets stored as orc tblproperties 
(\"transactional\"=\"true\")")
+            .run("insert into t2 partition(country='usa') values ('san 
francisco')")
+            .run("create table t3 (id int) stored as orc tblproperties 
(\"transactional\"=\"true\")")
+            .run("insert into t3 values(1110), (2220)");
+  }
+
+  // Intentionally corrupt or uncorrupt a file of the table to cause an 
AbortTxn.
+  private void alterBucketFile(Path warehouseRoot, String dbName, String 
tableName, boolean toCorrupted) throws IOException {
+    DistributedFileSystem fs = primary.miniDFSCluster.getFileSystem();
+
+    String bucket = "bucket_00000_0";
+
+    PathBuilder pb = new PathBuilder(warehouseRoot.toString())
+            .addDescendant(dbName.toLowerCase() + ".db")
+            .addDescendant(tableName)
+            .addDescendant("delta_0000001_0000001_0000")
+            .addDescendant(bucket);
+    Path location = pb.build();
+    File junkFile = new File(tempFolder.getRoot(), "junk");
+    File saveFolder = new File(tempFolder.getRoot(), dbName + "_" + tableName);
+
+    if (toCorrupted) {
+      if (!junkFile.exists()) {
+        File junk = tempFolder.newFile("junk");
+        FileUtils.writeStringToFile(junk, "junk", StandardCharsets.UTF_8);
+      }
+      Path dest = new Path(saveFolder.getAbsolutePath(), bucket);
+      fs.copyToLocalFile(true, location, dest);
+      fs.copyFromLocalFile(false, true, new Path(junkFile.getAbsolutePath()), 
location);
+    } else {
+      Path src = new Path(saveFolder.getAbsolutePath(), bucket);
+      fs.copyFromLocalFile(true, true, src, location);
+    }
+  }
+
+  private void prepareAbortTxn(String dbName, int value) throws Throwable {
+    // Forces an abortTxn even to be generated in the database.
+    // The abortTxn needs to be generated during the execution phase of the 
plan,
+    // to do so, the bucket file of the table is intentionally mangled to
+    // induce an error and abortTxn during the execution phase.
+
+    alterBucketFile(primary.warehouseRoot, dbName, "t3", true);
+    try {
+      primary.run("use " + dbName)
+             .run("update t3 set id = 999 where id = " + 
String.valueOf(value));
+      Assert.fail("Update should have failed");
+    } catch (Throwable t) {
+      Assert.assertTrue(t.getCause().getCause() instanceof 
org.apache.orc.FileFormatException);
+    }
+    alterBucketFile(primary.warehouseRoot, dbName, "t3", false);
+  }
+
+  private void prepareIncrementalData() throws Throwable {
+    primary.run("use " + primaryDbName)
+            .run("insert into t1 values (2), (3)")
+            .run("insert into t2 partition(country='india') values 
('chennai')")
+            .run("insert into t2 partition(country='india') values ('pune')");
+    prepareAbortTxn(primaryDbName, 222);
+    primary.run("use " + otherDbName)
+            .run("insert into t1 values (200), (300)")
+            .run("insert into t2 partition(country='usa') values ('santa 
clara')")
+            .run("insert into t2 partition(country='usa') values ('palo 
alto')");
+    prepareAbortTxn(otherDbName, 2220);
+  }
+
+  private List<String> withTxnOptimized(boolean optimizationOn) {
+    return Collections.singletonList(String.format("'%s'='%s'", 
HiveConf.ConfVars.REPL_FILTER_TRANSACTIONS.toString(),
+            String.valueOf(optimizationOn)));
+  }
+
+  @Test
+  public void testTxnEventsUnoptimized() throws Throwable {
+    prepareBootstrapData();
+    WarehouseInstance.Tuple tuple = primary.run("use " + primaryDbName)
+            .dump(primaryDbName, withTxnOptimized(false));
+    replica.load(replicatedDbName, primaryDbName, withTxnOptimized(false));
+    assertBootstrap(tuple);
+
+    PrimaryEventListenerTestImpl.reset();
+    ReplicaEventListenerTestImpl.reset();
+    
+    prepareIncrementalData();
+    tuple = primary.run("use " + primaryDbName)
+            .dump(primaryDbName, withTxnOptimized(false));
+    replica.load(replicatedDbName, primaryDbName, withTxnOptimized(false));
+
+    EventCount filtered = new EventCount(0, 0, 0);
+    assertTxnOptimization(false, tuple, filtered);
+  }
+
+  @Test
+  public void testTxnEventsOptimized() throws Throwable {
+    prepareBootstrapData();
+    WarehouseInstance.Tuple tuple = primary.run("use " + primaryDbName)
+            .dump(primaryDbName, withTxnOptimized(false));
+    replica.load(replicatedDbName, primaryDbName, withTxnOptimized(false));
+    assertBootstrap(tuple);
+    PrimaryEventListenerTestImpl.reset();
+    ReplicaEventListenerTestImpl.reset();
+    
+    prepareIncrementalData();
+    tuple = primary.run("use " + primaryDbName)
+            .dump(primaryDbName, withTxnOptimized(true));
+    replica.load(replicatedDbName, primaryDbName, withTxnOptimized(true));
+
+    EventCount filtered = new EventCount(4, 3, 1);
+    assertTxnOptimization(true, tuple, filtered);
+  }
+
+  private void assertBootstrap(WarehouseInstance.Tuple tuple) throws 
IOException {
+    List<Path> openTxns = new ArrayList<Path>();
+    List<Path> commitTxns = new ArrayList<Path>();
+    List<Path> abortTxns = new ArrayList<Path>();
+
+    ReplicationTestUtils.findTxnsFromDump(tuple, primary.hiveConf, openTxns, 
commitTxns, abortTxns);
+
+    Assert.assertEquals(openTxns.size(), 0);
+    Assert.assertEquals(commitTxns.size(), 0);
+    Assert.assertEquals(abortTxns.size(), 0);
+  }
+
+  private void assertTxnOptimization(boolean optimizationOn, 
WarehouseInstance.Tuple tuple, EventCount filtered) throws Exception {
+
+    List<Path> openTxns = new ArrayList<Path>();
+    List<Path> commitTxns = new ArrayList<Path>();
+    List<Path> abortTxns = new ArrayList<Path>();
+
+    // Find all Txn event files in the dump directory.
+    ReplicationTestUtils.findTxnsFromDump(tuple, primary.hiveConf, openTxns, 
commitTxns, abortTxns);
+
+    // Assert the number of Txn events that occurred on the primary
+    Assert.assertEquals(expected.getCountOpenTxn(), 
PrimaryEventListenerTestImpl.getCountOpenTxn());
+    Assert.assertEquals(expected.getCountCommitTxn(), 
PrimaryEventListenerTestImpl.getCountCommitTxn());
+    Assert.assertEquals(expected.getCountAbortTxn(), 
PrimaryEventListenerTestImpl.getCountAbortTxn());
+
+    // Assert the number of Txn events that occurred on the replica.
+    // When optimization is on, filtered has the number of Txn events that are 
expected to have been filtered.
+    // When optimization is off, filtered should be all all 0s.
+    Assert.assertEquals(expected.getCountOpenTxn() - 
filtered.getCountOpenTxn(), ReplicaEventListenerTestImpl.getCountOpenTxn());
+    Assert.assertEquals(expected.getCountCommitTxn() - 
filtered.getCountCommitTxn(), ReplicaEventListenerTestImpl.getCountCommitTxn());
+    Assert.assertEquals(expected.getCountAbortTxn() - 
filtered.getCountAbortTxn(), ReplicaEventListenerTestImpl.getCountAbortTxn());
+
+    // Assert the number of Txn event files found.
+    // When optimization is on, filtered has the number of Txn events that are 
expected to have been filtered.
+    // When optimization is off, filtered should be all all 0s.
+    // Note that when optimization is on, there should never be optnTxn events.
+    Assert.assertEquals(optimizationOn ? 0 : expected.getCountOpenTxn(), 
openTxns.size());
+    Assert.assertEquals(expected.getCountCommitTxn() - 
filtered.getCountCommitTxn(), commitTxns.size());
+    Assert.assertEquals(expected.getCountAbortTxn() - 
filtered.getCountAbortTxn(), abortTxns.size());
+
+    // Check replica TxnMapping.
+    // Since primary TxnMappings had 10 txnIds burnt, there should be a 
mapping on the replica.
+    Map<Long, Long> replicaTxnMapping = 
ReplicaEventListenerTestImpl.getTxnMapping();
+    Assert.assertEquals(ReplicaEventListenerTestImpl.getCountOpenTxn(), 
replicaTxnMapping.size());
+    for (Map.Entry<Long, Long> mapping : replicaTxnMapping.entrySet()) {
+      Assert.assertEquals(mapping.getKey().longValue()  - txnOffset, 
mapping.getValue().longValue());
+    }
+  }
+}
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/exec/ReplTxnTask.java 
b/ql/src/java/org/apache/hadoop/hive/ql/exec/ReplTxnTask.java
index d1e42c1cb7..fd12731cbe 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/exec/ReplTxnTask.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/exec/ReplTxnTask.java
@@ -33,7 +33,10 @@ import org.apache.hadoop.hive.ql.plan.ReplTxnWork;
 import org.apache.hadoop.hive.ql.plan.api.StageType;
 import org.apache.hadoop.security.UserGroupInformation;
 import org.apache.hadoop.util.StringUtils;
+
+import java.util.ArrayList;
 import java.util.List;
+import java.util.stream.Collectors;
 
 /**
  * ReplTxnTask.
@@ -100,6 +103,18 @@ public class ReplTxnTask extends Task<ReplTxnWork> {
         assert work.getTxnToWriteIdList() != null;
         String dbName = work.getDbName();
         List <TxnToWriteId> txnToWriteIdList = work.getTxnToWriteIdList();
+
+        if (ReplUtils.filterTransactionOperations(conf)) {
+          // Implicitly open transactions since the OpenTxn(s) were filtered 
out to save space.
+          List<Long> txnIdsToOpen = txnToWriteIdList.stream()
+                  .map(txn2WriteId -> new Long(txn2WriteId.getTxnId()))
+                  .collect(Collectors.toList());
+          List<Long> openedTxnIds = txnManager.replOpenTxn(replPolicy, 
txnIdsToOpen, user);
+          assert openedTxnIds.size() == txnIdsToOpen.size();
+          LOG.info("Replayed (implicit) OpenTxn Event for policy " + 
replPolicy + " with srcTxn " +
+                  txnIdsToOpen.toString() + " and target txn id " + 
openedTxnIds.toString());
+        }
+
         txnManager.replAllocateTableWriteIdsBatch(dbName, tableName, 
replPolicy, txnToWriteIdList);
         LOG.info("Replayed alloc write Id Event for repl policy: " + 
replPolicy + " db Name : " + dbName +
             " txnToWriteIdList: " +txnToWriteIdList.toString() + " table name: 
" + tableName);
diff --git 
a/ql/src/java/org/apache/hadoop/hive/ql/exec/repl/util/ReplUtils.java 
b/ql/src/java/org/apache/hadoop/hive/ql/exec/repl/util/ReplUtils.java
index f0330e021e..d04202e02b 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/exec/repl/util/ReplUtils.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/exec/repl/util/ReplUtils.java
@@ -540,4 +540,9 @@ public class ReplUtils {
     int errorCode = ErrorMsg.getErrorMsg(e.getMessage()).getErrorCode();
     return errorCode > ErrorMsg.GENERIC_ERROR.getErrorCode();
   }
+
+  // True if REPL DUMP should do transaction optimization
+  public static boolean filterTransactionOperations(HiveConf conf) {
+    return (conf.getBoolVar(HiveConf.ConfVars.REPL_FILTER_TRANSACTIONS));
+  }
 }
diff --git 
a/ql/src/java/org/apache/hadoop/hive/ql/parse/repl/dump/events/AbortTxnHandler.java
 
b/ql/src/java/org/apache/hadoop/hive/ql/parse/repl/dump/events/AbortTxnHandler.java
index e1b184df94..5191e82185 100644
--- 
a/ql/src/java/org/apache/hadoop/hive/ql/parse/repl/dump/events/AbortTxnHandler.java
+++ 
b/ql/src/java/org/apache/hadoop/hive/ql/parse/repl/dump/events/AbortTxnHandler.java
@@ -17,12 +17,20 @@
  */
 package org.apache.hadoop.hive.ql.parse.repl.dump.events;
 
+import com.google.common.collect.Collections2;
+import org.apache.hadoop.hive.metastore.api.GetAllWriteEventInfoRequest;
 import org.apache.hadoop.hive.metastore.api.NotificationEvent;
+import org.apache.hadoop.hive.metastore.api.WriteEventInfo;
 import org.apache.hadoop.hive.metastore.messaging.AbortTxnMessage;
+import org.apache.hadoop.hive.metastore.messaging.json.JSONAbortTxnMessage;
+import org.apache.hadoop.hive.metastore.utils.StringUtils;
 import org.apache.hadoop.hive.ql.exec.repl.util.ReplUtils;
 import org.apache.hadoop.hive.ql.parse.repl.DumpType;
 import org.apache.hadoop.hive.ql.parse.repl.load.DumpMetaData;
 
+import java.util.ArrayList;
+import java.util.List;
+
 class AbortTxnHandler extends AbstractEventHandler<AbortTxnMessage> {
 
   AbortTxnHandler(NotificationEvent event) {
@@ -39,6 +47,16 @@ class AbortTxnHandler extends 
AbstractEventHandler<AbortTxnMessage> {
     if (!ReplUtils.includeAcidTableInDump(withinContext.hiveConf)) {
       return;
     }
+
+    if (ReplUtils.filterTransactionOperations(withinContext.hiveConf)) {
+      String contextDbName = 
StringUtils.normalizeIdentifier(withinContext.replScope.getDbName());
+      JSONAbortTxnMessage abortMsg = (JSONAbortTxnMessage)eventMessage;
+      if ((abortMsg.getDbsUpdated() == null) || 
!abortMsg.getDbsUpdated().contains(contextDbName)) {
+        LOG.info("Filter out #{} ABORT_TXN message : {}", fromEventId(), 
eventMessageAsJSON);
+        return;
+      }
+    }
+
     LOG.info("Processing#{} ABORT_TXN message : {}", fromEventId(), 
eventMessageAsJSON);
     DumpMetaData dmd = withinContext.createDmd(this);
     dmd.setPayload(eventMessageAsJSON);
diff --git 
a/ql/src/java/org/apache/hadoop/hive/ql/parse/repl/dump/events/CommitTxnHandler.java
 
b/ql/src/java/org/apache/hadoop/hive/ql/parse/repl/dump/events/CommitTxnHandler.java
index c733e289f7..96f1f847d3 100644
--- 
a/ql/src/java/org/apache/hadoop/hive/ql/parse/repl/dump/events/CommitTxnHandler.java
+++ 
b/ql/src/java/org/apache/hadoop/hive/ql/parse/repl/dump/events/CommitTxnHandler.java
@@ -159,6 +159,13 @@ class CommitTxnHandler extends 
AbstractEventHandler<CommitTxnMessage> {
       List<WriteEventInfo> writeEventInfoList = null;
       if (replicatingAcidEvents) {
         writeEventInfoList = getAllWriteEventInfo(withinContext);
+
+        if (ReplUtils.filterTransactionOperations(withinContext.hiveConf)
+           && (writeEventInfoList == null || writeEventInfoList.size() == 0)) {
+          // If optimizing transactions, no need to dump this one
+          // if there were no write events.
+          return;
+        }
       }
 
       int numEntry = (writeEventInfoList != null ? writeEventInfoList.size() : 
0);
diff --git 
a/ql/src/java/org/apache/hadoop/hive/ql/parse/repl/dump/events/OpenTxnHandler.java
 
b/ql/src/java/org/apache/hadoop/hive/ql/parse/repl/dump/events/OpenTxnHandler.java
index f06b41083e..baa3b4a169 100644
--- 
a/ql/src/java/org/apache/hadoop/hive/ql/parse/repl/dump/events/OpenTxnHandler.java
+++ 
b/ql/src/java/org/apache/hadoop/hive/ql/parse/repl/dump/events/OpenTxnHandler.java
@@ -39,6 +39,11 @@ class OpenTxnHandler extends 
AbstractEventHandler<OpenTxnMessage> {
     if (!ReplUtils.includeAcidTableInDump(withinContext.hiveConf)) {
       return;
     }
+
+    if (ReplUtils.filterTransactionOperations(withinContext.hiveConf)) {
+      return;
+    }
+
     LOG.info("Processing#{} OPEN_TXN message : {}", fromEventId(), 
eventMessageAsJSON);
     DumpMetaData dmd = withinContext.createDmd(this);
     dmd.setPayload(eventMessageAsJSON);
diff --git 
a/standalone-metastore/metastore-server/src/main/java/org/apache/hadoop/hive/metastore/HMSHandler.java
 
b/standalone-metastore/metastore-server/src/main/java/org/apache/hadoop/hive/metastore/HMSHandler.java
index 564ae91637..2cde211c29 100644
--- 
a/standalone-metastore/metastore-server/src/main/java/org/apache/hadoop/hive/metastore/HMSHandler.java
+++ 
b/standalone-metastore/metastore-server/src/main/java/org/apache/hadoop/hive/metastore/HMSHandler.java
@@ -8704,6 +8704,9 @@ public class HMSHandler extends FacebookBase implements 
IHMSHandler {
     getTxnHandler().abortTxn(rqst);
     boolean isHiveReplTxn = rqst.isSetReplPolicy() && 
TxnType.DEFAULT.equals(rqst.getTxn_type());
     if (listeners != null && !listeners.isEmpty() && !isHiveReplTxn) {
+      // Not adding dbsUpdated to AbortTxnEvent because
+      // only DbNotificationListener cares about it, and this is already
+      // handled with transactional listeners in TxnHandler.
       MetaStoreListenerNotifier.notifyEvent(listeners, EventType.ABORT_TXN,
           new AbortTxnEvent(rqst.getTxnid(), this));
     }
@@ -8714,6 +8717,7 @@ public class HMSHandler extends FacebookBase implements 
IHMSHandler {
     getTxnHandler().abortTxns(rqst);
     if (listeners != null && !listeners.isEmpty()) {
       for (Long txnId : rqst.getTxn_ids()) {
+        // See above abort_txn() note about not adding dbsUpdated.
         MetaStoreListenerNotifier.notifyEvent(listeners, EventType.ABORT_TXN,
             new AbortTxnEvent(txnId, this));
       }
diff --git 
a/standalone-metastore/metastore-server/src/main/java/org/apache/hadoop/hive/metastore/events/AbortTxnEvent.java
 
b/standalone-metastore/metastore-server/src/main/java/org/apache/hadoop/hive/metastore/events/AbortTxnEvent.java
index 5e00fcc751..9c63603d39 100644
--- 
a/standalone-metastore/metastore-server/src/main/java/org/apache/hadoop/hive/metastore/events/AbortTxnEvent.java
+++ 
b/standalone-metastore/metastore-server/src/main/java/org/apache/hadoop/hive/metastore/events/AbortTxnEvent.java
@@ -23,6 +23,9 @@ import org.apache.hadoop.classification.InterfaceStability;
 import org.apache.hadoop.hive.metastore.IHMSHandler;
 import org.apache.hadoop.hive.metastore.api.TxnType;
 
+import java.util.ArrayList;
+import java.util.List;
+
 /**
  * AbortTxnEvent
  * Event generated for roll backing a transaction
@@ -33,24 +36,30 @@ public class AbortTxnEvent extends ListenerEvent {
 
   private final Long txnId;
   private final TxnType txnType;
+  private final List<String> dbsUpdated;
 
   public AbortTxnEvent(Long transactionId, IHMSHandler handler) {
-    this(transactionId, null, handler);
+    this(transactionId, null, handler, null);
   }
 
   public AbortTxnEvent(Long transactionId, TxnType txnType) {
-    this(transactionId, txnType, null);
+    this(transactionId, txnType, null, null);
   }
 
   /**
    * @param transactionId Unique identification for the transaction that got 
rolledback.
    * @param txnType type of transaction
    * @param handler handler that is firing the event
+   * @param dbsUpdated list of databases that had update events
    */
-  public AbortTxnEvent(Long transactionId, TxnType txnType, IHMSHandler 
handler) {
+  public AbortTxnEvent(Long transactionId, TxnType txnType, IHMSHandler 
handler, List<String> dbsUpdated) {
     super(true, handler);
     this.txnId = transactionId;
     this.txnType = txnType;
+    this.dbsUpdated = new ArrayList<String>();
+    if (dbsUpdated != null) {
+      this.dbsUpdated.addAll(dbsUpdated);;
+    }
   }
 
   /**
@@ -66,4 +75,12 @@ public class AbortTxnEvent extends ListenerEvent {
   public TxnType getTxnType() {
     return txnType;
   }
+
+  /**
+   * Returns the list of the db names which might have written anything in 
this transaction.
+   * @return {@link List} of {@link String}
+   */
+  public List<String> getDbsUpdated() {
+    return dbsUpdated;
+  }
 }
diff --git 
a/standalone-metastore/metastore-server/src/main/java/org/apache/hadoop/hive/metastore/messaging/AbortTxnMessage.java
 
b/standalone-metastore/metastore-server/src/main/java/org/apache/hadoop/hive/metastore/messaging/AbortTxnMessage.java
index 1f75585123..e2ebed17f6 100644
--- 
a/standalone-metastore/metastore-server/src/main/java/org/apache/hadoop/hive/metastore/messaging/AbortTxnMessage.java
+++ 
b/standalone-metastore/metastore-server/src/main/java/org/apache/hadoop/hive/metastore/messaging/AbortTxnMessage.java
@@ -17,6 +17,8 @@
 
 package org.apache.hadoop.hive.metastore.messaging;
 
+import java.util.List;
+
 /**
  * HCat message sent when an abort transaction is done.
  */
@@ -33,4 +35,5 @@ public abstract class AbortTxnMessage extends EventMessage {
    */
   public abstract Long getTxnId();
 
+  public abstract List<String> getDbsUpdated();
 }
diff --git 
a/standalone-metastore/metastore-server/src/main/java/org/apache/hadoop/hive/metastore/messaging/MessageBuilder.java
 
b/standalone-metastore/metastore-server/src/main/java/org/apache/hadoop/hive/metastore/messaging/MessageBuilder.java
index 32ec325674..fef510d423 100644
--- 
a/standalone-metastore/metastore-server/src/main/java/org/apache/hadoop/hive/metastore/messaging/MessageBuilder.java
+++ 
b/standalone-metastore/metastore-server/src/main/java/org/apache/hadoop/hive/metastore/messaging/MessageBuilder.java
@@ -296,8 +296,8 @@ public class MessageBuilder {
     return new JSONCommitTxnMessage(MS_SERVER_URL, MS_SERVICE_PRINCIPAL, 
txnId, now());
   }
 
-  public AbortTxnMessage buildAbortTxnMessage(Long txnId) {
-    return new JSONAbortTxnMessage(MS_SERVER_URL, MS_SERVICE_PRINCIPAL, txnId, 
now());
+  public AbortTxnMessage buildAbortTxnMessage(Long txnId, List<String> 
dbsUpdated) {
+    return new JSONAbortTxnMessage(MS_SERVER_URL, MS_SERVICE_PRINCIPAL, txnId, 
now(), dbsUpdated);
   }
 
   public AllocWriteIdMessage buildAllocWriteIdMessage(List<TxnToWriteId> 
txnToWriteIdList,
diff --git 
a/standalone-metastore/metastore-server/src/main/java/org/apache/hadoop/hive/metastore/messaging/json/JSONAbortTxnMessage.java
 
b/standalone-metastore/metastore-server/src/main/java/org/apache/hadoop/hive/metastore/messaging/json/JSONAbortTxnMessage.java
index 5da816b51e..da72a3f0f5 100644
--- 
a/standalone-metastore/metastore-server/src/main/java/org/apache/hadoop/hive/metastore/messaging/json/JSONAbortTxnMessage.java
+++ 
b/standalone-metastore/metastore-server/src/main/java/org/apache/hadoop/hive/metastore/messaging/json/JSONAbortTxnMessage.java
@@ -22,6 +22,8 @@ import 
org.apache.hadoop.hive.metastore.messaging.AbortTxnMessage;
 
 import com.fasterxml.jackson.annotation.JsonProperty;
 
+import java.util.List;
+
 /**
  * JSON implementation of AbortTxnMessage
  */
@@ -39,17 +41,21 @@ public class JSONAbortTxnMessage extends AbortTxnMessage {
   @JsonProperty
   private String servicePrincipal;
 
+  @JsonProperty
+  private List<String> dbsUpdated;
+
   /**
    * Default constructor, needed for Jackson.
    */
   public JSONAbortTxnMessage() {
   }
 
-  public JSONAbortTxnMessage(String server, String servicePrincipal, Long 
txnid, Long timestamp) {
+  public JSONAbortTxnMessage(String server, String servicePrincipal, Long 
txnid, Long timestamp, List<String> dbsUpdated) {
     this.timestamp = timestamp;
     this.txnid = txnid;
     this.server = server;
     this.servicePrincipal = servicePrincipal;
+    this.dbsUpdated = dbsUpdated;
   }
 
   @Override
@@ -77,6 +83,11 @@ public class JSONAbortTxnMessage extends AbortTxnMessage {
     return server;
   }
 
+  @Override
+  public List<String> getDbsUpdated() {
+    return dbsUpdated;
+  }
+
   @Override
   public String toString() {
     try {
diff --git 
a/standalone-metastore/metastore-server/src/main/java/org/apache/hadoop/hive/metastore/txn/TxnHandler.java
 
b/standalone-metastore/metastore-server/src/main/java/org/apache/hadoop/hive/metastore/txn/TxnHandler.java
index cdd4eac53a..dec309243d 100644
--- 
a/standalone-metastore/metastore-server/src/main/java/org/apache/hadoop/hive/metastore/txn/TxnHandler.java
+++ 
b/standalone-metastore/metastore-server/src/main/java/org/apache/hadoop/hive/metastore/txn/TxnHandler.java
@@ -1072,8 +1072,10 @@ abstract class TxnHandler implements TxnStore, 
TxnStore.MutexAPI {
         }
 
         if (transactionalListeners != null && !isHiveReplTxn) {
+          List<String> dbsUpdated = getTxnDbsUpdated(txnid, dbConn);
           
MetaStoreListenerNotifier.notifyEventWithDirectSql(transactionalListeners,
-                  EventMessage.EventType.ABORT_TXN, new AbortTxnEvent(txnid, 
txnType), dbConn, sqlGenerator);
+                  EventMessage.EventType.ABORT_TXN,
+                  new AbortTxnEvent(txnid, txnType, null, dbsUpdated), dbConn, 
sqlGenerator);
         }
 
         LOG.debug("Going to commit");
@@ -1133,9 +1135,10 @@ abstract class TxnHandler implements TxnStore, 
TxnStore.MutexAPI {
 
         if (transactionalListeners != null){
           for (Long txnId : txnIds) {
+            List<String> dbsUpdated = getTxnDbsUpdated(txnId, dbConn);
             
MetaStoreListenerNotifier.notifyEventWithDirectSql(transactionalListeners,
                     EventMessage.EventType.ABORT_TXN, new AbortTxnEvent(txnId,
-                nonReadOnlyTxns.getOrDefault(txnId, TxnType.READ_ONLY)), 
dbConn, sqlGenerator);
+                nonReadOnlyTxns.getOrDefault(txnId, TxnType.READ_ONLY), null, 
dbsUpdated), dbConn, sqlGenerator);
           }
         }
         LOG.debug("Going to commit");
@@ -1668,6 +1671,39 @@ abstract class TxnHandler implements TxnStore, 
TxnStore.MutexAPI {
     }
   }
 
+  /**
+   * Returns the databases updated by txnId.
+   * Queries TXN_TO_WRITE_ID using txnId.
+   *
+   * @param txnId
+   * @throws MetaException
+   */
+    private List<String> getTxnDbsUpdated(long txnId, Connection dbConn) 
throws MetaException {
+    try {
+      try (Statement stmt = dbConn.createStatement()) {
+
+        String query = "SELECT DISTINCT T2W_DATABASE " +
+                " FROM \"TXN_TO_WRITE_ID\" \"COMMITTED\"" +
+                "   WHERE \"T2W_TXNID\" = " + txnId;
+
+        LOG.debug("Going to execute query: <" + query + ">");
+        try (ResultSet rs = stmt.executeQuery(query)) {
+          List<String> dbsUpdated = new ArrayList<String>();
+          while (rs.next()) {
+            dbsUpdated.add(rs.getString(1));
+          }
+          return dbsUpdated;
+        }
+      } catch (SQLException e) {
+        checkRetryable(e, "getTxnDbsUpdated");
+        throw new MetaException(StringUtils.stringifyException(e));
+      }
+    } catch (RetryException e) {
+      return getTxnDbsUpdated(txnId, dbConn);
+    }
+  }
+
+
   private ResultSet checkForWriteConflict(Statement stmt, long txnid) throws 
SQLException, MetaException {
     String writeConflictQuery = sqlGenerator.addLimitClause(1, 
"\"COMMITTED\".\"WS_TXNID\", \"COMMITTED\".\"WS_COMMIT_ID\", " +
             "\"COMMITTED\".\"WS_DATABASE\", \"COMMITTED\".\"WS_TABLE\", 
\"COMMITTED\".\"WS_PARTITION\", " +
diff --git 
a/standalone-metastore/metastore-server/src/test/java/org/apache/hadoop/hive/metastore/events/TestAbortTxnEventDbsUpdated.java
 
b/standalone-metastore/metastore-server/src/test/java/org/apache/hadoop/hive/metastore/events/TestAbortTxnEventDbsUpdated.java
new file mode 100644
index 0000000000..40b2432295
--- /dev/null
+++ 
b/standalone-metastore/metastore-server/src/test/java/org/apache/hadoop/hive/metastore/events/TestAbortTxnEventDbsUpdated.java
@@ -0,0 +1,68 @@
+/*
+ * 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
+ * <p>
+ * http://www.apache.org/licenses/LICENSE-2.0
+ * <p>
+ * 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.hadoop.hive.metastore.events;
+
+import org.apache.hadoop.hive.metastore.annotation.MetastoreUnitTest;
+import org.apache.hadoop.hive.metastore.api.NotificationEvent;
+import org.apache.hadoop.hive.metastore.api.TxnType;
+import org.apache.hadoop.hive.metastore.messaging.AbortTxnMessage;
+import org.apache.hadoop.hive.metastore.messaging.EventMessage;
+import org.apache.hadoop.hive.metastore.messaging.MessageBuilder;
+import org.apache.hadoop.hive.metastore.messaging.json.JSONMessageDeserializer;
+import org.apache.hadoop.hive.metastore.messaging.json.JSONMessageEncoder;
+import org.junit.Assert;
+import org.junit.Test;
+import org.junit.experimental.categories.Category;
+
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.HashSet;
+import java.util.List;
+import java.util.Set;
+
+@Category(MetastoreUnitTest.class)
+public class TestAbortTxnEventDbsUpdated {
+  @Test
+  public void testBackwardsCompatibility() {
+    final String json = 
"{\"txnid\":12787,\"timestamp\":1654116516,\"server\":\"\",\"servicePrincipal\":\"\"}";
+    JSONMessageDeserializer deserializer = new JSONMessageDeserializer();
+    AbortTxnMessage abortTxnMsg = deserializer.getAbortTxnMessage(json);
+    Assert.assertNull(abortTxnMsg.getDbsUpdated());
+    Assert.assertEquals(12787L, abortTxnMsg.getTxnId().longValue());
+  }
+
+  @Test
+  public void testSerializeDeserialize() {
+    List dbsUpdated = Arrays.asList("db1", "db22");
+    AbortTxnEvent event = new AbortTxnEvent(999L, TxnType.DEFAULT, null, 
dbsUpdated);
+    AbortTxnMessage msg =
+            
MessageBuilder.getInstance().buildAbortTxnMessage(event.getTxnId(), 
event.getDbsUpdated());
+    JSONMessageEncoder msgEncoder = new JSONMessageEncoder();
+    String json = msgEncoder.getSerializer().serialize(msg);
+
+    JSONMessageDeserializer deserializer = new JSONMessageDeserializer();
+    AbortTxnMessage abortTxnMsg = deserializer.getAbortTxnMessage(json);
+    Set<String> expected = new HashSet(dbsUpdated);
+    Assert.assertEquals(expected.size(), abortTxnMsg.getDbsUpdated().size());
+    List actual = abortTxnMsg.getDbsUpdated();
+    Assert.assertTrue(actual.remove("db1"));
+    Assert.assertTrue(actual.remove("db22"));
+    Assert.assertTrue(actual.isEmpty());
+    Assert.assertEquals(999L, abortTxnMsg.getTxnId().longValue());
+  }
+}

Reply via email to