[ https://issues.apache.org/jira/browse/HIVE-25154?focusedWorklogId=603584&page=com.atlassian.jira.plugin.system.issuetabpanels:worklog-tabpanel#worklog-603584 ]
ASF GitHub Bot logged work on HIVE-25154: ----------------------------------------- Author: ASF GitHub Bot Created on: 28/May/21 15:51 Start Date: 28/May/21 15:51 Worklog Time Spent: 10m Work Description: pkumarsinha commented on a change in pull request #2311: URL: https://github.com/apache/hive/pull/2311#discussion_r640356534 ########## File path: standalone-metastore/metastore-server/src/main/java/org/apache/hadoop/hive/metastore/PartitionManagementTask.java ########## @@ -99,6 +99,15 @@ public boolean isTargetOfReplication(Database db) { return false; } + public static boolean isBeingFailovedOver(Database db) { Review comment: We can move this to some util class to avoid duplication ########## File path: standalone-metastore/metastore-server/src/main/java/org/apache/hadoop/hive/metastore/PartitionManagementTask.java ########## @@ -162,6 +164,10 @@ public void run() { setupMsckPathInvalidation(); Configuration msckConf = Msck.getMsckConf(conf); for (Table table : candidateTables) { + if (MetaStoreUtils.isDbBeingFailedOver(msc.getDatabase(table.getCatName(), table.getDbName()))) { Review comment: This is going to be costly. One HMS call per table. Can we maintain a cache? ########## File path: standalone-metastore/metastore-server/src/main/java/org/apache/hadoop/hive/metastore/PartitionManagementTask.java ########## @@ -99,6 +99,15 @@ public boolean isTargetOfReplication(Database db) { return false; } + public static boolean isBeingFailovedOver(Database db) { Review comment: nit: Typo ########## File path: standalone-metastore/metastore-server/src/test/java/org/apache/hadoop/hive/metastore/TestPartitionManagement.java ########## @@ -659,6 +712,45 @@ public void testNoPartitionRetentionForReplTarget() throws TException, Interrupt assertEquals(3, partitions.size()); } + @Test + public void testNoPartitionRetentionForFailoverDb() throws TException, InterruptedException { + String dbName = "db_failover"; + String tableName = "tbl_failover"; + Map<String, Column> colMap = buildAllColumns(); + List<String> partKeys = Lists.newArrayList("state", "dt"); + List<String> partKeyTypes = Lists.newArrayList("string", "date"); + List<List<String>> partVals = Lists.newArrayList( + Lists.newArrayList("__HIVE_DEFAULT_PARTITION__", "1990-01-01"), + Lists.newArrayList("CA", "1986-04-28"), + Lists.newArrayList("MN", "2018-11-31")); + // Check for the existence of partitions 10 seconds after the partition retention period has + // elapsed. Gives enough time for the partition retention task to work. + long partitionRetentionPeriodMs = 20000; + long waitingPeriodForTest = partitionRetentionPeriodMs + 10 * 1000; + createMetadata(DEFAULT_CATALOG_NAME, dbName, tableName, partKeys, partKeyTypes, partVals, colMap, false); + Table table = client.getTable(dbName, tableName); + List<Partition> partitions = client.listPartitions(dbName, tableName, (short) -1); + assertEquals(3, partitions.size()); + + table.getParameters().put(PartitionManagementTask.DISCOVER_PARTITIONS_TBLPROPERTY, "true"); + table.getParameters().put(PartitionManagementTask.PARTITION_RETENTION_PERIOD_TBLPROPERTY, + partitionRetentionPeriodMs + "ms"); + client.alter_table(dbName, tableName, table); + Database db = client.getDatabase(table.getDbName()); + db.putToParameters(ReplConst.REPL_FAILOVER_ENABLED, "true"); Review comment: May be we. can have two both cases covered, with and without ReplConst.REPL_FAILOVER_ENABLED in the same test. ########## File path: standalone-metastore/metastore-server/src/test/java/org/apache/hadoop/hive/metastore/TestPartitionManagement.java ########## @@ -620,6 +620,59 @@ public void testNoPartitionDiscoveryForReplTable() throws Exception { assertEquals(3, partitions.size()); } + @Test + public void testNoPartitionDiscoveryForFailoverDb() throws Exception { + String dbName = "db_failover"; + String tableName = "tbl_failover"; + Map<String, Column> colMap = buildAllColumns(); + List<String> partKeys = Lists.newArrayList("state", "dt"); + List<String> partKeyTypes = Lists.newArrayList("string", "date"); + List<List<String>> partVals = Lists.newArrayList( + Lists.newArrayList("__HIVE_DEFAULT_PARTITION__", "1990-01-01"), + Lists.newArrayList("CA", "1986-04-28"), + Lists.newArrayList("MN", "2018-11-31")); + createMetadata(DEFAULT_CATALOG_NAME, dbName, tableName, partKeys, partKeyTypes, partVals, colMap, false); + Table table = client.getTable(dbName, tableName); + List<Partition> partitions = client.listPartitions(dbName, tableName, (short) -1); + assertEquals(3, partitions.size()); + String tableLocation = table.getSd().getLocation(); + URI location = URI.create(tableLocation); + Path tablePath = new Path(location); + FileSystem fs = FileSystem.get(location, conf); + Path newPart1 = new Path(tablePath, "state=WA/dt=2018-12-01"); + Path newPart2 = new Path(tablePath, "state=UT/dt=2018-12-02"); + fs.mkdirs(newPart1); + fs.mkdirs(newPart2); + assertEquals(5, fs.listStatus(tablePath).length); + partitions = client.listPartitions(dbName, tableName, (short) -1); + assertEquals(3, partitions.size()); + + // table property is set to true, but the table is marked as replication target. The new + // partitions should not be created + table.getParameters().put(PartitionManagementTask.DISCOVER_PARTITIONS_TBLPROPERTY, "true"); + Database db = client.getDatabase(table.getDbName()); + db.putToParameters(ReplConst.REPL_FAILOVER_ENABLED, "true"); + client.alterDatabase(table.getDbName(), db); + client.alter_table(dbName, tableName, table); Review comment: Why is this needed? ########## File path: standalone-metastore/metastore-common/src/main/java/org/apache/hadoop/hive/metastore/utils/MetaStoreUtils.java ########## @@ -228,6 +230,15 @@ public static boolean isExternalTable(Table table) { return isExternal(params); } + public static boolean isDbBeingFailedOver(Database db) { + assert (db != null); + Map<String, String> dbParameters = db.getParameters(); + if ((dbParameters != null) && (dbParameters.containsKey(ReplConst.REPL_FAILOVER_ENABLED))) { + return !StringUtils.isEmpty(dbParameters.get(ReplConst.REPL_FAILOVER_ENABLED)); Review comment: What would the value of repl.failover.enabled be? Is it not 'true' during failover, if so, check for value being true. -- 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: 603584) Time Spent: 20m (was: 10m) > Disable StatsUpdaterThread and PartitionManagementTask for db that is being > failoved over. > ------------------------------------------------------------------------------------------ > > Key: HIVE-25154 > URL: https://issues.apache.org/jira/browse/HIVE-25154 > Project: Hive > Issue Type: Improvement > Reporter: Haymant Mangla > Assignee: Haymant Mangla > Priority: Major > Labels: pull-request-available > Time Spent: 20m > Remaining Estimate: 0h > -- This message was sent by Atlassian Jira (v8.3.4#803005)