[ 
https://issues.apache.org/jira/browse/HIVE-21880?focusedWorklogId=270020&page=com.atlassian.jira.plugin.system.issuetabpanels:worklog-tabpanel#worklog-270020
 ]

ASF GitHub Bot logged work on HIVE-21880:
-----------------------------------------

                Author: ASF GitHub Bot
            Created on: 01/Jul/19 07:57
            Start Date: 01/Jul/19 07:57
    Worklog Time Spent: 10m 
      Work Description: maheshk114 commented on pull request #684: HIVE-21880 : 
Enable flaky test 
TestReplicationScenariosAcidTablesBootstrap.testBootstrapAcidTablesDuringIncrementalWithConcurrentWrites.
URL: https://github.com/apache/hive/pull/684#discussion_r298916973
 
 

 ##########
 File path: 
standalone-metastore/metastore-server/src/main/java/org/apache/hadoop/hive/metastore/ObjectStore.java
 ##########
 @@ -10000,17 +10000,25 @@ private void prepareQuotes() throws SQLException {
     }
   }
 
-  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. Also we not
+      // suggest Derby to be used in production so that's fine.
+      directSql.lockDbTable("NOTIFICATION_SEQUENCE");
 
 Review comment:
   if it can have only one records ..does select for update makes any sense ? 
can we make it lock table for other kind of db also ?
 
----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
us...@infra.apache.org


Issue Time Tracking
-------------------

    Worklog Id:     (was: 270020)
    Time Spent: 50m  (was: 40m)

