[ https://issues.apache.org/jira/browse/HIVE-21722?focusedWorklogId=241496&page=com.atlassian.jira.plugin.system.issuetabpanels:worklog-tabpanel#worklog-241496 ]
ASF GitHub Bot logged work on HIVE-21722: ----------------------------------------- Author: ASF GitHub Bot Created on: 14/May/19 04:15 Start Date: 14/May/19 04:15 Worklog Time Spent: 10m Work Description: maheshk114 commented on pull request #625: HIVE-21722 : REPL::END event log is not included in hiveStatement.getQueryLog output. URL: https://github.com/apache/hive/pull/625#discussion_r283620187 ########## File path: itests/hive-unit/src/test/java/org/apache/hive/jdbc/TestJdbcDriver2.java ########## @@ -2793,6 +2795,124 @@ public void run() { verifyFetchedLog(incrementalLogs, expectedLogs); } + private static int next = 0; + private synchronized void advanceDumpDir() { + next++; + ReplDumpWork.injectNextDumpDirForTest(String.valueOf(next)); + } + + /** + * Test getting query log method in Jdbc for REPL commands + * @throws Exception + */ + @Test + public void testGetQueryLogForReplCommands() throws Exception { + // Prepare + String primaryDb = "primaryDb"; + String replicaDb = "replicaDb"; + String primaryTblName = "primaryDb.t1"; + String sql = "select count(*) from " + tableName; + Path replDir = new Path(conf.get("test.data.files")); + replDir = new Path(replDir, "repl"); + FileSystem fs = FileSystem.get(replDir.toUri(), conf); + fs.mkdirs(replDir); + + // Prepare + HiveStatement stmt = (HiveStatement)con.createStatement(); + assertNotNull("Statement is null", stmt); + stmt.execute("set hive.exec.parallel = true"); + stmt.execute("set hive.server2.logging.operation.level = execution"); + stmt.execute("set hive.metastore.transactional.event.listeners = org.apache.hive.hcatalog.listener.DbNotificationListener"); + stmt.execute("set hive.metastore.dml.events = true"); + stmt.execute("create database " + primaryDb + " with dbproperties('repl.source.for'='1,2,3')"); + stmt.execute("create table " + primaryTblName + " (id int)"); + stmt.execute("insert into " + primaryTblName + " values (1), (2)"); + stmt.close(); + + // Test query logs for bootstrap dump and load + String[] expectedBootstrapDumpLogs = { + "REPL::START", + "REPL::TABLE_DUMP", + "REPL::END" + }; + + // Bootstrap dump + stmt = (HiveStatement)con.createStatement(); + advanceDumpDir(); + ResultSet replDumpRslt = stmt.executeQuery("repl dump " + primaryDb + + " with ('hive.repl.rootdir' = '" + replDir + "')"); + assertTrue(replDumpRslt.next()); + String dumpLocation = replDumpRslt.getString(1); + String lastReplId = replDumpRslt.getString(2); + List<String> logs = stmt.getQueryLog(false, 10000); + stmt.close(); + LOG.error("Query_Log for Bootstrap Dump"); + verifyFetchedLog(logs, expectedBootstrapDumpLogs); + + String[] expectedBootstrapLoadLogs = { + "REPL::START", + "REPL::TABLE_LOAD", + "REPL::END" + }; + + // Bootstrap load + stmt = (HiveStatement)con.createStatement(); + stmt.execute("repl load " + replicaDb + " from '" + dumpLocation + "'"); + logs = stmt.getQueryLog(false, 10000); + stmt.close(); + LOG.error("Query_Log for Bootstrap Load"); + verifyFetchedLog(logs, expectedBootstrapLoadLogs); + + // Perform operation for incremental replication + stmt = (HiveStatement)con.createStatement(); + stmt.execute("insert into " + primaryTblName + " values (3), (4)"); + stmt.close(); + + // Test query logs for incremental dump and load + String[] expectedIncrementalDumpLogs = { + "REPL::START", + "REPL::EVENT_DUMP", + "REPL::END" + }; + + // Incremental dump + stmt = (HiveStatement)con.createStatement(); + advanceDumpDir(); + replDumpRslt = stmt.executeQuery("repl dump " + primaryDb + " from " + lastReplId + + " with ('hive.repl.rootdir' = '" + replDir + "')"); + assertTrue(replDumpRslt.next()); + dumpLocation = replDumpRslt.getString(1); + lastReplId = replDumpRslt.getString(2); + logs = stmt.getQueryLog(false, 10000); + stmt.close(); + LOG.error("Query_Log for Incremental Dump"); + verifyFetchedLog(logs, expectedIncrementalDumpLogs); + + String[] expectedIncrementalLoadLogs = { + "REPL::START", + "REPL::EVENT_LOAD", + "REPL::END" + }; + + // Incremental load + stmt = (HiveStatement)con.createStatement(); + stmt.execute("repl load " + replicaDb + " from '" + dumpLocation + "'"); + logs = stmt.getQueryLog(false, 10000); + stmt.close(); + LOG.error("Query_Log for Incremental Load"); + verifyFetchedLog(logs, expectedIncrementalLoadLogs); + + // DB cleanup + stmt = (HiveStatement)con.createStatement(); Review comment: done in finally block as many settings are local to this test ---------------------------------------------------------------- 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: 241496) Time Spent: 1h 20m (was: 1h 10m) > REPL:: logs are missing in hiveStatement.getQueryLog output during parallel > execution mode. > ------------------------------------------------------------------------------------------- > > Key: HIVE-21722 > URL: https://issues.apache.org/jira/browse/HIVE-21722 > Project: Hive > Issue Type: Bug > Components: HiveServer2, repl > Affects Versions: 4.0.0 > Reporter: mahesh kumar behera > Assignee: mahesh kumar behera > Priority: Major > Labels: pull-request-available > Fix For: 4.0.0 > > Attachments: HIVE-21722.01.patch, HIVE-21722.02.patch > > Time Spent: 1h 20m > Remaining Estimate: 0h > > getQueryLog only reads logs from Background thread scope. If parallel > execution is set to true, a new thread is created for execution and all the > logs added by the new thread are not added to the parent Background thread > scope. In replication scope, replStateLogTasks are started in parallel mode > causing the logs to be skipped from getQueryLog scope. > There is one more issue, with the conf is not passed while creating > replStateLogTask during bootstrap load end. The same issue is there with > event load during incremental load. The incremental load end log task is > created with the proper config. -- This message was sent by Atlassian JIRA (v7.6.3#76005)