[ https://issues.apache.org/jira/browse/HBASE-26811?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ]
chenglei updated HBASE-26811: ----------------------------- Description: For read replica, when I set {{hbase.region.replica.wait.for.primary.flush}} to false, and set {{TableDescriptorBuilder.setRegionMemStoreReplication}} to true explicitly at table level, the secondary replica would be disabled for read forever, reading on this replica region would always throw : {code:java} java.io.IOException: The region's reads are disabled. Cannot serve the request at org.apache.hadoop.hbase.regionserver.HRegion.checkReadsEnabled(HRegion.java:5187) at org.apache.hadoop.hbase.regionserver.HRegion.startRegionOperation(HRegion.java:8279) {code} Very strange, if I don't {{TableDescriptorBuilder.setRegionMemStoreReplication}} to true explicitly (which default value is true), the secondary replica is normal. This problem is because when set {{hbase.region.replica.wait.for.primary.flush}} to false, the {{HRegionServer.startServices}} would not create the {{ExecutorType.RS_REGION_REPLICA_FLUSH_OPS}} for {{RegionReplicaFlushHandler}} at global level: {code:java} if (ServerRegionReplicaUtil.isRegionReplicaWaitForPrimaryFlushEnabled(conf)) { final int regionReplicaFlushThreads = conf.getInt( "hbase.regionserver.region.replica.flusher.threads", conf.getInt( "hbase.regionserver.executor.openregion.threads", 3)); executorService.startExecutorService(executorService.new ExecutorConfig().setExecutorType( ExecutorType.RS_REGION_REPLICA_FLUSH_OPS).setCorePoolSize(regionReplicaFlushThreads)); } {code} but when I set {{TableDescriptorBuilder.setRegionMemStoreReplication}} to true explicitly, it also set {{hbase.region.replica.wait.for.primary.flush}} to true at table-level(there is no public {{hbase.region.replica.wait.for.primary.flush} config for hbase user at table-level): {code:java} public ModifyableTableDescriptor setRegionMemStoreReplication(boolean memstoreReplication) { setValue(REGION_MEMSTORE_REPLICATION_KEY, Boolean.toString(memstoreReplication)); // If the memstore replication is setup, we do not have to wait for observing a flush event // from primary before starting to serve reads, because gaps from replication is not applicable return setValue(REGION_REPLICA_WAIT_FOR_PRIMARY_FLUSH_CONF_KEY, Boolean.toString(memstoreReplication)); } {code} So when the secondary replica region is open,{{HRegionServer.triggerFlushInPrimaryRegion}} is invoked for this region, because {{hbase.region.replica.wait.for.primary.flush}} to true at table-level, the line 2234 is skipped, secondary replica is disabled for read at line 2238, but there is no {{ExecutorType.RS_REGION_REPLICA_FLUSH_OPS}} for {{RegionReplicaFlushHandler}} at HRegionServer-level, so line 2243 would not schedule {{RegionReplicaFlushHandler}}, the secondary replica would be disabled for read forever. {code:java} 2227 private void triggerFlushInPrimaryRegion(final HRegion region) { ... 2232 if (!ServerRegionReplicaUtil.isRegionReplicaReplicationEnabled(region.conf, tn) || 2233 !ServerRegionReplicaUtil.isRegionReplicaWaitForPrimaryFlushEnabled(region.conf)) { 2234 region.setReadsEnabled(true); 2235 return; 2236 } 2237 2238 region.setReadsEnabled(false); // disable reads before marking the region as opened. 2239 // RegionReplicaFlushHandler might reset this. 2240 2241 // Submit it to be handled by one of the handlers so that we do not block OpenRegionHandler 2242 if (this.executorService != null) { 2243 this.executorService.submit(new RegionReplicaFlushHandler(this, region)); 2244 } else { ... } } {code} was: For read replica, when I set {{hbase.region.replica.wait.for.primary.flush}} to false, and set {{TableDescriptorBuilder.setRegionMemStoreReplication}} to true explicitly at table level, the secondary replica would be disabled for read forever, reading on this replica region would always throw : {code:java} java.io.IOException: The region's reads are disabled. Cannot serve the request at org.apache.hadoop.hbase.regionserver.HRegion.checkReadsEnabled(HRegion.java:5187) at org.apache.hadoop.hbase.regionserver.HRegion.startRegionOperation(HRegion.java:8279) {code} Very strange, if I don't {{TableDescriptorBuilder.setRegionMemStoreReplication}} to true explicitly (which default value is true), the secondary replica is normal. This problem is because when set {{hbase.region.replica.wait.for.primary.flush}} to false, the {{HRegionServer.startServices}} would not create the {{ExecutorType.RS_REGION_REPLICA_FLUSH_OPS}} for {{RegionReplicaFlushHandler}} at global level: {code:java} if (ServerRegionReplicaUtil.isRegionReplicaWaitForPrimaryFlushEnabled(conf)) { final int regionReplicaFlushThreads = conf.getInt( "hbase.regionserver.region.replica.flusher.threads", conf.getInt( "hbase.regionserver.executor.openregion.threads", 3)); executorService.startExecutorService(executorService.new ExecutorConfig().setExecutorType( ExecutorType.RS_REGION_REPLICA_FLUSH_OPS).setCorePoolSize(regionReplicaFlushThreads)); } {code:java} but when I set {{TableDescriptorBuilder.setRegionMemStoreReplication}} to true explicitly, it also set {{hbase.region.replica.wait.for.primary.flush}} to true at table-level(there is no public {{hbase.region.replica.wait.for.primary.flush} config for hbase user at table-level): {code:java} public ModifyableTableDescriptor setRegionMemStoreReplication(boolean memstoreReplication) { setValue(REGION_MEMSTORE_REPLICATION_KEY, Boolean.toString(memstoreReplication)); // If the memstore replication is setup, we do not have to wait for observing a flush event // from primary before starting to serve reads, because gaps from replication is not applicable return setValue(REGION_REPLICA_WAIT_FOR_PRIMARY_FLUSH_CONF_KEY, Boolean.toString(memstoreReplication)); } {code} So when the secondary replica region is open,{{HRegionServer.triggerFlushInPrimaryRegion}} is invoked for this region, because {{hbase.region.replica.wait.for.primary.flush}} to true at table-level, the line 2234 is skipped, secondary replica is disabled for read at line 2238, but there is no {{ExecutorType.RS_REGION_REPLICA_FLUSH_OPS}} for {{RegionReplicaFlushHandler}} at HRegionServer-level, so line 2243 would not schedule {{RegionReplicaFlushHandler}}, the secondary replica would be disabled for read forever. {code:java} 2227 private void triggerFlushInPrimaryRegion(final HRegion region) { ... 2232 if (!ServerRegionReplicaUtil.isRegionReplicaReplicationEnabled(region.conf, tn) || 2233 !ServerRegionReplicaUtil.isRegionReplicaWaitForPrimaryFlushEnabled(region.conf)) { 2234 region.setReadsEnabled(true); 2235 return; 2236 } 2237 2238 region.setReadsEnabled(false); // disable reads before marking the region as opened. 2239 // RegionReplicaFlushHandler might reset this. 2240 2241 // Submit it to be handled by one of the handlers so that we do not block OpenRegionHandler 2242 if (this.executorService != null) { 2243 this.executorService.submit(new RegionReplicaFlushHandler(this, region)); 2244 } else { ... } } {{code}} > Secondary replica may be disabled for read forever > -------------------------------------------------- > > Key: HBASE-26811 > URL: https://issues.apache.org/jira/browse/HBASE-26811 > Project: HBase > Issue Type: Bug > Components: read replicas > Affects Versions: 3.0.0-alpha-2, 2.4.10 > Reporter: chenglei > Priority: Major > > For read replica, when I set {{hbase.region.replica.wait.for.primary.flush}} > to false, and set {{TableDescriptorBuilder.setRegionMemStoreReplication}} to > true explicitly at table level, the secondary replica would be disabled for > read forever, reading on this replica region would always throw : > {code:java} > java.io.IOException: The region's reads are disabled. Cannot serve the > request > at > org.apache.hadoop.hbase.regionserver.HRegion.checkReadsEnabled(HRegion.java:5187) > at > org.apache.hadoop.hbase.regionserver.HRegion.startRegionOperation(HRegion.java:8279) > {code} > Very strange, if I don't > {{TableDescriptorBuilder.setRegionMemStoreReplication}} to true explicitly > (which default value is true), the secondary replica is normal. > This problem is because when set > {{hbase.region.replica.wait.for.primary.flush}} to false, the > {{HRegionServer.startServices}} would not create the > {{ExecutorType.RS_REGION_REPLICA_FLUSH_OPS}} for > {{RegionReplicaFlushHandler}} at global level: > {code:java} > if > (ServerRegionReplicaUtil.isRegionReplicaWaitForPrimaryFlushEnabled(conf)) { > final int regionReplicaFlushThreads = conf.getInt( > "hbase.regionserver.region.replica.flusher.threads", conf.getInt( > "hbase.regionserver.executor.openregion.threads", 3)); > executorService.startExecutorService(executorService.new > ExecutorConfig().setExecutorType( > > ExecutorType.RS_REGION_REPLICA_FLUSH_OPS).setCorePoolSize(regionReplicaFlushThreads)); > } > {code} > but when I set {{TableDescriptorBuilder.setRegionMemStoreReplication}} to > true explicitly, it also set {{hbase.region.replica.wait.for.primary.flush}} > to true at table-level(there is no public > {{hbase.region.replica.wait.for.primary.flush} config for hbase user at > table-level): > {code:java} > public ModifyableTableDescriptor setRegionMemStoreReplication(boolean > memstoreReplication) { > setValue(REGION_MEMSTORE_REPLICATION_KEY, > Boolean.toString(memstoreReplication)); > // If the memstore replication is setup, we do not have to wait for > observing a flush event > // from primary before starting to serve reads, because gaps from > replication is not applicable > return setValue(REGION_REPLICA_WAIT_FOR_PRIMARY_FLUSH_CONF_KEY, > Boolean.toString(memstoreReplication)); > } > {code} > So when the secondary replica region is > open,{{HRegionServer.triggerFlushInPrimaryRegion}} is invoked for this > region, because {{hbase.region.replica.wait.for.primary.flush}} to true at > table-level, the line 2234 is skipped, secondary replica is disabled for read > at line 2238, but there is no {{ExecutorType.RS_REGION_REPLICA_FLUSH_OPS}} > for {{RegionReplicaFlushHandler}} at HRegionServer-level, so line 2243 would > not schedule {{RegionReplicaFlushHandler}}, the secondary replica would be > disabled for read forever. > {code:java} > 2227 private void triggerFlushInPrimaryRegion(final HRegion region) { > ... > 2232 if > (!ServerRegionReplicaUtil.isRegionReplicaReplicationEnabled(region.conf, tn) > || > 2233 > !ServerRegionReplicaUtil.isRegionReplicaWaitForPrimaryFlushEnabled(region.conf)) > { > 2234 region.setReadsEnabled(true); > 2235 return; > 2236 } > 2237 > 2238 region.setReadsEnabled(false); // disable reads before marking the > region as opened. > 2239 // RegionReplicaFlushHandler might reset this. > 2240 > 2241 // Submit it to be handled by one of the handlers so that we do not > block OpenRegionHandler > 2242 if (this.executorService != null) { > 2243 this.executorService.submit(new RegionReplicaFlushHandler(this, > region)); > 2244 } else { > ... > } > } > {code} -- This message was sent by Atlassian Jira (v8.20.1#820001)