Details
-
Bug
-
Status: Open
-
Major
-
Resolution: Unresolved
-
4.0.0-alpha-2
-
None
-
None
Description
sysdb.q and strict_managed_tables_sysdb.q fail when using MySQL as standalone metastore db.
The issue can be reproduced with the following command:
mvn test -Dtest=TestMiniLlapLocalCliDriver -Dqfile="sysdb.q,strict_managed_tables_sysdb.q" -Dtest.metastore.db=mysql -pl itests/qtest -Pitests
The errors are as follows:
------------------------------------------------------------------------------- Test set: org.apache.hadoop.hive.cli.TestMysqlMetastoreCliDriver ------------------------------------------------------------------------------- Tests run: 3, Failures: 2, Errors: 0, Skipped: 0, Time elapsed: 282.638 s <<< FAILURE! - in org.apache.hadoop.hive.cli.TestMysqlMetastoreCliDriver org.apache.hadoop.hive.cli.TestMysqlMetastoreCliDriver.testCliDriver[strict_managed_tables_sysdb] Time elapsed: 41.104 s <<< FAILURE! java.lang.AssertionError: Client execution failed with error code = 2 running select tbl_name, tbl_type from tbls where tbl_name like 'smt_sysdb%' order by tbl_name fname=strict_managed_tables_sysdb.q See ./ql/target/tmp/log/hive.log or ./itests/qtest/target/tmp/log/hive.log, or check ./ql/target/surefire-reports or ./itests/qtest/target/surefire-reports/ for specific test cases logs. org.apache.hadoop.hive.ql.metadata.HiveException: Vertex failed, vertexName=Map 1, vertexId=vertex_1649344918728_0001_33_00, diagnostics=[Task failed, taskId=task_1649344918728_0001_33_00_000000, diagnostics=[TaskAttempt 0 failed, info=[Error: Error while running task ( failure ) : attempt_1649344918728_0001_33_00_000000_0:java.lang.RuntimeException: org.apache.hadoop.hive.ql.metadata.HiveException: java.io.IOException: java.io.IOException: org.apache.hive.storage.jdbc.exception.HiveJdbcDatabaseAccessException: Caught exception while trying to execute query:You have an error in your SQL syntax; check the manual that corresponds to your MySQL server version for the right syntax to use near '"TBLS"' at line 14 at org.apache.hadoop.hive.ql.exec.tez.TezProcessor.initializeAndRunProcessor(TezProcessor.java:348) at org.apache.hadoop.hive.ql.exec.tez.TezProcessor.run(TezProcessor.java:276) at org.apache.tez.runtime.LogicalIOProcessorRuntimeTask.run(LogicalIOProcessorRuntimeTask.java:381) at org.apache.tez.runtime.task.TaskRunner2Callable$1.run(TaskRunner2Callable.java:82) at org.apache.tez.runtime.task.TaskRunner2Callable$1.run(TaskRunner2Callable.java:69) at java.security.AccessController.doPrivileged(Native Method) at javax.security.auth.Subject.doAs(Subject.java:422) at org.apache.hadoop.security.UserGroupInformation.doAs(UserGroupInformation.java:1682) at org.apache.tez.runtime.task.TaskRunner2Callable.callInternal(TaskRunner2Callable.java:69) at org.apache.tez.runtime.task.TaskRunner2Callable.callInternal(TaskRunner2Callable.java:39) at org.apache.tez.common.CallableWithNdc.call(CallableWithNdc.java:36) at org.apache.hadoop.hive.llap.daemon.impl.StatsRecordingThreadPool$WrappedCallable.call(StatsRecordingThreadPool.java:118) at java.util.concurrent.FutureTask.run(FutureTask.java:266) at java.util.concurrent.ThreadPoolExecutor.runWorker(ThreadPoolExecutor.java:1149) at java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:624) at java.lang.Thread.run(Thread.java:748) Caused by: org.apache.hadoop.hive.ql.metadata.HiveException: java.io.IOException: java.io.IOException: org.apache.hive.storage.jdbc.exception.HiveJdbcDatabaseAccessException: Caught exception while trying to execute query:You have an error in your SQL syntax; check the manual that corresponds to your MySQL server version for the right syntax to use near '"TBLS"' at line 14 at org.apache.hadoop.hive.ql.exec.tez.MapRecordSource.pushRecord(MapRecordSource.java:89) at org.apache.hadoop.hive.ql.exec.tez.MapRecordProcessor.run(MapRecordProcessor.java:414) at org.apache.hadoop.hive.ql.exec.tez.TezProcessor.initializeAndRunProcessor(TezProcessor.java:293) ... 15 more Caused by: java.io.IOException: java.io.IOException: org.apache.hive.storage.jdbc.exception.HiveJdbcDatabaseAccessException: Caught exception while trying to execute query:You have an error in your SQL syntax; check the manual that corresponds to your MySQL server version for the right syntax to use near '"TBLS"' at line 14 at org.apache.hadoop.hive.io.HiveIOExceptionHandlerChain.handleRecordReaderNextException(HiveIOExceptionHandlerChain.java:121) at org.apache.hadoop.hive.io.HiveIOExceptionHandlerUtil.handleRecordReaderNextException(HiveIOExceptionHandlerUtil.java:77) at org.apache.hadoop.hive.ql.io.HiveContextAwareRecordReader.doNext(HiveContextAwareRecordReader.java:380) at org.apache.hadoop.hive.ql.io.HiveRecordReader.doNext(HiveRecordReader.java:82) at org.apache.hadoop.hive.ql.io.HiveContextAwareRecordReader.next(HiveContextAwareRecordReader.java:119) at org.apache.hadoop.hive.ql.io.HiveContextAwareRecordReader.next(HiveContextAwareRecordReader.java:59) at org.apache.hadoop.mapred.split.TezGroupedSplitsInputFormat$TezGroupedSplitsRecordReader.next(TezGroupedSplitsInputFormat.java:145) at org.apache.tez.mapreduce.lib.MRReaderMapred.next(MRReaderMapred.java:116) at org.apache.hadoop.hive.ql.exec.tez.MapRecordSource.pushRecord(MapRecordSource.java:75) ... 17 more Caused by: java.io.IOException: org.apache.hive.storage.jdbc.exception.HiveJdbcDatabaseAccessException: Caught exception while trying to execute query:You have an error in your SQL syntax; check the manual that corresponds to your MySQL server version for the right syntax to use near '"TBLS"' at line 14 at org.apache.hive.storage.jdbc.JdbcRecordReader.next(JdbcRecordReader.java:85) at org.apache.hive.storage.jdbc.JdbcRecordReader.next(JdbcRecordReader.java:35) at org.apache.hadoop.hive.ql.io.HiveContextAwareRecordReader.doNext(HiveContextAwareRecordReader.java:375) ... 23 more Caused by: org.apache.hive.storage.jdbc.exception.HiveJdbcDatabaseAccessException: Caught exception while trying to execute query:You have an error in your SQL syntax; check the manual that corresponds to your MySQL server version for the right syntax to use near '"TBLS"' at line 14 at org.apache.hive.storage.jdbc.dao.GenericJdbcDatabaseAccessor.getRecordIterator(GenericJdbcDatabaseAccessor.java:187) at org.apache.hive.storage.jdbc.JdbcRecordReader.next(JdbcRecordReader.java:58) ... 25 more Caused by: java.sql.SQLSyntaxErrorException: You have an error in your SQL syntax; check the manual that corresponds to your MySQL server version for the right syntax to use near '"TBLS"' at line 14 at com.mysql.cj.jdbc.exceptions.SQLError.createSQLException(SQLError.java:120) at com.mysql.cj.jdbc.exceptions.SQLExceptionsMapping.translateException(SQLExceptionsMapping.java:122) at com.mysql.cj.jdbc.ClientPreparedStatement.executeInternal(ClientPreparedStatement.java:953) at com.mysql.cj.jdbc.ClientPreparedStatement.executeQuery(ClientPreparedStatement.java:1009) at org.apache.commons.dbcp2.DelegatingPreparedStatement.executeQuery(DelegatingPreparedStatement.java:122) at org.apache.commons.dbcp2.DelegatingPreparedStatement.executeQuery(DelegatingPreparedStatement.java:122) at org.apache.hive.storage.jdbc.dao.GenericJdbcDatabaseAccessor.getRecordIterator(GenericJdbcDatabaseAccessor.java:180) ... 26 more ], TaskAttempt 1 failed, info=[Error: Error while running task ( failure ) : attempt_1649344918728_0001_33_00_000000_1:java.lang.RuntimeException: org.apache.hadoop.hive.ql.metadata.HiveException: java.io.IOException: java.io.IOException: org.apache.hive.storage.jdbc.exception.HiveJdbcDatabaseAccessException: Caught exception while trying to execute query:You have an error in your SQL syntax; check the manual that corresponds to your MySQL server version for the right syntax to use near '"TBLS"' at line 14 at org.apache.hadoop.hive.ql.exec.tez.TezProcessor.initializeAndRunProcessor(TezProcessor.java:348) at org.apache.hadoop.hive.ql.exec.tez.TezProcessor.run(TezProcessor.java:276) at org.apache.tez.runtime.LogicalIOProcessorRuntimeTask.run(LogicalIOProcessorRuntimeTask.java:381) at org.apache.tez.runtime.task.TaskRunner2Callable$1.run(TaskRunner2Callable.java:82) at org.apache.tez.runtime.task.TaskRunner2Callable$1.run(TaskRunner2Callable.java:69) at java.security.AccessController.doPrivileged(Native Method) at javax.security.auth.Subject.doAs(Subject.java:422) at org.apache.hadoop.security.UserGroupInformation.doAs(UserGroupInformation.java:1682) at org.apache.tez.runtime.task.TaskRunner2Callable.callInternal(TaskRunner2Callable.java:69) at org.apache.tez.runtime.task.TaskRunner2Callable.callInternal(TaskRunner2Callable.java:39) at org.apache.tez.common.CallableWithNdc.call(CallableWithNdc.java:36) at org.apache.hadoop.hive.llap.daemon.impl.StatsRecordingThreadPool$WrappedCallable.call(StatsRecordingThreadPool.java:118) at java.util.concurrent.FutureTask.run(FutureTask.java:266) at java.util.concurrent.ThreadPoolExecutor.runWorker(ThreadPoolExecutor.java:1149) at java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:624) at java.lang.Thread.run(Thread.java:748) Caused by: org.apache.hadoop.hive.ql.metadata.HiveException: java.io.IOException: java.io.IOException: org.apache.hive.storage.jdbc.exception.HiveJdbcDatabaseAccessException: Caught exception while trying to execute query:You have an error in your SQL syntax; check the manual that corresponds to your MySQL server version for the right syntax to use near '"TBLS"' at line 14 at org.apache.hadoop.hive.ql.exec.tez.MapRecordSource.pushRecord(MapRecordSource.java:89) at org.apache.hadoop.hive.ql.exec.tez.MapRecordProcessor.run(MapRecordProcessor.java:414) at org.apache.hadoop.hive.ql.exec.tez.TezProcessor.initializeAndRunProcessor(TezProcessor.java:293) ... 15 more Caused by: java.io.IOException: java.io.IOException: org.apache.hive.storage.jdbc.exception.HiveJdbcDatabaseAccessException: Caught exception while trying to execute query:You have an error in your SQL syntax; check the manual that corresponds to your MySQL server version for the right syntax to use near '"TBLS"' at line 14 at org.apache.hadoop.hive.io.HiveIOExceptionHandlerChain.handleRecordReaderNextException(HiveIOExceptionHandlerChain.java:121) at org.apache.hadoop.hive.io.HiveIOExceptionHandlerUtil.handleRecordReaderNextException(HiveIOExceptionHandlerUtil.java:77) at org.apache.hadoop.hive.ql.io.HiveContextAwareRecordReader.doNext(HiveContextAwareRecordReader.java:380) at org.apache.hadoop.hive.ql.io.HiveRecordReader.doNext(HiveRecordReader.java:82) at org.apache.hadoop.hive.ql.io.HiveContextAwareRecordReader.next(HiveContextAwareRecordReader.java:119) at org.apache.hadoop.hive.ql.io.HiveContextAwareRecordReader.next(HiveContextAwareRecordReader.java:59) at org.apache.hadoop.mapred.split.TezGroupedSplitsInputFormat$TezGroupedSplitsRecordReader.next(TezGroupedSplitsInputFormat.java:145) at org.apache.tez.mapreduce.lib.MRReaderMapred.next(MRReaderMapred.java:116) at org.apache.hadoop.hive.ql.exec.tez.MapRecordSource.pushRecord(MapRecordSource.java:75) ... 17 more Caused by: java.io.IOException: org.apache.hive.storage.jdbc.exception.HiveJdbcDatabaseAccessException: Caught exception while trying to execute query:You have an error in your SQL syntax; check the manual that corresponds to your MySQL server version for the right syntax to use near '"TBLS"' at line 14 at org.apache.hive.storage.jdbc.JdbcRecordReader.next(JdbcRecordReader.java:85) at org.apache.hive.storage.jdbc.JdbcRecordReader.next(JdbcRecordReader.java:35) at org.apache.hadoop.hive.ql.io.HiveContextAwareRecordReader.doNext(HiveContextAwareRecordReader.java:375) ... 23 more Caused by: org.apache.hive.storage.jdbc.exception.HiveJdbcDatabaseAccessException: Caught exception while trying to execute query:You have an error in your SQL syntax; check the manual that corresponds to your MySQL server version for the right syntax to use near '"TBLS"' at line 14 at org.apache.hive.storage.jdbc.dao.GenericJdbcDatabaseAccessor.getRecordIterator(GenericJdbcDatabaseAccessor.java:187) at org.apache.hive.storage.jdbc.JdbcRecordReader.next(JdbcRecordReader.java:58) ... 25 more Caused by: java.sql.SQLSyntaxErrorException: You have an error in your SQL syntax; check the manual that corresponds to your MySQL server version for the right syntax to use near '"TBLS"' at line 14 at com.mysql.cj.jdbc.exceptions.SQLError.createSQLException(SQLError.java:120) at com.mysql.cj.jdbc.exceptions.SQLExceptionsMapping.translateException(SQLExceptionsMapping.java:122) at com.mysql.cj.jdbc.ClientPreparedStatement.executeInternal(ClientPreparedStatement.java:953) at com.mysql.cj.jdbc.ClientPreparedStatement.executeQuery(ClientPreparedStatement.java:1009) at org.apache.commons.dbcp2.DelegatingPreparedStatement.executeQuery(DelegatingPreparedStatement.java:122) at org.apache.commons.dbcp2.DelegatingPreparedStatement.executeQuery(DelegatingPreparedStatement.java:122) at org.apache.hive.storage.jdbc.dao.GenericJdbcDatabaseAccessor.getRecordIterator(GenericJdbcDatabaseAccessor.java:180) ... 26 more ]], Vertex did not succeed due to OWN_TASK_FAILURE, failedTasks:1 killedTasks:0, Vertex vertex_1649344918728_0001_33_00 [Map 1] killed/failed due to:OWN_TASK_FAILURE]Vertex killed, vertexName=Reducer 2, vertexId=vertex_1649344918728_0001_33_01, diagnostics=[Vertex received Kill while in RUNNING state., Vertex did not succeed due to OTHER_VERTEX_FAILURE, failedTasks:0 killedTasks:1, Vertex vertex_1649344918728_0001_33_01 [Reducer 2] killed/failed due to:OTHER_VERTEX_FAILURE]DAG did not succeed due to VERTEX_FAILURE. failedVertices:1 killedVertices:1 at org.apache.hadoop.hive.ql.exec.tez.TezTask.execute(TezTask.java:260) at org.apache.hadoop.hive.ql.exec.Task.executeTask(Task.java:212) at org.apache.hadoop.hive.ql.exec.TaskRunner.runSequential(TaskRunner.java:105) at org.apache.hadoop.hive.ql.Executor.launchTask(Executor.java:361) at org.apache.hadoop.hive.ql.Executor.launchTasks(Executor.java:334) at org.apache.hadoop.hive.ql.Executor.runTasks(Executor.java:245) at org.apache.hadoop.hive.ql.Executor.execute(Executor.java:106) at org.apache.hadoop.hive.ql.Driver.execute(Driver.java:348) at org.apache.hadoop.hive.ql.Driver.runInternal(Driver.java:204) at org.apache.hadoop.hive.ql.Driver.run(Driver.java:153) at org.apache.hadoop.hive.ql.Driver.run(Driver.java:148) at org.apache.hadoop.hive.ql.reexec.ReExecDriver.run(ReExecDriver.java:185) at org.apache.hadoop.hive.ql.reexec.ReExecDriver.run(ReExecDriver.java:228) at org.apache.hadoop.hive.cli.CliDriver.processLocalCmd(CliDriver.java:255) at org.apache.hadoop.hive.cli.CliDriver.processCmd1(CliDriver.java:200) at org.apache.hadoop.hive.cli.CliDriver.processCmd(CliDriver.java:126) at org.apache.hadoop.hive.cli.CliDriver.processLine(CliDriver.java:421) at org.apache.hadoop.hive.cli.CliDriver.processLine(CliDriver.java:352) at org.apache.hadoop.hive.ql.QTestUtil.executeClientInternal(QTestUtil.java:727) at org.apache.hadoop.hive.ql.QTestUtil.executeClient(QTestUtil.java:697) at org.apache.hadoop.hive.cli.control.CoreCliDriver.runTest(CoreCliDriver.java:114) at org.apache.hadoop.hive.cli.control.CliAdapter.runTest(CliAdapter.java:157) at org.apache.hadoop.hive.cli.TestMysqlMetastoreCliDriver.testCliDriver(TestMysqlMetastoreCliDriver.java:73) 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:59) at org.junit.internal.runners.model.ReflectiveCallable.run(ReflectiveCallable.java:12) at org.junit.runners.model.FrameworkMethod.invokeExplosively(FrameworkMethod.java:56) at org.junit.internal.runners.statements.InvokeMethod.evaluate(InvokeMethod.java:17) at org.apache.hadoop.hive.cli.TestMysqlMetastoreCliDriver$1.evaluate(TestMysqlMetastoreCliDriver.java:56) at org.junit.runners.ParentRunner$3.evaluate(ParentRunner.java:306) at org.junit.runners.BlockJUnit4ClassRunner$1.evaluate(BlockJUnit4ClassRunner.java:100) at org.junit.runners.ParentRunner.runLeaf(ParentRunner.java:366) at org.junit.runners.BlockJUnit4ClassRunner.runChild(BlockJUnit4ClassRunner.java:103) at org.junit.runners.BlockJUnit4ClassRunner.runChild(BlockJUnit4ClassRunner.java:63) at org.junit.runners.ParentRunner$4.run(ParentRunner.java:331) at org.junit.runners.ParentRunner$1.schedule(ParentRunner.java:79) at org.junit.runners.ParentRunner.runChildren(ParentRunner.java:329) at org.junit.runners.ParentRunner.access$100(ParentRunner.java:66) at org.junit.runners.ParentRunner$2.evaluate(ParentRunner.java:293) at org.junit.runners.ParentRunner.run(ParentRunner.java:413) at org.junit.runners.Suite.runChild(Suite.java:128) at org.junit.runners.Suite.runChild(Suite.java:27) at org.junit.runners.ParentRunner$4.run(ParentRunner.java:331) at org.junit.runners.ParentRunner$1.schedule(ParentRunner.java:79) at org.junit.runners.ParentRunner.runChildren(ParentRunner.java:329) at org.junit.runners.ParentRunner.access$100(ParentRunner.java:66) at org.junit.runners.ParentRunner$2.evaluate(ParentRunner.java:293) at org.apache.hadoop.hive.cli.control.CliAdapter$1$1.evaluate(CliAdapter.java:95) at org.junit.rules.RunRules.evaluate(RunRules.java:20) at org.junit.runners.ParentRunner$3.evaluate(ParentRunner.java:306) at org.junit.runners.ParentRunner.run(ParentRunner.java:413) 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.runSuitesInProcess(ForkedBooter.java:377) at org.apache.maven.surefire.booter.ForkedBooter.execute(ForkedBooter.java:138) at org.apache.maven.surefire.booter.ForkedBooter.run(ForkedBooter.java:465) at org.apache.maven.surefire.booter.ForkedBooter.main(ForkedBooter.java:451) at org.junit.Assert.fail(Assert.java:89) at org.apache.hadoop.hive.ql.QTestUtil.failedQuery(QTestUtil.java:1008) at org.apache.hadoop.hive.cli.control.CoreCliDriver.runTest(CoreCliDriver.java:117) at org.apache.hadoop.hive.cli.control.CliAdapter.runTest(CliAdapter.java:157) at org.apache.hadoop.hive.cli.TestMysqlMetastoreCliDriver.testCliDriver(TestMysqlMetastoreCliDriver.java:73) 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:59) at org.junit.internal.runners.model.ReflectiveCallable.run(ReflectiveCallable.java:12) at org.junit.runners.model.FrameworkMethod.invokeExplosively(FrameworkMethod.java:56) at org.junit.internal.runners.statements.InvokeMethod.evaluate(InvokeMethod.java:17) at org.apache.hadoop.hive.cli.TestMysqlMetastoreCliDriver$1.evaluate(TestMysqlMetastoreCliDriver.java:56) at org.junit.runners.ParentRunner$3.evaluate(ParentRunner.java:306) at org.junit.runners.BlockJUnit4ClassRunner$1.evaluate(BlockJUnit4ClassRunner.java:100) at org.junit.runners.ParentRunner.runLeaf(ParentRunner.java:366) at org.junit.runners.BlockJUnit4ClassRunner.runChild(BlockJUnit4ClassRunner.java:103) at org.junit.runners.BlockJUnit4ClassRunner.runChild(BlockJUnit4ClassRunner.java:63) at org.junit.runners.ParentRunner$4.run(ParentRunner.java:331) at org.junit.runners.ParentRunner$1.schedule(ParentRunner.java:79) at org.junit.runners.ParentRunner.runChildren(ParentRunner.java:329) at org.junit.runners.ParentRunner.access$100(ParentRunner.java:66) at org.junit.runners.ParentRunner$2.evaluate(ParentRunner.java:293) at org.junit.runners.ParentRunner.run(ParentRunner.java:413) at org.junit.runners.Suite.runChild(Suite.java:128) at org.junit.runners.Suite.runChild(Suite.java:27) at org.junit.runners.ParentRunner$4.run(ParentRunner.java:331) at org.junit.runners.ParentRunner$1.schedule(ParentRunner.java:79) at org.junit.runners.ParentRunner.runChildren(ParentRunner.java:329) at org.junit.runners.ParentRunner.access$100(ParentRunner.java:66) at org.junit.runners.ParentRunner$2.evaluate(ParentRunner.java:293) at org.apache.hadoop.hive.cli.control.CliAdapter$1$1.evaluate(CliAdapter.java:95) at org.junit.rules.RunRules.evaluate(RunRules.java:20) at org.junit.runners.ParentRunner$3.evaluate(ParentRunner.java:306) at org.junit.runners.ParentRunner.run(ParentRunner.java:413) 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.runSuitesInProcess(ForkedBooter.java:377) at org.apache.maven.surefire.booter.ForkedBooter.execute(ForkedBooter.java:138) at org.apache.maven.surefire.booter.ForkedBooter.run(ForkedBooter.java:465) at org.apache.maven.surefire.booter.ForkedBooter.main(ForkedBooter.java:451) org.apache.hadoop.hive.cli.TestMysqlMetastoreCliDriver.testCliDriver[sysdb] Time elapsed: 41.35 s <<< FAILURE! java.lang.AssertionError: Client execution failed with error code = 2 running select bucket_col_name, integer_idx from bucketing_cols order by bucket_col_name, integer_idx limit 5 fname=sysdb.q See ./ql/target/tmp/log/hive.log or ./itests/qtest/target/tmp/log/hive.log, or check ./ql/target/surefire-reports or ./itests/qtest/target/surefire-reports/ for specific test cases logs. org.apache.hadoop.hive.ql.metadata.HiveException: Vertex failed, vertexName=Map 1, vertexId=vertex_1649344918728_0001_34_00, diagnostics=[Task failed, taskId=task_1649344918728_0001_34_00_000000, diagnostics=[TaskAttempt 0 failed, info=[Error: Error while running task ( failure ) : attempt_1649344918728_0001_34_00_000000_0:java.lang.RuntimeException: org.apache.hadoop.hive.ql.metadata.HiveException: java.io.IOException: java.io.IOException: org.apache.hive.storage.jdbc.exception.HiveJdbcDatabaseAccessException: Caught exception while trying to execute query:You have an error in your SQL syntax; check the manual that corresponds to your MySQL server version for the right syntax to use near '"BUCKETING_COLS"' at line 6 at org.apache.hadoop.hive.ql.exec.tez.TezProcessor.initializeAndRunProcessor(TezProcessor.java:348) at org.apache.hadoop.hive.ql.exec.tez.TezProcessor.run(TezProcessor.java:276) at org.apache.tez.runtime.LogicalIOProcessorRuntimeTask.run(LogicalIOProcessorRuntimeTask.java:381) at org.apache.tez.runtime.task.TaskRunner2Callable$1.run(TaskRunner2Callable.java:82) at org.apache.tez.runtime.task.TaskRunner2Callable$1.run(TaskRunner2Callable.java:69) at java.security.AccessController.doPrivileged(Native Method) at javax.security.auth.Subject.doAs(Subject.java:422) at org.apache.hadoop.security.UserGroupInformation.doAs(UserGroupInformation.java:1682) at org.apache.tez.runtime.task.TaskRunner2Callable.callInternal(TaskRunner2Callable.java:69) at org.apache.tez.runtime.task.TaskRunner2Callable.callInternal(TaskRunner2Callable.java:39) at org.apache.tez.common.CallableWithNdc.call(CallableWithNdc.java:36) at org.apache.hadoop.hive.llap.daemon.impl.StatsRecordingThreadPool$WrappedCallable.call(StatsRecordingThreadPool.java:118) at java.util.concurrent.FutureTask.run(FutureTask.java:266) at java.util.concurrent.ThreadPoolExecutor.runWorker(ThreadPoolExecutor.java:1149) at java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:624) at java.lang.Thread.run(Thread.java:748) Caused by: org.apache.hadoop.hive.ql.metadata.HiveException: java.io.IOException: java.io.IOException: org.apache.hive.storage.jdbc.exception.HiveJdbcDatabaseAccessException: Caught exception while trying to execute query:You have an error in your SQL syntax; check the manual that corresponds to your MySQL server version for the right syntax to use near '"BUCKETING_COLS"' at line 6 at org.apache.hadoop.hive.ql.exec.tez.MapRecordSource.pushRecord(MapRecordSource.java:89) at org.apache.hadoop.hive.ql.exec.tez.MapRecordProcessor.run(MapRecordProcessor.java:414) at org.apache.hadoop.hive.ql.exec.tez.TezProcessor.initializeAndRunProcessor(TezProcessor.java:293) ... 15 more Caused by: java.io.IOException: java.io.IOException: org.apache.hive.storage.jdbc.exception.HiveJdbcDatabaseAccessException: Caught exception while trying to execute query:You have an error in your SQL syntax; check the manual that corresponds to your MySQL server version for the right syntax to use near '"BUCKETING_COLS"' at line 6 at org.apache.hadoop.hive.io.HiveIOExceptionHandlerChain.handleRecordReaderNextException(HiveIOExceptionHandlerChain.java:121) at org.apache.hadoop.hive.io.HiveIOExceptionHandlerUtil.handleRecordReaderNextException(HiveIOExceptionHandlerUtil.java:77) at org.apache.hadoop.hive.ql.io.HiveContextAwareRecordReader.doNext(HiveContextAwareRecordReader.java:380) at org.apache.hadoop.hive.ql.io.HiveRecordReader.doNext(HiveRecordReader.java:82) at org.apache.hadoop.hive.ql.io.HiveContextAwareRecordReader.next(HiveContextAwareRecordReader.java:119) at org.apache.hadoop.hive.ql.io.HiveContextAwareRecordReader.next(HiveContextAwareRecordReader.java:59) at org.apache.hadoop.mapred.split.TezGroupedSplitsInputFormat$TezGroupedSplitsRecordReader.next(TezGroupedSplitsInputFormat.java:145) at org.apache.tez.mapreduce.lib.MRReaderMapred.next(MRReaderMapred.java:116) at org.apache.hadoop.hive.ql.exec.tez.MapRecordSource.pushRecord(MapRecordSource.java:75) ... 17 more Caused by: java.io.IOException: org.apache.hive.storage.jdbc.exception.HiveJdbcDatabaseAccessException: Caught exception while trying to execute query:You have an error in your SQL syntax; check the manual that corresponds to your MySQL server version for the right syntax to use near '"BUCKETING_COLS"' at line 6 at org.apache.hive.storage.jdbc.JdbcRecordReader.next(JdbcRecordReader.java:85) at org.apache.hive.storage.jdbc.JdbcRecordReader.next(JdbcRecordReader.java:35) at org.apache.hadoop.hive.ql.io.HiveContextAwareRecordReader.doNext(HiveContextAwareRecordReader.java:375) ... 23 more Caused by: org.apache.hive.storage.jdbc.exception.HiveJdbcDatabaseAccessException: Caught exception while trying to execute query:You have an error in your SQL syntax; check the manual that corresponds to your MySQL server version for the right syntax to use near '"BUCKETING_COLS"' at line 6 at org.apache.hive.storage.jdbc.dao.GenericJdbcDatabaseAccessor.getRecordIterator(GenericJdbcDatabaseAccessor.java:187) at org.apache.hive.storage.jdbc.JdbcRecordReader.next(JdbcRecordReader.java:58) ... 25 more Caused by: java.sql.SQLSyntaxErrorException: You have an error in your SQL syntax; check the manual that corresponds to your MySQL server version for the right syntax to use near '"BUCKETING_COLS"' at line 6 at com.mysql.cj.jdbc.exceptions.SQLError.createSQLException(SQLError.java:120) at com.mysql.cj.jdbc.exceptions.SQLExceptionsMapping.translateException(SQLExceptionsMapping.java:122) at com.mysql.cj.jdbc.ClientPreparedStatement.executeInternal(ClientPreparedStatement.java:953) at com.mysql.cj.jdbc.ClientPreparedStatement.executeQuery(ClientPreparedStatement.java:1009) at org.apache.commons.dbcp2.DelegatingPreparedStatement.executeQuery(DelegatingPreparedStatement.java:122) at org.apache.commons.dbcp2.DelegatingPreparedStatement.executeQuery(DelegatingPreparedStatement.java:122) at org.apache.hive.storage.jdbc.dao.GenericJdbcDatabaseAccessor.getRecordIterator(GenericJdbcDatabaseAccessor.java:180) ... 26 more ], TaskAttempt 1 failed, info=[Error: Error while running task ( failure ) : attempt_1649344918728_0001_34_00_000000_1:java.lang.RuntimeException: org.apache.hadoop.hive.ql.metadata.HiveException: java.io.IOException: java.io.IOException: org.apache.hive.storage.jdbc.exception.HiveJdbcDatabaseAccessException: Caught exception while trying to execute query:You have an error in your SQL syntax; check the manual that corresponds to your MySQL server version for the right syntax to use near '"BUCKETING_COLS"' at line 6 at org.apache.hadoop.hive.ql.exec.tez.TezProcessor.initializeAndRunProcessor(TezProcessor.java:348) at org.apache.hadoop.hive.ql.exec.tez.TezProcessor.run(TezProcessor.java:276) at org.apache.tez.runtime.LogicalIOProcessorRuntimeTask.run(LogicalIOProcessorRuntimeTask.java:381) at org.apache.tez.runtime.task.TaskRunner2Callable$1.run(TaskRunner2Callable.java:82) at org.apache.tez.runtime.task.TaskRunner2Callable$1.run(TaskRunner2Callable.java:69) at java.security.AccessController.doPrivileged(Native Method) at javax.security.auth.Subject.doAs(Subject.java:422) at org.apache.hadoop.security.UserGroupInformation.doAs(UserGroupInformation.java:1682) at org.apache.tez.runtime.task.TaskRunner2Callable.callInternal(TaskRunner2Callable.java:69) at org.apache.tez.runtime.task.TaskRunner2Callable.callInternal(TaskRunner2Callable.java:39) at org.apache.tez.common.CallableWithNdc.call(CallableWithNdc.java:36) at org.apache.hadoop.hive.llap.daemon.impl.StatsRecordingThreadPool$WrappedCallable.call(StatsRecordingThreadPool.java:118) at java.util.concurrent.FutureTask.run(FutureTask.java:266) at java.util.concurrent.ThreadPoolExecutor.runWorker(ThreadPoolExecutor.java:1149) at java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:624) at java.lang.Thread.run(Thread.java:748) Caused by: org.apache.hadoop.hive.ql.metadata.HiveException: java.io.IOException: java.io.IOException: org.apache.hive.storage.jdbc.exception.HiveJdbcDatabaseAccessException: Caught exception while trying to execute query:You have an error in your SQL syntax; check the manual that corresponds to your MySQL server version for the right syntax to use near '"BUCKETING_COLS"' at line 6 at org.apache.hadoop.hive.ql.exec.tez.MapRecordSource.pushRecord(MapRecordSource.java:89) at org.apache.hadoop.hive.ql.exec.tez.MapRecordProcessor.run(MapRecordProcessor.java:414) at org.apache.hadoop.hive.ql.exec.tez.TezProcessor.initializeAndRunProcessor(TezProcessor.java:293) ... 15 more Caused by: java.io.IOException: java.io.IOException: org.apache.hive.storage.jdbc.exception.HiveJdbcDatabaseAccessException: Caught exception while trying to execute query:You have an error in your SQL syntax; check the manual that corresponds to your MySQL server version for the right syntax to use near '"BUCKETING_COLS"' at line 6 at org.apache.hadoop.hive.io.HiveIOExceptionHandlerChain.handleRecordReaderNextException(HiveIOExceptionHandlerChain.java:121) at org.apache.hadoop.hive.io.HiveIOExceptionHandlerUtil.handleRecordReaderNextException(HiveIOExceptionHandlerUtil.java:77) at org.apache.hadoop.hive.ql.io.HiveContextAwareRecordReader.doNext(HiveContextAwareRecordReader.java:380) at org.apache.hadoop.hive.ql.io.HiveRecordReader.doNext(HiveRecordReader.java:82) at org.apache.hadoop.hive.ql.io.HiveContextAwareRecordReader.next(HiveContextAwareRecordReader.java:119) at org.apache.hadoop.hive.ql.io.HiveContextAwareRecordReader.next(HiveContextAwareRecordReader.java:59) at org.apache.hadoop.mapred.split.TezGroupedSplitsInputFormat$TezGroupedSplitsRecordReader.next(TezGroupedSplitsInputFormat.java:145) at org.apache.tez.mapreduce.lib.MRReaderMapred.next(MRReaderMapred.java:116) at org.apache.hadoop.hive.ql.exec.tez.MapRecordSource.pushRecord(MapRecordSource.java:75) ... 17 more Caused by: java.io.IOException: org.apache.hive.storage.jdbc.exception.HiveJdbcDatabaseAccessException: Caught exception while trying to execute query:You have an error in your SQL syntax; check the manual that corresponds to your MySQL server version for the right syntax to use near '"BUCKETING_COLS"' at line 6 at org.apache.hive.storage.jdbc.JdbcRecordReader.next(JdbcRecordReader.java:85) at org.apache.hive.storage.jdbc.JdbcRecordReader.next(JdbcRecordReader.java:35) at org.apache.hadoop.hive.ql.io.HiveContextAwareRecordReader.doNext(HiveContextAwareRecordReader.java:375) ... 23 more Caused by: org.apache.hive.storage.jdbc.exception.HiveJdbcDatabaseAccessException: Caught exception while trying to execute query:You have an error in your SQL syntax; check the manual that corresponds to your MySQL server version for the right syntax to use near '"BUCKETING_COLS"' at line 6 at org.apache.hive.storage.jdbc.dao.GenericJdbcDatabaseAccessor.getRecordIterator(GenericJdbcDatabaseAccessor.java:187) at org.apache.hive.storage.jdbc.JdbcRecordReader.next(JdbcRecordReader.java:58) ... 25 more Caused by: java.sql.SQLSyntaxErrorException: You have an error in your SQL syntax; check the manual that corresponds to your MySQL server version for the right syntax to use near '"BUCKETING_COLS"' at line 6 at com.mysql.cj.jdbc.exceptions.SQLError.createSQLException(SQLError.java:120) at com.mysql.cj.jdbc.exceptions.SQLExceptionsMapping.translateException(SQLExceptionsMapping.java:122) at com.mysql.cj.jdbc.ClientPreparedStatement.executeInternal(ClientPreparedStatement.java:953) at com.mysql.cj.jdbc.ClientPreparedStatement.executeQuery(ClientPreparedStatement.java:1009) at org.apache.commons.dbcp2.DelegatingPreparedStatement.executeQuery(DelegatingPreparedStatement.java:122) at org.apache.commons.dbcp2.DelegatingPreparedStatement.executeQuery(DelegatingPreparedStatement.java:122) at org.apache.hive.storage.jdbc.dao.GenericJdbcDatabaseAccessor.getRecordIterator(GenericJdbcDatabaseAccessor.java:180) ... 26 more ]], Vertex did not succeed due to OWN_TASK_FAILURE, failedTasks:1 killedTasks:0, Vertex vertex_1649344918728_0001_34_00 [Map 1] killed/failed due to:OWN_TASK_FAILURE]Vertex killed, vertexName=Reducer 2, vertexId=vertex_1649344918728_0001_34_01, diagnostics=[Vertex received Kill while in RUNNING state., Vertex did not succeed due to OTHER_VERTEX_FAILURE, failedTasks:0 killedTasks:1, Vertex vertex_1649344918728_0001_34_01 [Reducer 2] killed/failed due to:OTHER_VERTEX_FAILURE]DAG did not succeed due to VERTEX_FAILURE. failedVertices:1 killedVertices:1 at org.apache.hadoop.hive.ql.exec.tez.TezTask.execute(TezTask.java:260) at org.apache.hadoop.hive.ql.exec.Task.executeTask(Task.java:212) at org.apache.hadoop.hive.ql.exec.TaskRunner.runSequential(TaskRunner.java:105) at org.apache.hadoop.hive.ql.Executor.launchTask(Executor.java:361) at org.apache.hadoop.hive.ql.Executor.launchTasks(Executor.java:334) at org.apache.hadoop.hive.ql.Executor.runTasks(Executor.java:245) at org.apache.hadoop.hive.ql.Executor.execute(Executor.java:106) at org.apache.hadoop.hive.ql.Driver.execute(Driver.java:348) at org.apache.hadoop.hive.ql.Driver.runInternal(Driver.java:204) at org.apache.hadoop.hive.ql.Driver.run(Driver.java:153) at org.apache.hadoop.hive.ql.Driver.run(Driver.java:148) at org.apache.hadoop.hive.ql.reexec.ReExecDriver.run(ReExecDriver.java:185) at org.apache.hadoop.hive.ql.reexec.ReExecDriver.run(ReExecDriver.java:228) at org.apache.hadoop.hive.cli.CliDriver.processLocalCmd(CliDriver.java:255) at org.apache.hadoop.hive.cli.CliDriver.processCmd1(CliDriver.java:200) at org.apache.hadoop.hive.cli.CliDriver.processCmd(CliDriver.java:126) at org.apache.hadoop.hive.cli.CliDriver.processLine(CliDriver.java:421) at org.apache.hadoop.hive.cli.CliDriver.processLine(CliDriver.java:352) at org.apache.hadoop.hive.ql.QTestUtil.executeClientInternal(QTestUtil.java:727) at org.apache.hadoop.hive.ql.QTestUtil.executeClient(QTestUtil.java:697) at org.apache.hadoop.hive.cli.control.CoreCliDriver.runTest(CoreCliDriver.java:114) at org.apache.hadoop.hive.cli.control.CliAdapter.runTest(CliAdapter.java:157) at org.apache.hadoop.hive.cli.TestMysqlMetastoreCliDriver.testCliDriver(TestMysqlMetastoreCliDriver.java:73) 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:59) at org.junit.internal.runners.model.ReflectiveCallable.run(ReflectiveCallable.java:12) at org.junit.runners.model.FrameworkMethod.invokeExplosively(FrameworkMethod.java:56) at org.junit.internal.runners.statements.InvokeMethod.evaluate(InvokeMethod.java:17) at org.apache.hadoop.hive.cli.TestMysqlMetastoreCliDriver$1.evaluate(TestMysqlMetastoreCliDriver.java:56) at org.junit.runners.ParentRunner$3.evaluate(ParentRunner.java:306) at org.junit.runners.BlockJUnit4ClassRunner$1.evaluate(BlockJUnit4ClassRunner.java:100) at org.junit.runners.ParentRunner.runLeaf(ParentRunner.java:366) at org.junit.runners.BlockJUnit4ClassRunner.runChild(BlockJUnit4ClassRunner.java:103) at org.junit.runners.BlockJUnit4ClassRunner.runChild(BlockJUnit4ClassRunner.java:63) at org.junit.runners.ParentRunner$4.run(ParentRunner.java:331) at org.junit.runners.ParentRunner$1.schedule(ParentRunner.java:79) at org.junit.runners.ParentRunner.runChildren(ParentRunner.java:329) at org.junit.runners.ParentRunner.access$100(ParentRunner.java:66) at org.junit.runners.ParentRunner$2.evaluate(ParentRunner.java:293) at org.junit.runners.ParentRunner.run(ParentRunner.java:413) at org.junit.runners.Suite.runChild(Suite.java:128) at org.junit.runners.Suite.runChild(Suite.java:27) at org.junit.runners.ParentRunner$4.run(ParentRunner.java:331) at org.junit.runners.ParentRunner$1.schedule(ParentRunner.java:79) at org.junit.runners.ParentRunner.runChildren(ParentRunner.java:329) at org.junit.runners.ParentRunner.access$100(ParentRunner.java:66) at org.junit.runners.ParentRunner$2.evaluate(ParentRunner.java:293) at org.apache.hadoop.hive.cli.control.CliAdapter$1$1.evaluate(CliAdapter.java:95) at org.junit.rules.RunRules.evaluate(RunRules.java:20) at org.junit.runners.ParentRunner$3.evaluate(ParentRunner.java:306) at org.junit.runners.ParentRunner.run(ParentRunner.java:413) 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.runSuitesInProcess(ForkedBooter.java:377) at org.apache.maven.surefire.booter.ForkedBooter.execute(ForkedBooter.java:138) at org.apache.maven.surefire.booter.ForkedBooter.run(ForkedBooter.java:465) at org.apache.maven.surefire.booter.ForkedBooter.main(ForkedBooter.java:451) at org.junit.Assert.fail(Assert.java:89) at org.apache.hadoop.hive.ql.QTestUtil.failedQuery(QTestUtil.java:1008) at org.apache.hadoop.hive.cli.control.CoreCliDriver.runTest(CoreCliDriver.java:117) at org.apache.hadoop.hive.cli.control.CliAdapter.runTest(CliAdapter.java:157) at org.apache.hadoop.hive.cli.TestMysqlMetastoreCliDriver.testCliDriver(TestMysqlMetastoreCliDriver.java:73) 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:59) at org.junit.internal.runners.model.ReflectiveCallable.run(ReflectiveCallable.java:12) at org.junit.runners.model.FrameworkMethod.invokeExplosively(FrameworkMethod.java:56) at org.junit.internal.runners.statements.InvokeMethod.evaluate(InvokeMethod.java:17) at org.apache.hadoop.hive.cli.TestMysqlMetastoreCliDriver$1.evaluate(TestMysqlMetastoreCliDriver.java:56) at org.junit.runners.ParentRunner$3.evaluate(ParentRunner.java:306) at org.junit.runners.BlockJUnit4ClassRunner$1.evaluate(BlockJUnit4ClassRunner.java:100) at org.junit.runners.ParentRunner.runLeaf(ParentRunner.java:366) at org.junit.runners.BlockJUnit4ClassRunner.runChild(BlockJUnit4ClassRunner.java:103) at org.junit.runners.BlockJUnit4ClassRunner.runChild(BlockJUnit4ClassRunner.java:63) at org.junit.runners.ParentRunner$4.run(ParentRunner.java:331) at org.junit.runners.ParentRunner$1.schedule(ParentRunner.java:79) at org.junit.runners.ParentRunner.runChildren(ParentRunner.java:329) at org.junit.runners.ParentRunner.access$100(ParentRunner.java:66) at org.junit.runners.ParentRunner$2.evaluate(ParentRunner.java:293) at org.junit.runners.ParentRunner.run(ParentRunner.java:413) at org.junit.runners.Suite.runChild(Suite.java:128) at org.junit.runners.Suite.runChild(Suite.java:27) at org.junit.runners.ParentRunner$4.run(ParentRunner.java:331) at org.junit.runners.ParentRunner$1.schedule(ParentRunner.java:79) at org.junit.runners.ParentRunner.runChildren(ParentRunner.java:329) at org.junit.runners.ParentRunner.access$100(ParentRunner.java:66) at org.junit.runners.ParentRunner$2.evaluate(ParentRunner.java:293) at org.apache.hadoop.hive.cli.control.CliAdapter$1$1.evaluate(CliAdapter.java:95) at org.junit.rules.RunRules.evaluate(RunRules.java:20) at org.junit.runners.ParentRunner$3.evaluate(ParentRunner.java:306) at org.junit.runners.ParentRunner.run(ParentRunner.java:413) 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.runSuitesInProcess(ForkedBooter.java:377) at org.apache.maven.surefire.booter.ForkedBooter.execute(ForkedBooter.java:138) at org.apache.maven.surefire.booter.ForkedBooter.run(ForkedBooter.java:465) at org.apache.maven.surefire.booter.ForkedBooter.main(ForkedBooter.java:451)
Attachments
Issue Links
- Discovered while testing
-
HIVE-26123 Introduce test coverage for sysdb for the different metastores
- In Progress