You are viewing a plain text version of this content. The canonical link for it is here.
Posted to issues@hbase.apache.org by "Biju Nair (JIRA)" <ji...@apache.org> on 2019/05/12 21:49:00 UTC

[jira] [Updated] (HBASE-22293) region server OOM risk when doing replica replication

     [ https://issues.apache.org/jira/browse/HBASE-22293?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ]

Biju Nair updated HBASE-22293:
------------------------------
    Labels: read-replicas  (was: )

> region server OOM risk when doing replica replication
> -----------------------------------------------------
>
>                 Key: HBASE-22293
>                 URL: https://issues.apache.org/jira/browse/HBASE-22293
>             Project: HBase
>          Issue Type: Bug
>          Components: read replicas
>    Affects Versions: 2.2.0, 2.1.3
>            Reporter: Xinhui Xu
>            Priority: Major
>              Labels: read-replicas
>
> Trying region replica feature recently on version 2.1.3 and 2.2.0. And saw some of region servers were killed due to OOM after changing REGION_REPLICATION from 3 to 2 followed by some put operations. (the operations I performed are kind of similar with https://issues.apache.org/jira/browse/HBASE-20908)
> I looked into the code and had following findings (based on the code of version 2.2.0)
> 1. A WriterThread, which is responsible for writing per region wal entries to target replicas,  would just exit if it meets any exception and write its error to controller. One more thing to note, once an exception is written into the controller it will never be erased.
> {code:java}
> public void run()  {
>   try {
>     doRun();
>   } catch (Throwable t) {
>     LOG.error("Exiting thread", t);
>     controller.writerThreadError(t);
>   }
> }
> private void doRun() throws IOException {
>   LOG.trace("Writer thread starting");
>   while (true) {
>     RegionEntryBuffer buffer = entryBuffers.getChunkToWrite();
>     // ...
>     assert buffer != null;
>     try {
>       writeBuffer(buffer);
>     } finally {
>       entryBuffers.doneWriting(buffer);
>     }
>   }
> }
> {code}
> {code:java}
> public static class PipelineController {
>   AtomicReference<Throwable> thrown = new AtomicReference<>();
>   void writerThreadError(Throwable t) {
>     thrown.compareAndSet(null, t);
>   }
>   void checkForErrors() throws IOException {
>     Throwable thrown = this.thrown.get();
>     if (thrown == null) return;
>     if (thrown instanceof IOException) {
>       throw new IOException(thrown);
>     } else {
>       throw new RuntimeException(thrown);
>     }
>   }
> }
> {code}
> 2. For the replicate() logic in RegionReplicaReplicationEndpoint class, it will repeatedly append the same entry again if it meets any IOException.
> {code:java}
> public boolean replicate(ReplicateContext replicateContext) {
>   while (this.isRunning()) {
>     try {
>       for (Entry entry: replicateContext.getEntries()) {
>         entryBuffers.appendEntry(entry);
>       }
>       // ...
>     } catch (IOException e) {
>       LOG.warn("Received IOException while trying to replicate"
>           + StringUtils.stringifyException(e));
>     }
>   }
> {code}
> 3. While for the appendEntry() logic, it does aim to block the thread when the buffer already used exceed a pre-defined limit, but the buggy part is it's only valid when there is no error in the controller (controller.thrown.get() == null). In other words, once the controller has any error, the blocking logic will lose effect and it will eventually cause OOM.
> {code:java}
> public void appendEntry(Entry entry) throws InterruptedException, IOException {
>   WALKey key = entry.getKey();
>   RegionEntryBuffer buffer;
>   long incrHeap;
>   synchronized (this) {
>     buffer = buffers.get(key.getEncodedRegionName());
>     if (buffer == null) {
>       buffer = new RegionEntryBuffer(key.getTableName(), key.getEncodedRegionName());
>       buffers.put(key.getEncodedRegionName(), buffer);
>     }
>     incrHeap= buffer.appendEntry(entry);
>   }
>   synchronized (controller.dataAvailable) {
>     totalBuffered += incrHeap;
>     while (totalBuffered > maxHeapUsage && controller.thrown.get() == null) {
>       LOG.debug("Used {} bytes of buffered edits, waiting for IO threads", totalBuffered);
>       controller.dataAvailable.wait(2000);
>     }
>     controller.dataAvailable.notifyAll();
>   }
>   controller.checkForErrors();
> }{code}
> 4. The throwable is rooted in the implementation of RegionReplicaSinkWriter.append(). And most of them are region location errors as I see. (again like the one mentioned in https://issues.apache.org/jira/browse/HBASE-20908)
> Below are some of my inmature thoughts on how-to-fix: 
>  # EntryBuffers's appendEntry interface should be a real blocking interface regardless of any errors
>  # a WriterThread is the only consumer of EntryBuffers, it shouldn't exit when seeing any errors, as these errors might be transient and can be later retryed
>  # there is no need to write any error to the controller, except any DoNotRetryExceptions I am not aware of as of now
> just to arouse more discussions
>  



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