This is an automated email from the ASF dual-hosted git repository.
mahesh 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 8e64482 HIVE-21880 : Enable flaky test
TestReplicationScenariosAcidTablesBootstrap.testBootstrapAcidTablesDuringIncrementalWithConcurrentWrites.
(Ashutosh Bapat reviewed by Mahesh Kumar Behera)
8e64482 is described below
commit 8e6448213c35361567f29808da6e74997494c5b9
Author: Ashutosh Bapat <[email protected]>
AuthorDate: Tue Jul 9 07:56:28 2019 +0530
HIVE-21880 : Enable flaky test
TestReplicationScenariosAcidTablesBootstrap.testBootstrapAcidTablesDuringIncrementalWithConcurrentWrites.
(Ashutosh Bapat reviewed by Mahesh Kumar Behera)
Signed-off-by: mbehera <[email protected]>
---
.../hcatalog/listener/DbNotificationListener.java | 10 +++++++
...estReplicationScenariosAcidTablesBootstrap.java | 2 --
.../hadoop/hive/metastore/HiveMetaStoreClient.java | 25 ++++++++++++----
.../hadoop/hive/metastore/MetaStoreDirectSql.java | 9 ++++++
.../apache/hadoop/hive/metastore/ObjectStore.java | 33 ++++++++++++++--------
5 files changed, 59 insertions(+), 20 deletions(-)
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 af01178..5f9d809 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
@@ -35,6 +35,7 @@ import org.apache.hadoop.fs.FileStatus;
import org.apache.hadoop.fs.FileSystem;
import org.apache.hadoop.fs.Path;
import org.apache.hadoop.fs.PathFilter;
+import org.apache.hadoop.hive.metastore.DatabaseProduct;
import org.apache.hadoop.hive.metastore.HiveMetaStore.HMSHandler;
import org.apache.hadoop.hive.metastore.MetaStoreEventListenerConstants;
import org.apache.hadoop.hive.metastore.RawStore;
@@ -997,6 +998,14 @@ public class DbNotificationListener extends
TransactionalMetaStoreEventListener
stmt.execute("SET @@session.sql_mode=ANSI_QUOTES");
}
+ // Derby doesn't allow FOR UPDATE to lock the row being selected (See
https://db.apache
+ // .org/derby/docs/10.1/ref/rrefsqlj31783.html) . So lock the whole
table. Since there's
+ // only one row in the table, this shouldn't cause any performance
degradation.
+ if (sqlGenerator.getDbProduct() == DatabaseProduct.DERBY) {
+ String lockingQuery = "lock table \"NOTIFICATION_SEQUENCE\" in
exclusive mode";
+ LOG.info("Going to execute query <" + lockingQuery + ">");
+ stmt.executeUpdate(lockingQuery);
+ }
String s = sqlGenerator.addForUpdateClause("select \"NEXT_EVENT_ID\" " +
" from \"NOTIFICATION_SEQUENCE\"");
LOG.debug("Going to execute query <" + s + ">");
@@ -1081,6 +1090,7 @@ public class DbNotificationListener extends
TransactionalMetaStoreEventListener
String.join(", ", params) + ")");
pst.execute();
+ event.setEventId(nextEventId);
// Set the DB_NOTIFICATION_EVENT_ID for future reference by other
listeners.
if (event.isSetEventId()) {
listenerEvent.putParameter(
diff --git
a/itests/hive-unit/src/test/java/org/apache/hadoop/hive/ql/parse/TestReplicationScenariosAcidTablesBootstrap.java
b/itests/hive-unit/src/test/java/org/apache/hadoop/hive/ql/parse/TestReplicationScenariosAcidTablesBootstrap.java
index 94070e5..f475b1e 100644
---
a/itests/hive-unit/src/test/java/org/apache/hadoop/hive/ql/parse/TestReplicationScenariosAcidTablesBootstrap.java
+++
b/itests/hive-unit/src/test/java/org/apache/hadoop/hive/ql/parse/TestReplicationScenariosAcidTablesBootstrap.java
@@ -30,7 +30,6 @@ import org.apache.hadoop.hive.ql.ErrorMsg;
import org.apache.hadoop.hive.ql.exec.repl.util.ReplUtils;
import org.junit.Assert;
-import org.junit.Ignore;
import org.junit.Test;
import org.junit.BeforeClass;
@@ -256,7 +255,6 @@ public class TestReplicationScenariosAcidTablesBootstrap
verifyCompactionQueue(tables, replicatedDbName, replicaConf);
}
- @Ignore("HIVE-21879: Disabling this testcase as it is flaky.")
@Test
public void testBootstrapAcidTablesDuringIncrementalWithConcurrentWrites()
throws Throwable {
// Dump and load bootstrap without ACID tables.
diff --git
a/standalone-metastore/metastore-common/src/main/java/org/apache/hadoop/hive/metastore/HiveMetaStoreClient.java
b/standalone-metastore/metastore-common/src/main/java/org/apache/hadoop/hive/metastore/HiveMetaStoreClient.java
index f947ee1..153f4b8 100644
---
a/standalone-metastore/metastore-common/src/main/java/org/apache/hadoop/hive/metastore/HiveMetaStoreClient.java
+++
b/standalone-metastore/metastore-common/src/main/java/org/apache/hadoop/hive/metastore/HiveMetaStoreClient.java
@@ -148,6 +148,9 @@ public class HiveMetaStoreClient implements
IMetaStoreClient, AutoCloseable {
//copied from ErrorMsg.java
private static final String REPL_EVENTS_MISSING_IN_METASTORE = "Notification
events are missing in the meta store.";
+ private static final String REPL_EVENTS_WITH_DUPLICATE_ID_IN_METASTORE =
+ "Notification events with duplicate event ids in the meta store.";
+
static final protected Logger LOG =
LoggerFactory.getLogger(HiveMetaStoreClient.class);
public HiveMetaStoreClient(Configuration conf) throws MetaException {
@@ -3236,18 +3239,28 @@ public class HiveMetaStoreClient implements
IMetaStoreClient, AutoCloseable {
NotificationEventResponse filtered = new NotificationEventResponse();
if (rsp != null && rsp.getEvents() != null) {
long nextEventId = lastEventId + 1;
+ long prevEventId = lastEventId;
for (NotificationEvent e : rsp.getEvents()) {
+ LOG.debug("Got event with id : " + e.getEventId());
if (e.getEventId() != nextEventId) {
- LOG.error("Requested events are found missing in NOTIFICATION_LOG
table. Expected: {}, Actual: {}. "
- + "Probably, cleaner would've cleaned it up. "
- + "Try setting higher value for
hive.metastore.event.db.listener.timetolive. "
- + "Also, bootstrap the system again to get back the
consistent replicated state.",
- nextEventId, e.getEventId());
- throw new IllegalStateException(REPL_EVENTS_MISSING_IN_METASTORE);
+ if (e.getEventId() == prevEventId) {
+ LOG.error("NOTIFICATION_LOG table has multiple events with the
same event Id {}. " +
+ "Something went wrong when inserting notification events.
Bootstrap the system " +
+ "again to get back teh consistent replicated state.",
prevEventId);
+ throw new
IllegalStateException(REPL_EVENTS_WITH_DUPLICATE_ID_IN_METASTORE);
+ } else {
+ LOG.error("Requested events are found missing in NOTIFICATION_LOG
table. Expected: {}, Actual: {}. "
+ + "Probably, cleaner would've cleaned it up. "
+ + "Try setting higher value for
hive.metastore.event.db.listener.timetolive. "
+ + "Also, bootstrap the system again to get back
the consistent replicated state.",
+ nextEventId, e.getEventId());
+ throw new IllegalStateException(REPL_EVENTS_MISSING_IN_METASTORE);
+ }
}
if ((filter != null) && filter.accept(e)) {
filtered.addToEvents(e);
}
+ prevEventId = nextEventId;
nextEventId++;
}
}
diff --git
a/standalone-metastore/metastore-server/src/main/java/org/apache/hadoop/hive/metastore/MetaStoreDirectSql.java
b/standalone-metastore/metastore-server/src/main/java/org/apache/hadoop/hive/metastore/MetaStoreDirectSql.java
index c11b3b8..acb4646 100644
---
a/standalone-metastore/metastore-server/src/main/java/org/apache/hadoop/hive/metastore/MetaStoreDirectSql.java
+++
b/standalone-metastore/metastore-server/src/main/java/org/apache/hadoop/hive/metastore/MetaStoreDirectSql.java
@@ -2673,4 +2673,13 @@ class MetaStoreDirectSql {
query.closeAll();
}
}
+
+ public void lockDbTable(String tableName) throws MetaException {
+ String lockCommand = "lock table \"" + tableName + "\" in exclusive mode";
+ try {
+ executeNoResult(lockCommand);
+ } catch (SQLException sqle) {
+ throw new MetaException("Error while locking table " + tableName + ": "
+ sqle.getMessage());
+ }
+ }
}
diff --git
a/standalone-metastore/metastore-server/src/main/java/org/apache/hadoop/hive/metastore/ObjectStore.java
b/standalone-metastore/metastore-server/src/main/java/org/apache/hadoop/hive/metastore/ObjectStore.java
index 072761a..509fcb2 100644
---
a/standalone-metastore/metastore-server/src/main/java/org/apache/hadoop/hive/metastore/ObjectStore.java
+++
b/standalone-metastore/metastore-server/src/main/java/org/apache/hadoop/hive/metastore/ObjectStore.java
@@ -10000,17 +10000,26 @@ public class ObjectStore implements RawStore,
Configurable {
}
}
- private void lockForUpdate() throws MetaException {
- String selectQuery = "select \"NEXT_EVENT_ID\" from
\"NOTIFICATION_SEQUENCE\"";
- String selectForUpdateQuery = sqlGenerator.addForUpdateClause(selectQuery);
- new RetryingExecutor(conf, () -> {
- prepareQuotes();
- Query query = pm.newQuery("javax.jdo.query.SQL", selectForUpdateQuery);
- query.setUnique(true);
- // only need to execute it to get db Lock
- query.execute();
- query.closeAll();
- }).run();
+ private void lockNotificationSequenceForUpdate() throws MetaException {
+ if (sqlGenerator.getDbProduct() == DatabaseProduct.DERBY && directSql !=
null) {
+ // Derby doesn't allow FOR UPDATE to lock the row being selected (See
https://db.apache
+ // .org/derby/docs/10.1/ref/rrefsqlj31783.html) . So lock the whole
table. Since there's
+ // only one row in the table, this shouldn't cause any performance
degradation.
+ new RetryingExecutor(conf, () -> {
+ directSql.lockDbTable("NOTIFICATION_SEQUENCE");
+ }).run();
+ } else {
+ String selectQuery = "select \"NEXT_EVENT_ID\" from
\"NOTIFICATION_SEQUENCE\"";
+ String lockingQuery = sqlGenerator.addForUpdateClause(selectQuery);
+ new RetryingExecutor(conf, () -> {
+ prepareQuotes();
+ Query query = pm.newQuery("javax.jdo.query.SQL", lockingQuery);
+ query.setUnique(true);
+ // only need to execute it to get db Lock
+ query.execute();
+ query.closeAll();
+ }).run();
+ }
}
static class RetryingExecutor {
@@ -10074,7 +10083,7 @@ public class ObjectStore implements RawStore,
Configurable {
Query query = null;
try {
openTransaction();
- lockForUpdate();
+ lockNotificationSequenceForUpdate();
query = pm.newQuery(MNotificationNextId.class);
Collection<MNotificationNextId> ids = (Collection) query.execute();
MNotificationNextId mNotificationNextId = null;