> Enable flaky test 
> TestReplicationScenariosAcidTablesBootstrap.testBootstrapAcidTablesDuringIncrementalWithConcurrentWrites.
> ---------------------------------------------------------------------------------------------------------------------------
>
>                 Key: HIVE-21880
>                 URL: https://issues.apache.org/jira/browse/HIVE-21880
>             Project: Hive
>          Issue Type: Bug
>          Components: repl
>    Affects Versions: 4.0.0
>            Reporter: Sankar Hariappan
>            Assignee: Ashutosh Bapat
>            Priority: Major
>              Labels: pull-request-available
>         Attachments: HIVE-21880.01.patch, HIVE-21880.02.patch, 
> HIVE-21880.03.patch
>
>          Time Spent: 50m
>  Remaining Estimate: 0h
>
> Need tp enable 
> TestReplicationScenariosAcidTablesBootstrap.testBootstrapAcidTablesDuringIncrementalWithConcurrentWrites
>  which is disabled as it is flaky and randomly failing with below error.
> {code}
> Error Message
> Notification events are missing in the meta store.
> Stacktrace
> java.lang.IllegalStateException: Notification events are missing in the meta 
> store.
>       at 
> org.apache.hadoop.hive.metastore.HiveMetaStoreClient.getNextNotification(HiveMetaStoreClient.java:3246)
>       at sun.reflect.NativeMethodAccessorImpl.invoke0(Native Method)
>       at 
> sun.reflect.NativeMethodAccessorImpl.invoke(NativeMethodAccessorImpl.java:62)
>       at 
> sun.reflect.DelegatingMethodAccessorImpl.invoke(DelegatingMethodAccessorImpl.java:43)
>       at java.lang.reflect.Method.invoke(Method.java:498)
>       at 
> org.apache.hadoop.hive.metastore.RetryingMetaStoreClient.invoke(RetryingMetaStoreClient.java:212)
>       at com.sun.proxy.$Proxy58.getNextNotification(Unknown Source)
>       at 
> org.apache.hadoop.hive.ql.metadata.events.EventUtils$MSClientNotificationFetcher.getNextNotificationEvents(EventUtils.java:107)
>       at 
> org.apache.hadoop.hive.ql.metadata.events.EventUtils$NotificationEventIterator.fetchNextBatch(EventUtils.java:159)
>       at 
> org.apache.hadoop.hive.ql.metadata.events.EventUtils$NotificationEventIterator.hasNext(EventUtils.java:189)
>       at 
> org.apache.hadoop.hive.ql.exec.repl.ReplDumpTask.incrementalDump(ReplDumpTask.java:231)
>       at 
> org.apache.hadoop.hive.ql.exec.repl.ReplDumpTask.execute(ReplDumpTask.java:121)
>       at org.apache.hadoop.hive.ql.exec.Task.executeTask(Task.java:212)
>       at 
> org.apache.hadoop.hive.ql.exec.TaskRunner.runSequential(TaskRunner.java:103)
>       at org.apache.hadoop.hive.ql.Driver.launchTask(Driver.java:2709)
>       at org.apache.hadoop.hive.ql.Driver.execute(Driver.java:2361)
>       at org.apache.hadoop.hive.ql.Driver.runInternal(Driver.java:2028)
>       at org.apache.hadoop.hive.ql.Driver.run(Driver.java:1788)
>       at org.apache.hadoop.hive.ql.Driver.run(Driver.java:1782)
>       at 
> org.apache.hadoop.hive.ql.reexec.ReExecDriver.run(ReExecDriver.java:162)
>       at 
> org.apache.hadoop.hive.ql.reexec.ReExecDriver.run(ReExecDriver.java:223)
>       at 
> org.apache.hadoop.hive.ql.parse.WarehouseInstance.run(WarehouseInstance.java:227)
>       at 
> org.apache.hadoop.hive.ql.parse.WarehouseInstance.dump(WarehouseInstance.java:282)
>       at 
> org.apache.hadoop.hive.ql.parse.WarehouseInstance.dump(WarehouseInstance.java:265)
>       at 
> org.apache.hadoop.hive.ql.parse.WarehouseInstance.dump(WarehouseInstance.java:289)
>       at 
> org.apache.hadoop.hive.ql.parse.TestReplicationScenariosAcidTablesBootstrap.testBootstrapAcidTablesDuringIncrementalWithConcurrentWrites(TestReplicationScenariosAcidTablesBootstrap.java:328)
>       at sun.reflect.NativeMethodAccessorImpl.invoke0(Native Method)
>       at 
> sun.reflect.NativeMethodAccessorImpl.invoke(NativeMethodAccessorImpl.java:62)
>       at 
> sun.reflect.DelegatingMethodAccessorImpl.invoke(DelegatingMethodAccessorImpl.java:43)
>       at java.lang.reflect.Method.invoke(Method.java:498)
>       at 
> org.junit.runners.model.FrameworkMethod$1.runReflectiveCall(FrameworkMethod.java:47)
>       at 
> org.junit.internal.runners.model.ReflectiveCallable.run(ReflectiveCallable.java:12)
>       at 
> org.junit.runners.model.FrameworkMethod.invokeExplosively(FrameworkMethod.java:44)
>       at 
> org.junit.internal.runners.statements.InvokeMethod.evaluate(InvokeMethod.java:17)
>       at 
> org.junit.internal.runners.statements.RunBefores.evaluate(RunBefores.java:26)
>       at 
> org.junit.internal.runners.statements.RunAfters.evaluate(RunAfters.java:27)
>       at org.junit.rules.TestWatcher$1.evaluate(TestWatcher.java:55)
>       at org.junit.rules.RunRules.evaluate(RunRules.java:20)
>       at org.junit.runners.ParentRunner.runLeaf(ParentRunner.java:271)
>       at 
> org.junit.runners.BlockJUnit4ClassRunner.runChild(BlockJUnit4ClassRunner.java:70)
>       at 
> org.junit.runners.BlockJUnit4ClassRunner.runChild(BlockJUnit4ClassRunner.java:50)
>       at org.junit.runners.ParentRunner$3.run(ParentRunner.java:238)
>       at org.junit.runners.ParentRunner$1.schedule(ParentRunner.java:63)
>       at org.junit.runners.ParentRunner.runChildren(ParentRunner.java:236)
>       at org.junit.runners.ParentRunner.access$000(ParentRunner.java:53)
>       at org.junit.runners.ParentRunner$2.evaluate(ParentRunner.java:229)
>       at 
> org.junit.internal.runners.statements.RunBefores.evaluate(RunBefores.java:26)
>       at 
> org.junit.internal.runners.statements.RunAfters.evaluate(RunAfters.java:27)
>       at org.junit.runners.ParentRunner.run(ParentRunner.java:309)
>       at 
> org.apache.maven.surefire.junit4.JUnit4Provider.execute(JUnit4Provider.java:365)
>       at 
> org.apache.maven.surefire.junit4.JUnit4Provider.executeWithRerun(JUnit4Provider.java:273)
>       at 
> org.apache.maven.surefire.junit4.JUnit4Provider.executeTestSet(JUnit4Provider.java:238)
>       at 
> org.apache.maven.surefire.junit4.JUnit4Provider.invoke(JUnit4Provider.java:159)
>       at 
> org.apache.maven.surefire.booter.ForkedBooter.invokeProviderInSameClassLoader(ForkedBooter.java:379)
>       at 
> org.apache.maven.surefire.booter.ForkedBooter.runSuitesInProcess(ForkedBooter.java:340)
>       at 
> org.apache.maven.surefire.booter.ForkedBooter.execute(ForkedBooter.java:125)
>       at 
> org.apache.maven.surefire.booter.ForkedBooter.main(ForkedBooter.java:413)
> {code}
> https://builds.apache.org/job/PreCommit-HIVE-Build/17591/testReport/org.apache.hadoop.hive.ql.parse/TestReplicationScenariosAcidTablesBootstrap/testBootstrapAcidTablesDuringIncrementalWithConcurrentWrites/



--
This message was sent by Atlassian JIRA
(v7.6.3#76005)

Reply via email to