You are viewing a plain text version of this content. The canonical link for it is here.
Posted to user@hbase.apache.org by Kang Minwoo <mi...@outlook.com> on 2018/05/23 00:17:13 UTC

can not write to HBase

Hello, Users

My HBase client does not work after print the following logs.

Call exception, tries=23, retries=35, started=291277 ms ago, cancelled=false, msg=row '{row}' on table '{table}' at region={region}, hostname={hostname}, seqNum=100353531

There are no special logs in the Master and Region Servers.
Is it something wrong in the client?

Best regards,
Minwoo Kang

Re: can not write to HBase

Posted by Kang Minwoo <mi...@outlook.com>.
I left a comment on JIRA.

( https://issues.apache.org/jira/browse/HBASE-15871 )

Best regards,
Minwoo Kang

________________________________________
보낸 사람: Sean Busbey <bu...@apache.org>
보낸 날짜: 2018년 5월 29일 화요일 23:12
받는 사람: user@hbase.apache.org
제목: Re: can not write to HBase

On Tue, May 29, 2018 at 1:25 AM, Kang Minwoo <mi...@outlook.com> wrote:
> Yes, I use reverse scan at that time.
> The situation you have shared exactly matches our situation.
>
> Thank you for share Good material !!
>
> And I think HBASE-15871 should backport 1.2 branch.
>

Let's discuss suitability for backport of HBASE-15871 either on a
backport-specific JIRA or the dev@hbase mailing list.

(For context, I'm the release manager for the 1.2 release line.)

Re: can not write to HBase

Posted by Sean Busbey <bu...@apache.org>.
On Tue, May 29, 2018 at 1:25 AM, Kang Minwoo <mi...@outlook.com> wrote:
> Yes, I use reverse scan at that time.
> The situation you have shared exactly matches our situation.
>
> Thank you for share Good material !!
>
> And I think HBASE-15871 should backport 1.2 branch.
>

Let's discuss suitability for backport of HBASE-15871 either on a
backport-specific JIRA or the dev@hbase mailing list.

(For context, I'm the release manager for the 1.2 release line.)

Re: can not write to HBase

Posted by Kang Minwoo <mi...@outlook.com>.
Yes, I use reverse scan at that time.
The situation you have shared exactly matches our situation.

Thank you for share Good material !!

And I think HBASE-15871 should backport 1.2 branch.

----

(Yes, I am Korean.)

Best regards,
Minwoo Kang

________________________________________
보낸 사람: Jungdae Kim <kj...@gmail.com>
보낸 날짜: 2018년 5월 29일 화요일 11:21
받는 사람: user@hbase.apache.org
제목: Re: can not write to HBase

 Memstore flusher waits until all current store scanners'
operations(next()) are done.
I think you have to check why scaner.next() is too slow.

Does your applications(hbase client) use reverse scans?

If so, i think you are sufferiing the issue related to HBASE-15871(
https://issues.apache.org/jira/browse/HBASE-15871)
If you want more details about this issue or a memstore flusher, please
check the following link, notes written to Korean.(case #4: page 48 ~ 55)
(I think you are Korean. ^^)

 -
https://www.evernote.com/shard/s167/sh/39eb6b44-25e7-4e61-ad2a-a0d1b076c7d1/159db49e3e49b189

Best regards,
Jeongdae Kim


김정대 드림.


On Thu, May 24, 2018 at 1:22 PM, Kang Minwoo <mi...@outlook.com>
wrote:

> I have a same error on today.
> thread dump is here.
>
> ----------------
>
> Thread 286 (MemStoreFlusher.1):
>   State: WAITING
>   Blocked count: 10704
>   Waited count: 10936
>   Waiting on java.util.concurrent.locks.ReentrantLock$NonfairSync@2afc16fd
>   Stack:
>     sun.misc.Unsafe.park(Native Method)
>     java.util.concurrent.locks.LockSupport.park(LockSupport.java:175)
>     java.util.concurrent.locks.AbstractQueuedSynchronizer.
> parkAndCheckInterrupt(AbstractQueuedSynchronizer.java:836)
>     java.util.concurrent.locks.AbstractQueuedSynchronizer.acquireQueued(
> AbstractQueuedSynchronizer.java:870)
>     java.util.concurrent.locks.AbstractQueuedSynchronizer.acquire(
> AbstractQueuedSynchronizer.java:1199)
>     java.util.concurrent.locks.ReentrantLock$NonfairSync.
> lock(ReentrantLock.java:209)
>     java.util.concurrent.locks.ReentrantLock.lock(ReentrantLock.java:285)
>     org.apache.hadoop.hbase.regionserver.StoreScanner.
> updateReaders(StoreScanner.java:693)
>     org.apache.hadoop.hbase.regionserver.HStore.
> notifyChangedReadersObservers(HStore.java:1093)
>     org.apache.hadoop.hbase.regionserver.HStore.
> updateStorefiles(HStore.java:1072)
>     org.apache.hadoop.hbase.regionserver.HStore.access$
> 700(HStore.java:118)
>     org.apache.hadoop.hbase.regionserver.HStore$StoreFlusherImpl.commit(
> HStore.java:2310)
>     org.apache.hadoop.hbase.regionserver.HRegion.
> internalFlushCacheAndCommit(HRegion.java:2386)
>     org.apache.hadoop.hbase.regionserver.HRegion.
> internalFlushcache(HRegion.java:2108)
>     org.apache.hadoop.hbase.regionserver.HRegion.
> internalFlushcache(HRegion.java:2070)
>     org.apache.hadoop.hbase.regionserver.HRegion.
> flushcache(HRegion.java:1961)
>     org.apache.hadoop.hbase.regionserver.HRegion.flush(HRegion.java:1887)
>     org.apache.hadoop.hbase.regionserver.MemStoreFlusher.
> flushRegion(MemStoreFlusher.java:514)
>     org.apache.hadoop.hbase.regionserver.MemStoreFlusher.
> flushRegion(MemStoreFlusher.java:475)
>     org.apache.hadoop.hbase.regionserver.MemStoreFlusher.
> access$900(MemStoreFlusher.java:75)
>
>
> ----------------
>
> I deleted many row these days.
> I think that affected.
>
> Best regards,
> Minwoo Kang
>
> ________________________________________
> 보낸 사람: Kang Minwoo <mi...@outlook.com>
> 보낸 날짜: 2018년 5월 23일 수요일 16:53
> 받는 사람: Hbase-User
> 제목: Re: can not write to HBase
>
> In HRegion#internalFlushCacheAndCommit
> There is following code.
>
>     synchronized (this) {
>       notifyAll(); // FindBugs NN_NAKED_NOTIFY
>     }
>
> one question.
> Where is the lock acquired?
>
> Best regards,
> Minwoo Kang
>
> ________________________________________
> 보낸 사람: Kang Minwoo <mi...@outlook.com>
> 보낸 날짜: 2018년 5월 23일 수요일 16:37
> 받는 사람: Hbase-User
> 제목: Re: can not write to HBase
>
> Next time, If I have a same problem. I will save the jstack of the RS.
> (I could not think of saving jstack this time.)
>
> I did not see any special logs.
> There was only a warn log that the HBase Scan was slow.
>
> Best regards,
> Minwoo Kang
>
> ________________________________________
> 보낸 사람: Yu Li <ca...@gmail.com>
> 보낸 날짜: 2018년 5월 23일 수요일 15:53
> 받는 사람: Hbase-User
> 제목: Re: can not write to HBase
>
> Please save the jstack of the RS when slow flush is ongoing and confirm
> whether it stuck at the HDFS writing phase. If so, check log of the local
> DN co-located with the RegionServer to see whether any notice-able
> exceptions.
>
> Best Regards,
> Yu
>
> On 23 May 2018 at 14:19, Kang Minwoo <mi...@outlook.com> wrote:
>
> > @Duo Zhang
> > This means that you're writing too fast and memstore has reached its
> upper
> > limit. Is the flush and compaction fine at RS side?
> >
> > -> No, flush took very long time.
> > I attach code that took a long time to run. (about 30min)
> >
> > https://github.com/apache/hbase/blob/branch-1.2/hbase-
> > server/src/main/java/org/apache/hadoop/hbase/regionserver/HRegion.java#
> > L2424-L2508
> >
> > Best regards,
> > Minwoo Kang
> > ________________________________________
> > 보낸 사람: Kang Minwoo <mi...@outlook.com>
> > 보낸 날짜: 2018년 5월 23일 수요일 15:16
> > 받는 사람: user@hbase.apache.org
> > 제목: Re: can not write to HBase
> >
> > I am using salt, prevent write hotspot.
> > And Table has 4000 regions.
> >
> > HBase version is 1.2.6.
> >
> > I attach code that took a long time to run. (about 30min)
> > https://github.com/apache/hbase/blob/branch-1.2/hbase-
> > server/src/main/java/org/apache/hadoop/hbase/regionserver/HRegion.java#
> > L2424-L2508
> >
> > Best regards,
> > Minwoo Kang
> >
> > ________________________________________
> > 보낸 사람: Anil Gupta <an...@gmail.com>
> > 보낸 날짜: 2018년 5월 23일 수요일 14:13
> > 받는 사람: user@hbase.apache.org
> > 제목: Re: can not write to HBase
> >
> > It seems you might have a write hotspot.
> > Are your writes evenly distributed across the cluster? Do you have more
> > than 15-20 regions for that table?
> >
> > Sent from my iPhone
> >
> > &gt; On May 22, 2018, at 9:52 PM, Kang Minwoo <mi...@outlook.com>
> > wrote:
> > &gt;
> > &gt; I think hbase flush is too slow.
> > &gt; so memstore reached upper limit.
> > &gt;
> > &gt; flush took about 30min.
> > &gt; I don't know why flush is too long.
> > &gt;
> > &gt; Best regards,
> > &gt; Minwoo Kang
> > &gt;
> > &gt; ________________________________________
> > &gt; 보낸 사람: 张铎(Duo Zhang) <pa...@gmail.com>
> > &gt; 보낸 날짜: 2018년 5월 23일 수요일 11:37
> > &gt; 받는 사람: hbase-user
> > &gt; 제목: Re: can not write to HBase
> > &gt;
> > &gt; org.apache.hadoop.hbase.RegionTooBusyException:
> > &gt; org.apache.hadoop.hbase.RegionTooBusyException:
> > &gt; Above memstore limit, regionName={region}, server={server},
> > &gt; memstoreSize=2600502128, blockingMemStoreSize=2600468480
> > &gt;
> > &gt; This means that you're writing too fast and memstore has reached its
> > upper
> > &gt; limit. Is the flush and compaction fine at RS side?
> > &gt;
> > &gt; 2018-05-23 10:20 GMT+08:00 Kang Minwoo <mi...@outlook.com>:
> > &gt;
> > &gt;&gt; attach client exception and stacktrace.
> > &gt;&gt;
> > &gt;&gt; I've looked more.
> > &gt;&gt; It seems to be the reason why it takes 1290 seconds to flush in
> > the Region
> > &gt;&gt; Server.
> > &gt;&gt;
> > &gt;&gt; 2018-05-23T07:24:31.202 [INFO] Call exception, tries=34,
> > retries=35,
> > &gt;&gt; started=513393 ms ago, cancelled=false, msg=row '{row}' on table
> > '{table}'
> > &gt;&gt; at region={region}, hostname={host}, seqNum=155455658
> > &gt;&gt; 2018-05-23T07:24:31.208 [ERROR]
> > &gt;&gt; java.lang.RuntimeException: com.google.protobuf.
> ServiceException:
> > Error
> > &gt;&gt; calling method MultiRowMutationService.MutateRows
> > &gt;&gt;        at com.google.common.base.Throwables.propagate(
> > Throwables.java:160)
> > &gt;&gt; ~[stormjar.jar:?]
> > &gt;&gt;        at ...
> > &gt;&gt;        at org.apache.storm.daemon.executor$fn__8058$tuple_
> > &gt;&gt; action_fn__8060.invoke(executor.clj:731)
> > [storm-core-1.0.2.jar:1.0.2]
> > &gt;&gt;        at org.apache.storm.daemon.
> executor$mk_task_receiver$fn__
> > 7979.invoke(executor.clj:464)
> > &gt;&gt; [storm-core-1.0.2.jar:1.0.2]
> > &gt;&gt;        at org.apache.storm.disruptor$
> clojure_handler$reify__7492.
> > onEvent(disruptor.clj:40)
> > &gt;&gt; [storm-core-1.0.2.jar:1.0.2]
> > &gt;&gt;        at org.apache.storm.utils.DisruptorQueue.
> > consumeBatchToCursor(DisruptorQueue.java:451)
> > &gt;&gt; [storm-core-1.0.2.jar:1.0.2]
> > &gt;&gt;        at org.apache.storm.utils.DisruptorQueue.
> > &gt;&gt; consumeBatchWhenAvailable(DisruptorQueue.java:430)
> > &gt;&gt; [storm-core-1.0.2.jar:1.0.2]
> > &gt;&gt;        at org.apache.storm.disruptor$
> > consume_batch_when_available.invoke(disruptor.clj:73)
> > &gt;&gt; [storm-core-1.0.2.jar:1.0.2]
> > &gt;&gt;        at org.apache.storm.daemon.
> executor$fn__8058$fn__8071$fn_
> > _8124.invoke(executor.clj:850)
> > &gt;&gt; [storm-core-1.0.2.jar:1.0.2]
> > &gt;&gt;        at org.apache.storm.util$async_
> > loop$fn__624.invoke(util.clj:484)
> > &gt;&gt; [storm-core-1.0.2.jar:1.0.2]
> > &gt;&gt;        at clojure.lang.AFn.run(AFn.java:22)
> [clojure-1.7.0.jar:?]
> > &gt;&gt;        at java.lang.Thread.run(Thread.java:745) [?:1.7.0_80]
> > &gt;&gt; Caused by: com.google.protobuf.ServiceException: Error calling
> > method
> > &gt;&gt; MultiRowMutationService.MutateRows
> > &gt;&gt;        at org.apache.hadoop.hbase.ipc.CoprocessorRpcChannel.
> > &gt;&gt; callBlockingMethod(CoprocessorRpcChannel.java:75)
> > ~[stormjar.jar:?]
> > &gt;&gt;        at org.apache.hadoop.hbase.protobuf.generated.
> > &gt;&gt; MultiRowMutationProtos$MultiRowMutationService$
> > BlockingStub.mutateRows(
> > &gt;&gt; MultiRowMutationProtos.java:2149) ~[stormjar.jar:?]
> > &gt;&gt;        at ...
> > &gt;&gt;        ... 13 more
> > &gt;&gt; Caused by: org.apache.hadoop.hbase.client.
> > RetriesExhaustedException:
> > &gt;&gt; Failed after attempts=35, exceptions:
> > &gt;&gt; Wed May 23 07:15:57 KST 2018, RpcRetryingCaller{
> > globalStartTime=1527027357808,
> > &gt;&gt; pause=100, retries=35}, org.apache.hadoop.hbase.
> > RegionTooBusyException:
> > &gt;&gt; org.apache.hadoop.hbase.RegionTooBusyException: Above memstore
> > limit,
> > &gt;&gt; regionName={region}, server={server}, memstoreSize=2600502128,
> > &gt;&gt; blockingMemStoreSize=2600468480
> > &gt;&gt;        at org.apache.hadoop.hbase.regionserver.HRegion.
> > &gt;&gt; checkResources(HRegion.java:3649)
> > &gt;&gt;        at org.apache.hadoop.hbase.regionserver.HRegion.
> > &gt;&gt; processRowsWithLocks(HRegion.java:6935)
> > &gt;&gt;        at org.apache.hadoop.hbase.regionserver.HRegion.
> > &gt;&gt; mutateRowsWithLocks(HRegion.java:6885)
> > &gt;&gt;        at org.apache.hadoop.hbase.coprocessor.
> > MultiRowMutationEndpoint.
> > &gt;&gt; mutateRows(MultiRowMutationEndpoint.java:116)
> > &gt;&gt;        at org.apache.hadoop.hbase.protobuf.generated.
> > &gt;&gt; MultiRowMutationProtos$MultiRowMutationService.callMethod(
> > &gt;&gt; MultiRowMutationProtos.java:2053)
> > &gt;&gt;        at org.apache.hadoop.hbase.regionserver.HRegion.
> > &gt;&gt; execService(HRegion.java:7875)
> > &gt;&gt;        at org.apache.hadoop.hbase.regionserver.RSRpcServices.
> > &gt;&gt; execServiceOnRegion(RSRpcServices.java:2008)
> > &gt;&gt;
> > &gt;&gt;
> > &gt;&gt; Best regards,
> > &gt;&gt; Minwoo Kang
> > &gt;&gt;
> > &gt;&gt; ________________________________________
> > &gt;&gt; 보낸 사람: 张铎(Duo Zhang) <pa...@gmail.com>
> > &gt;&gt; 보낸 날짜: 2018년 5월 23일 수요일 09:22
> > &gt;&gt; 받는 사람: hbase-user
> > &gt;&gt; 제목: Re: can not write to HBase
> > &gt;&gt;
> > &gt;&gt; What is the exception? And the stacktrace?
> > &gt;&gt;
> > &gt;&gt; 2018-05-23 8:17 GMT+08:00 Kang Minwoo <minwoo.kang@outlook.com
> >:
> > &gt;&gt;
> > &gt;&gt;&gt; Hello, Users
> > &gt;&gt;&gt;
> > &gt;&gt;&gt; My HBase client does not work after print the following
> logs.
> > &gt;&gt;&gt;
> > &gt;&gt;&gt; Call exception, tries=23, retries=35, started=291277 ms ago,
> > &gt;&gt;&gt; cancelled=false, msg=row '{row}' on table '{table}' at
> > region={region},
> > &gt;&gt;&gt; hostname={hostname}, seqNum=100353531
> > &gt;&gt;&gt;
> > &gt;&gt;&gt; There are no special logs in the Master and Region Servers.
> > &gt;&gt;&gt; Is it something wrong in the client?
> > &gt;&gt;&gt;
> > &gt;&gt;&gt; Best regards,
> > &gt;&gt;&gt; Minwoo Kang
> > &gt;&gt;&gt;
> > &gt;&gt;
> > </m...@gmail.com></minwoo.
> > kang@outlook.com></p...@outlook.com></
> > anilgupta84@gmail.com></m...@outlook.com>
>

Re: can not write to HBase

Posted by Jungdae Kim <kj...@gmail.com>.
 Memstore flusher waits until all current store scanners'
operations(next()) are done.
I think you have to check why scaner.next() is too slow.

Does your applications(hbase client) use reverse scans?

If so, i think you are sufferiing the issue related to HBASE-15871(
https://issues.apache.org/jira/browse/HBASE-15871)
If you want more details about this issue or a memstore flusher, please
check the following link, notes written to Korean.(case #4: page 48 ~ 55)
(I think you are Korean. ^^)

 -
https://www.evernote.com/shard/s167/sh/39eb6b44-25e7-4e61-ad2a-a0d1b076c7d1/159db49e3e49b189

Best regards,
Jeongdae Kim


김정대 드림.


On Thu, May 24, 2018 at 1:22 PM, Kang Minwoo <mi...@outlook.com>
wrote:

> I have a same error on today.
> thread dump is here.
>
> ----------------
>
> Thread 286 (MemStoreFlusher.1):
>   State: WAITING
>   Blocked count: 10704
>   Waited count: 10936
>   Waiting on java.util.concurrent.locks.ReentrantLock$NonfairSync@2afc16fd
>   Stack:
>     sun.misc.Unsafe.park(Native Method)
>     java.util.concurrent.locks.LockSupport.park(LockSupport.java:175)
>     java.util.concurrent.locks.AbstractQueuedSynchronizer.
> parkAndCheckInterrupt(AbstractQueuedSynchronizer.java:836)
>     java.util.concurrent.locks.AbstractQueuedSynchronizer.acquireQueued(
> AbstractQueuedSynchronizer.java:870)
>     java.util.concurrent.locks.AbstractQueuedSynchronizer.acquire(
> AbstractQueuedSynchronizer.java:1199)
>     java.util.concurrent.locks.ReentrantLock$NonfairSync.
> lock(ReentrantLock.java:209)
>     java.util.concurrent.locks.ReentrantLock.lock(ReentrantLock.java:285)
>     org.apache.hadoop.hbase.regionserver.StoreScanner.
> updateReaders(StoreScanner.java:693)
>     org.apache.hadoop.hbase.regionserver.HStore.
> notifyChangedReadersObservers(HStore.java:1093)
>     org.apache.hadoop.hbase.regionserver.HStore.
> updateStorefiles(HStore.java:1072)
>     org.apache.hadoop.hbase.regionserver.HStore.access$
> 700(HStore.java:118)
>     org.apache.hadoop.hbase.regionserver.HStore$StoreFlusherImpl.commit(
> HStore.java:2310)
>     org.apache.hadoop.hbase.regionserver.HRegion.
> internalFlushCacheAndCommit(HRegion.java:2386)
>     org.apache.hadoop.hbase.regionserver.HRegion.
> internalFlushcache(HRegion.java:2108)
>     org.apache.hadoop.hbase.regionserver.HRegion.
> internalFlushcache(HRegion.java:2070)
>     org.apache.hadoop.hbase.regionserver.HRegion.
> flushcache(HRegion.java:1961)
>     org.apache.hadoop.hbase.regionserver.HRegion.flush(HRegion.java:1887)
>     org.apache.hadoop.hbase.regionserver.MemStoreFlusher.
> flushRegion(MemStoreFlusher.java:514)
>     org.apache.hadoop.hbase.regionserver.MemStoreFlusher.
> flushRegion(MemStoreFlusher.java:475)
>     org.apache.hadoop.hbase.regionserver.MemStoreFlusher.
> access$900(MemStoreFlusher.java:75)
>
>
> ----------------
>
> I deleted many row these days.
> I think that affected.
>
> Best regards,
> Minwoo Kang
>
> ________________________________________
> 보낸 사람: Kang Minwoo <mi...@outlook.com>
> 보낸 날짜: 2018년 5월 23일 수요일 16:53
> 받는 사람: Hbase-User
> 제목: Re: can not write to HBase
>
> In HRegion#internalFlushCacheAndCommit
> There is following code.
>
>     synchronized (this) {
>       notifyAll(); // FindBugs NN_NAKED_NOTIFY
>     }
>
> one question.
> Where is the lock acquired?
>
> Best regards,
> Minwoo Kang
>
> ________________________________________
> 보낸 사람: Kang Minwoo <mi...@outlook.com>
> 보낸 날짜: 2018년 5월 23일 수요일 16:37
> 받는 사람: Hbase-User
> 제목: Re: can not write to HBase
>
> Next time, If I have a same problem. I will save the jstack of the RS.
> (I could not think of saving jstack this time.)
>
> I did not see any special logs.
> There was only a warn log that the HBase Scan was slow.
>
> Best regards,
> Minwoo Kang
>
> ________________________________________
> 보낸 사람: Yu Li <ca...@gmail.com>
> 보낸 날짜: 2018년 5월 23일 수요일 15:53
> 받는 사람: Hbase-User
> 제목: Re: can not write to HBase
>
> Please save the jstack of the RS when slow flush is ongoing and confirm
> whether it stuck at the HDFS writing phase. If so, check log of the local
> DN co-located with the RegionServer to see whether any notice-able
> exceptions.
>
> Best Regards,
> Yu
>
> On 23 May 2018 at 14:19, Kang Minwoo <mi...@outlook.com> wrote:
>
> > @Duo Zhang
> > This means that you're writing too fast and memstore has reached its
> upper
> > limit. Is the flush and compaction fine at RS side?
> >
> > -> No, flush took very long time.
> > I attach code that took a long time to run. (about 30min)
> >
> > https://github.com/apache/hbase/blob/branch-1.2/hbase-
> > server/src/main/java/org/apache/hadoop/hbase/regionserver/HRegion.java#
> > L2424-L2508
> >
> > Best regards,
> > Minwoo Kang
> > ________________________________________
> > 보낸 사람: Kang Minwoo <mi...@outlook.com>
> > 보낸 날짜: 2018년 5월 23일 수요일 15:16
> > 받는 사람: user@hbase.apache.org
> > 제목: Re: can not write to HBase
> >
> > I am using salt, prevent write hotspot.
> > And Table has 4000 regions.
> >
> > HBase version is 1.2.6.
> >
> > I attach code that took a long time to run. (about 30min)
> > https://github.com/apache/hbase/blob/branch-1.2/hbase-
> > server/src/main/java/org/apache/hadoop/hbase/regionserver/HRegion.java#
> > L2424-L2508
> >
> > Best regards,
> > Minwoo Kang
> >
> > ________________________________________
> > 보낸 사람: Anil Gupta <an...@gmail.com>
> > 보낸 날짜: 2018년 5월 23일 수요일 14:13
> > 받는 사람: user@hbase.apache.org
> > 제목: Re: can not write to HBase
> >
> > It seems you might have a write hotspot.
> > Are your writes evenly distributed across the cluster? Do you have more
> > than 15-20 regions for that table?
> >
> > Sent from my iPhone
> >
> > &gt; On May 22, 2018, at 9:52 PM, Kang Minwoo <mi...@outlook.com>
> > wrote:
> > &gt;
> > &gt; I think hbase flush is too slow.
> > &gt; so memstore reached upper limit.
> > &gt;
> > &gt; flush took about 30min.
> > &gt; I don't know why flush is too long.
> > &gt;
> > &gt; Best regards,
> > &gt; Minwoo Kang
> > &gt;
> > &gt; ________________________________________
> > &gt; 보낸 사람: 张铎(Duo Zhang) <pa...@gmail.com>
> > &gt; 보낸 날짜: 2018년 5월 23일 수요일 11:37
> > &gt; 받는 사람: hbase-user
> > &gt; 제목: Re: can not write to HBase
> > &gt;
> > &gt; org.apache.hadoop.hbase.RegionTooBusyException:
> > &gt; org.apache.hadoop.hbase.RegionTooBusyException:
> > &gt; Above memstore limit, regionName={region}, server={server},
> > &gt; memstoreSize=2600502128, blockingMemStoreSize=2600468480
> > &gt;
> > &gt; This means that you're writing too fast and memstore has reached its
> > upper
> > &gt; limit. Is the flush and compaction fine at RS side?
> > &gt;
> > &gt; 2018-05-23 10:20 GMT+08:00 Kang Minwoo <mi...@outlook.com>:
> > &gt;
> > &gt;&gt; attach client exception and stacktrace.
> > &gt;&gt;
> > &gt;&gt; I've looked more.
> > &gt;&gt; It seems to be the reason why it takes 1290 seconds to flush in
> > the Region
> > &gt;&gt; Server.
> > &gt;&gt;
> > &gt;&gt; 2018-05-23T07:24:31.202 [INFO] Call exception, tries=34,
> > retries=35,
> > &gt;&gt; started=513393 ms ago, cancelled=false, msg=row '{row}' on table
> > '{table}'
> > &gt;&gt; at region={region}, hostname={host}, seqNum=155455658
> > &gt;&gt; 2018-05-23T07:24:31.208 [ERROR]
> > &gt;&gt; java.lang.RuntimeException: com.google.protobuf.
> ServiceException:
> > Error
> > &gt;&gt; calling method MultiRowMutationService.MutateRows
> > &gt;&gt;        at com.google.common.base.Throwables.propagate(
> > Throwables.java:160)
> > &gt;&gt; ~[stormjar.jar:?]
> > &gt;&gt;        at ...
> > &gt;&gt;        at org.apache.storm.daemon.executor$fn__8058$tuple_
> > &gt;&gt; action_fn__8060.invoke(executor.clj:731)
> > [storm-core-1.0.2.jar:1.0.2]
> > &gt;&gt;        at org.apache.storm.daemon.
> executor$mk_task_receiver$fn__
> > 7979.invoke(executor.clj:464)
> > &gt;&gt; [storm-core-1.0.2.jar:1.0.2]
> > &gt;&gt;        at org.apache.storm.disruptor$
> clojure_handler$reify__7492.
> > onEvent(disruptor.clj:40)
> > &gt;&gt; [storm-core-1.0.2.jar:1.0.2]
> > &gt;&gt;        at org.apache.storm.utils.DisruptorQueue.
> > consumeBatchToCursor(DisruptorQueue.java:451)
> > &gt;&gt; [storm-core-1.0.2.jar:1.0.2]
> > &gt;&gt;        at org.apache.storm.utils.DisruptorQueue.
> > &gt;&gt; consumeBatchWhenAvailable(DisruptorQueue.java:430)
> > &gt;&gt; [storm-core-1.0.2.jar:1.0.2]
> > &gt;&gt;        at org.apache.storm.disruptor$
> > consume_batch_when_available.invoke(disruptor.clj:73)
> > &gt;&gt; [storm-core-1.0.2.jar:1.0.2]
> > &gt;&gt;        at org.apache.storm.daemon.
> executor$fn__8058$fn__8071$fn_
> > _8124.invoke(executor.clj:850)
> > &gt;&gt; [storm-core-1.0.2.jar:1.0.2]
> > &gt;&gt;        at org.apache.storm.util$async_
> > loop$fn__624.invoke(util.clj:484)
> > &gt;&gt; [storm-core-1.0.2.jar:1.0.2]
> > &gt;&gt;        at clojure.lang.AFn.run(AFn.java:22)
> [clojure-1.7.0.jar:?]
> > &gt;&gt;        at java.lang.Thread.run(Thread.java:745) [?:1.7.0_80]
> > &gt;&gt; Caused by: com.google.protobuf.ServiceException: Error calling
> > method
> > &gt;&gt; MultiRowMutationService.MutateRows
> > &gt;&gt;        at org.apache.hadoop.hbase.ipc.CoprocessorRpcChannel.
> > &gt;&gt; callBlockingMethod(CoprocessorRpcChannel.java:75)
> > ~[stormjar.jar:?]
> > &gt;&gt;        at org.apache.hadoop.hbase.protobuf.generated.
> > &gt;&gt; MultiRowMutationProtos$MultiRowMutationService$
> > BlockingStub.mutateRows(
> > &gt;&gt; MultiRowMutationProtos.java:2149) ~[stormjar.jar:?]
> > &gt;&gt;        at ...
> > &gt;&gt;        ... 13 more
> > &gt;&gt; Caused by: org.apache.hadoop.hbase.client.
> > RetriesExhaustedException:
> > &gt;&gt; Failed after attempts=35, exceptions:
> > &gt;&gt; Wed May 23 07:15:57 KST 2018, RpcRetryingCaller{
> > globalStartTime=1527027357808,
> > &gt;&gt; pause=100, retries=35}, org.apache.hadoop.hbase.
> > RegionTooBusyException:
> > &gt;&gt; org.apache.hadoop.hbase.RegionTooBusyException: Above memstore
> > limit,
> > &gt;&gt; regionName={region}, server={server}, memstoreSize=2600502128,
> > &gt;&gt; blockingMemStoreSize=2600468480
> > &gt;&gt;        at org.apache.hadoop.hbase.regionserver.HRegion.
> > &gt;&gt; checkResources(HRegion.java:3649)
> > &gt;&gt;        at org.apache.hadoop.hbase.regionserver.HRegion.
> > &gt;&gt; processRowsWithLocks(HRegion.java:6935)
> > &gt;&gt;        at org.apache.hadoop.hbase.regionserver.HRegion.
> > &gt;&gt; mutateRowsWithLocks(HRegion.java:6885)
> > &gt;&gt;        at org.apache.hadoop.hbase.coprocessor.
> > MultiRowMutationEndpoint.
> > &gt;&gt; mutateRows(MultiRowMutationEndpoint.java:116)
> > &gt;&gt;        at org.apache.hadoop.hbase.protobuf.generated.
> > &gt;&gt; MultiRowMutationProtos$MultiRowMutationService.callMethod(
> > &gt;&gt; MultiRowMutationProtos.java:2053)
> > &gt;&gt;        at org.apache.hadoop.hbase.regionserver.HRegion.
> > &gt;&gt; execService(HRegion.java:7875)
> > &gt;&gt;        at org.apache.hadoop.hbase.regionserver.RSRpcServices.
> > &gt;&gt; execServiceOnRegion(RSRpcServices.java:2008)
> > &gt;&gt;
> > &gt;&gt;
> > &gt;&gt; Best regards,
> > &gt;&gt; Minwoo Kang
> > &gt;&gt;
> > &gt;&gt; ________________________________________
> > &gt;&gt; 보낸 사람: 张铎(Duo Zhang) <pa...@gmail.com>
> > &gt;&gt; 보낸 날짜: 2018년 5월 23일 수요일 09:22
> > &gt;&gt; 받는 사람: hbase-user
> > &gt;&gt; 제목: Re: can not write to HBase
> > &gt;&gt;
> > &gt;&gt; What is the exception? And the stacktrace?
> > &gt;&gt;
> > &gt;&gt; 2018-05-23 8:17 GMT+08:00 Kang Minwoo <minwoo.kang@outlook.com
> >:
> > &gt;&gt;
> > &gt;&gt;&gt; Hello, Users
> > &gt;&gt;&gt;
> > &gt;&gt;&gt; My HBase client does not work after print the following
> logs.
> > &gt;&gt;&gt;
> > &gt;&gt;&gt; Call exception, tries=23, retries=35, started=291277 ms ago,
> > &gt;&gt;&gt; cancelled=false, msg=row '{row}' on table '{table}' at
> > region={region},
> > &gt;&gt;&gt; hostname={hostname}, seqNum=100353531
> > &gt;&gt;&gt;
> > &gt;&gt;&gt; There are no special logs in the Master and Region Servers.
> > &gt;&gt;&gt; Is it something wrong in the client?
> > &gt;&gt;&gt;
> > &gt;&gt;&gt; Best regards,
> > &gt;&gt;&gt; Minwoo Kang
> > &gt;&gt;&gt;
> > &gt;&gt;
> > </m...@gmail.com></minwoo.
> > kang@outlook.com></p...@outlook.com></
> > anilgupta84@gmail.com></m...@outlook.com>
>

Re: can not write to HBase

Posted by Kang Minwoo <mi...@outlook.com>.
I have a same error on today.
thread dump is here.

----------------

Thread 286 (MemStoreFlusher.1):
  State: WAITING
  Blocked count: 10704
  Waited count: 10936
  Waiting on java.util.concurrent.locks.ReentrantLock$NonfairSync@2afc16fd
  Stack:
    sun.misc.Unsafe.park(Native Method)
    java.util.concurrent.locks.LockSupport.park(LockSupport.java:175)
    java.util.concurrent.locks.AbstractQueuedSynchronizer.parkAndCheckInterrupt(AbstractQueuedSynchronizer.java:836)
    java.util.concurrent.locks.AbstractQueuedSynchronizer.acquireQueued(AbstractQueuedSynchronizer.java:870)
    java.util.concurrent.locks.AbstractQueuedSynchronizer.acquire(AbstractQueuedSynchronizer.java:1199)
    java.util.concurrent.locks.ReentrantLock$NonfairSync.lock(ReentrantLock.java:209)
    java.util.concurrent.locks.ReentrantLock.lock(ReentrantLock.java:285)
    org.apache.hadoop.hbase.regionserver.StoreScanner.updateReaders(StoreScanner.java:693)
    org.apache.hadoop.hbase.regionserver.HStore.notifyChangedReadersObservers(HStore.java:1093)
    org.apache.hadoop.hbase.regionserver.HStore.updateStorefiles(HStore.java:1072)
    org.apache.hadoop.hbase.regionserver.HStore.access$700(HStore.java:118)
    org.apache.hadoop.hbase.regionserver.HStore$StoreFlusherImpl.commit(HStore.java:2310)
    org.apache.hadoop.hbase.regionserver.HRegion.internalFlushCacheAndCommit(HRegion.java:2386)
    org.apache.hadoop.hbase.regionserver.HRegion.internalFlushcache(HRegion.java:2108)
    org.apache.hadoop.hbase.regionserver.HRegion.internalFlushcache(HRegion.java:2070)
    org.apache.hadoop.hbase.regionserver.HRegion.flushcache(HRegion.java:1961)
    org.apache.hadoop.hbase.regionserver.HRegion.flush(HRegion.java:1887)
    org.apache.hadoop.hbase.regionserver.MemStoreFlusher.flushRegion(MemStoreFlusher.java:514)
    org.apache.hadoop.hbase.regionserver.MemStoreFlusher.flushRegion(MemStoreFlusher.java:475)
    org.apache.hadoop.hbase.regionserver.MemStoreFlusher.access$900(MemStoreFlusher.java:75)


----------------

I deleted many row these days.
I think that affected.

Best regards,
Minwoo Kang

________________________________________
보낸 사람: Kang Minwoo <mi...@outlook.com>
보낸 날짜: 2018년 5월 23일 수요일 16:53
받는 사람: Hbase-User
제목: Re: can not write to HBase

In HRegion#internalFlushCacheAndCommit
There is following code.

    synchronized (this) {
      notifyAll(); // FindBugs NN_NAKED_NOTIFY
    }

one question.
Where is the lock acquired?

Best regards,
Minwoo Kang

________________________________________
보낸 사람: Kang Minwoo <mi...@outlook.com>
보낸 날짜: 2018년 5월 23일 수요일 16:37
받는 사람: Hbase-User
제목: Re: can not write to HBase

Next time, If I have a same problem. I will save the jstack of the RS.
(I could not think of saving jstack this time.)

I did not see any special logs.
There was only a warn log that the HBase Scan was slow.

Best regards,
Minwoo Kang

________________________________________
보낸 사람: Yu Li <ca...@gmail.com>
보낸 날짜: 2018년 5월 23일 수요일 15:53
받는 사람: Hbase-User
제목: Re: can not write to HBase

Please save the jstack of the RS when slow flush is ongoing and confirm
whether it stuck at the HDFS writing phase. If so, check log of the local
DN co-located with the RegionServer to see whether any notice-able
exceptions.

Best Regards,
Yu

On 23 May 2018 at 14:19, Kang Minwoo <mi...@outlook.com> wrote:

> @Duo Zhang
> This means that you're writing too fast and memstore has reached its upper
> limit. Is the flush and compaction fine at RS side?
>
> -> No, flush took very long time.
> I attach code that took a long time to run. (about 30min)
>
> https://github.com/apache/hbase/blob/branch-1.2/hbase-
> server/src/main/java/org/apache/hadoop/hbase/regionserver/HRegion.java#
> L2424-L2508
>
> Best regards,
> Minwoo Kang
> ________________________________________
> 보낸 사람: Kang Minwoo <mi...@outlook.com>
> 보낸 날짜: 2018년 5월 23일 수요일 15:16
> 받는 사람: user@hbase.apache.org
> 제목: Re: can not write to HBase
>
> I am using salt, prevent write hotspot.
> And Table has 4000 regions.
>
> HBase version is 1.2.6.
>
> I attach code that took a long time to run. (about 30min)
> https://github.com/apache/hbase/blob/branch-1.2/hbase-
> server/src/main/java/org/apache/hadoop/hbase/regionserver/HRegion.java#
> L2424-L2508
>
> Best regards,
> Minwoo Kang
>
> ________________________________________
> 보낸 사람: Anil Gupta <an...@gmail.com>
> 보낸 날짜: 2018년 5월 23일 수요일 14:13
> 받는 사람: user@hbase.apache.org
> 제목: Re: can not write to HBase
>
> It seems you might have a write hotspot.
> Are your writes evenly distributed across the cluster? Do you have more
> than 15-20 regions for that table?
>
> Sent from my iPhone
>
> &gt; On May 22, 2018, at 9:52 PM, Kang Minwoo <mi...@outlook.com>
> wrote:
> &gt;
> &gt; I think hbase flush is too slow.
> &gt; so memstore reached upper limit.
> &gt;
> &gt; flush took about 30min.
> &gt; I don't know why flush is too long.
> &gt;
> &gt; Best regards,
> &gt; Minwoo Kang
> &gt;
> &gt; ________________________________________
> &gt; 보낸 사람: 张铎(Duo Zhang) <pa...@gmail.com>
> &gt; 보낸 날짜: 2018년 5월 23일 수요일 11:37
> &gt; 받는 사람: hbase-user
> &gt; 제목: Re: can not write to HBase
> &gt;
> &gt; org.apache.hadoop.hbase.RegionTooBusyException:
> &gt; org.apache.hadoop.hbase.RegionTooBusyException:
> &gt; Above memstore limit, regionName={region}, server={server},
> &gt; memstoreSize=2600502128, blockingMemStoreSize=2600468480
> &gt;
> &gt; This means that you're writing too fast and memstore has reached its
> upper
> &gt; limit. Is the flush and compaction fine at RS side?
> &gt;
> &gt; 2018-05-23 10:20 GMT+08:00 Kang Minwoo <mi...@outlook.com>:
> &gt;
> &gt;&gt; attach client exception and stacktrace.
> &gt;&gt;
> &gt;&gt; I've looked more.
> &gt;&gt; It seems to be the reason why it takes 1290 seconds to flush in
> the Region
> &gt;&gt; Server.
> &gt;&gt;
> &gt;&gt; 2018-05-23T07:24:31.202 [INFO] Call exception, tries=34,
> retries=35,
> &gt;&gt; started=513393 ms ago, cancelled=false, msg=row '{row}' on table
> '{table}'
> &gt;&gt; at region={region}, hostname={host}, seqNum=155455658
> &gt;&gt; 2018-05-23T07:24:31.208 [ERROR]
> &gt;&gt; java.lang.RuntimeException: com.google.protobuf.ServiceException:
> Error
> &gt;&gt; calling method MultiRowMutationService.MutateRows
> &gt;&gt;        at com.google.common.base.Throwables.propagate(
> Throwables.java:160)
> &gt;&gt; ~[stormjar.jar:?]
> &gt;&gt;        at ...
> &gt;&gt;        at org.apache.storm.daemon.executor$fn__8058$tuple_
> &gt;&gt; action_fn__8060.invoke(executor.clj:731)
> [storm-core-1.0.2.jar:1.0.2]
> &gt;&gt;        at org.apache.storm.daemon.executor$mk_task_receiver$fn__
> 7979.invoke(executor.clj:464)
> &gt;&gt; [storm-core-1.0.2.jar:1.0.2]
> &gt;&gt;        at org.apache.storm.disruptor$clojure_handler$reify__7492.
> onEvent(disruptor.clj:40)
> &gt;&gt; [storm-core-1.0.2.jar:1.0.2]
> &gt;&gt;        at org.apache.storm.utils.DisruptorQueue.
> consumeBatchToCursor(DisruptorQueue.java:451)
> &gt;&gt; [storm-core-1.0.2.jar:1.0.2]
> &gt;&gt;        at org.apache.storm.utils.DisruptorQueue.
> &gt;&gt; consumeBatchWhenAvailable(DisruptorQueue.java:430)
> &gt;&gt; [storm-core-1.0.2.jar:1.0.2]
> &gt;&gt;        at org.apache.storm.disruptor$
> consume_batch_when_available.invoke(disruptor.clj:73)
> &gt;&gt; [storm-core-1.0.2.jar:1.0.2]
> &gt;&gt;        at org.apache.storm.daemon.executor$fn__8058$fn__8071$fn_
> _8124.invoke(executor.clj:850)
> &gt;&gt; [storm-core-1.0.2.jar:1.0.2]
> &gt;&gt;        at org.apache.storm.util$async_
> loop$fn__624.invoke(util.clj:484)
> &gt;&gt; [storm-core-1.0.2.jar:1.0.2]
> &gt;&gt;        at clojure.lang.AFn.run(AFn.java:22) [clojure-1.7.0.jar:?]
> &gt;&gt;        at java.lang.Thread.run(Thread.java:745) [?:1.7.0_80]
> &gt;&gt; Caused by: com.google.protobuf.ServiceException: Error calling
> method
> &gt;&gt; MultiRowMutationService.MutateRows
> &gt;&gt;        at org.apache.hadoop.hbase.ipc.CoprocessorRpcChannel.
> &gt;&gt; callBlockingMethod(CoprocessorRpcChannel.java:75)
> ~[stormjar.jar:?]
> &gt;&gt;        at org.apache.hadoop.hbase.protobuf.generated.
> &gt;&gt; MultiRowMutationProtos$MultiRowMutationService$
> BlockingStub.mutateRows(
> &gt;&gt; MultiRowMutationProtos.java:2149) ~[stormjar.jar:?]
> &gt;&gt;        at ...
> &gt;&gt;        ... 13 more
> &gt;&gt; Caused by: org.apache.hadoop.hbase.client.
> RetriesExhaustedException:
> &gt;&gt; Failed after attempts=35, exceptions:
> &gt;&gt; Wed May 23 07:15:57 KST 2018, RpcRetryingCaller{
> globalStartTime=1527027357808,
> &gt;&gt; pause=100, retries=35}, org.apache.hadoop.hbase.
> RegionTooBusyException:
> &gt;&gt; org.apache.hadoop.hbase.RegionTooBusyException: Above memstore
> limit,
> &gt;&gt; regionName={region}, server={server}, memstoreSize=2600502128,
> &gt;&gt; blockingMemStoreSize=2600468480
> &gt;&gt;        at org.apache.hadoop.hbase.regionserver.HRegion.
> &gt;&gt; checkResources(HRegion.java:3649)
> &gt;&gt;        at org.apache.hadoop.hbase.regionserver.HRegion.
> &gt;&gt; processRowsWithLocks(HRegion.java:6935)
> &gt;&gt;        at org.apache.hadoop.hbase.regionserver.HRegion.
> &gt;&gt; mutateRowsWithLocks(HRegion.java:6885)
> &gt;&gt;        at org.apache.hadoop.hbase.coprocessor.
> MultiRowMutationEndpoint.
> &gt;&gt; mutateRows(MultiRowMutationEndpoint.java:116)
> &gt;&gt;        at org.apache.hadoop.hbase.protobuf.generated.
> &gt;&gt; MultiRowMutationProtos$MultiRowMutationService.callMethod(
> &gt;&gt; MultiRowMutationProtos.java:2053)
> &gt;&gt;        at org.apache.hadoop.hbase.regionserver.HRegion.
> &gt;&gt; execService(HRegion.java:7875)
> &gt;&gt;        at org.apache.hadoop.hbase.regionserver.RSRpcServices.
> &gt;&gt; execServiceOnRegion(RSRpcServices.java:2008)
> &gt;&gt;
> &gt;&gt;
> &gt;&gt; Best regards,
> &gt;&gt; Minwoo Kang
> &gt;&gt;
> &gt;&gt; ________________________________________
> &gt;&gt; 보낸 사람: 张铎(Duo Zhang) <pa...@gmail.com>
> &gt;&gt; 보낸 날짜: 2018년 5월 23일 수요일 09:22
> &gt;&gt; 받는 사람: hbase-user
> &gt;&gt; 제목: Re: can not write to HBase
> &gt;&gt;
> &gt;&gt; What is the exception? And the stacktrace?
> &gt;&gt;
> &gt;&gt; 2018-05-23 8:17 GMT+08:00 Kang Minwoo <mi...@outlook.com>:
> &gt;&gt;
> &gt;&gt;&gt; Hello, Users
> &gt;&gt;&gt;
> &gt;&gt;&gt; My HBase client does not work after print the following logs.
> &gt;&gt;&gt;
> &gt;&gt;&gt; Call exception, tries=23, retries=35, started=291277 ms ago,
> &gt;&gt;&gt; cancelled=false, msg=row '{row}' on table '{table}' at
> region={region},
> &gt;&gt;&gt; hostname={hostname}, seqNum=100353531
> &gt;&gt;&gt;
> &gt;&gt;&gt; There are no special logs in the Master and Region Servers.
> &gt;&gt;&gt; Is it something wrong in the client?
> &gt;&gt;&gt;
> &gt;&gt;&gt; Best regards,
> &gt;&gt;&gt; Minwoo Kang
> &gt;&gt;&gt;
> &gt;&gt;
> </m...@gmail.com></minwoo.
> kang@outlook.com></p...@outlook.com></
> anilgupta84@gmail.com></m...@outlook.com>

Re: can not write to HBase

Posted by Kang Minwoo <mi...@outlook.com>.
In HRegion#internalFlushCacheAndCommit
There is following code.

    synchronized (this) {
      notifyAll(); // FindBugs NN_NAKED_NOTIFY
    }

one question.
Where is the lock acquired?

Best regards,
Minwoo Kang

________________________________________
보낸 사람: Kang Minwoo <mi...@outlook.com>
보낸 날짜: 2018년 5월 23일 수요일 16:37
받는 사람: Hbase-User
제목: Re: can not write to HBase

Next time, If I have a same problem. I will save the jstack of the RS.
(I could not think of saving jstack this time.)

I did not see any special logs.
There was only a warn log that the HBase Scan was slow.

Best regards,
Minwoo Kang

________________________________________
보낸 사람: Yu Li <ca...@gmail.com>
보낸 날짜: 2018년 5월 23일 수요일 15:53
받는 사람: Hbase-User
제목: Re: can not write to HBase

Please save the jstack of the RS when slow flush is ongoing and confirm
whether it stuck at the HDFS writing phase. If so, check log of the local
DN co-located with the RegionServer to see whether any notice-able
exceptions.

Best Regards,
Yu

On 23 May 2018 at 14:19, Kang Minwoo <mi...@outlook.com> wrote:

> @Duo Zhang
> This means that you're writing too fast and memstore has reached its upper
> limit. Is the flush and compaction fine at RS side?
>
> -> No, flush took very long time.
> I attach code that took a long time to run. (about 30min)
>
> https://github.com/apache/hbase/blob/branch-1.2/hbase-
> server/src/main/java/org/apache/hadoop/hbase/regionserver/HRegion.java#
> L2424-L2508
>
> Best regards,
> Minwoo Kang
> ________________________________________
> 보낸 사람: Kang Minwoo <mi...@outlook.com>
> 보낸 날짜: 2018년 5월 23일 수요일 15:16
> 받는 사람: user@hbase.apache.org
> 제목: Re: can not write to HBase
>
> I am using salt, prevent write hotspot.
> And Table has 4000 regions.
>
> HBase version is 1.2.6.
>
> I attach code that took a long time to run. (about 30min)
> https://github.com/apache/hbase/blob/branch-1.2/hbase-
> server/src/main/java/org/apache/hadoop/hbase/regionserver/HRegion.java#
> L2424-L2508
>
> Best regards,
> Minwoo Kang
>
> ________________________________________
> 보낸 사람: Anil Gupta <an...@gmail.com>
> 보낸 날짜: 2018년 5월 23일 수요일 14:13
> 받는 사람: user@hbase.apache.org
> 제목: Re: can not write to HBase
>
> It seems you might have a write hotspot.
> Are your writes evenly distributed across the cluster? Do you have more
> than 15-20 regions for that table?
>
> Sent from my iPhone
>
> &gt; On May 22, 2018, at 9:52 PM, Kang Minwoo <mi...@outlook.com>
> wrote:
> &gt;
> &gt; I think hbase flush is too slow.
> &gt; so memstore reached upper limit.
> &gt;
> &gt; flush took about 30min.
> &gt; I don't know why flush is too long.
> &gt;
> &gt; Best regards,
> &gt; Minwoo Kang
> &gt;
> &gt; ________________________________________
> &gt; 보낸 사람: 张铎(Duo Zhang) <pa...@gmail.com>
> &gt; 보낸 날짜: 2018년 5월 23일 수요일 11:37
> &gt; 받는 사람: hbase-user
> &gt; 제목: Re: can not write to HBase
> &gt;
> &gt; org.apache.hadoop.hbase.RegionTooBusyException:
> &gt; org.apache.hadoop.hbase.RegionTooBusyException:
> &gt; Above memstore limit, regionName={region}, server={server},
> &gt; memstoreSize=2600502128, blockingMemStoreSize=2600468480
> &gt;
> &gt; This means that you're writing too fast and memstore has reached its
> upper
> &gt; limit. Is the flush and compaction fine at RS side?
> &gt;
> &gt; 2018-05-23 10:20 GMT+08:00 Kang Minwoo <mi...@outlook.com>:
> &gt;
> &gt;&gt; attach client exception and stacktrace.
> &gt;&gt;
> &gt;&gt; I've looked more.
> &gt;&gt; It seems to be the reason why it takes 1290 seconds to flush in
> the Region
> &gt;&gt; Server.
> &gt;&gt;
> &gt;&gt; 2018-05-23T07:24:31.202 [INFO] Call exception, tries=34,
> retries=35,
> &gt;&gt; started=513393 ms ago, cancelled=false, msg=row '{row}' on table
> '{table}'
> &gt;&gt; at region={region}, hostname={host}, seqNum=155455658
> &gt;&gt; 2018-05-23T07:24:31.208 [ERROR]
> &gt;&gt; java.lang.RuntimeException: com.google.protobuf.ServiceException:
> Error
> &gt;&gt; calling method MultiRowMutationService.MutateRows
> &gt;&gt;        at com.google.common.base.Throwables.propagate(
> Throwables.java:160)
> &gt;&gt; ~[stormjar.jar:?]
> &gt;&gt;        at ...
> &gt;&gt;        at org.apache.storm.daemon.executor$fn__8058$tuple_
> &gt;&gt; action_fn__8060.invoke(executor.clj:731)
> [storm-core-1.0.2.jar:1.0.2]
> &gt;&gt;        at org.apache.storm.daemon.executor$mk_task_receiver$fn__
> 7979.invoke(executor.clj:464)
> &gt;&gt; [storm-core-1.0.2.jar:1.0.2]
> &gt;&gt;        at org.apache.storm.disruptor$clojure_handler$reify__7492.
> onEvent(disruptor.clj:40)
> &gt;&gt; [storm-core-1.0.2.jar:1.0.2]
> &gt;&gt;        at org.apache.storm.utils.DisruptorQueue.
> consumeBatchToCursor(DisruptorQueue.java:451)
> &gt;&gt; [storm-core-1.0.2.jar:1.0.2]
> &gt;&gt;        at org.apache.storm.utils.DisruptorQueue.
> &gt;&gt; consumeBatchWhenAvailable(DisruptorQueue.java:430)
> &gt;&gt; [storm-core-1.0.2.jar:1.0.2]
> &gt;&gt;        at org.apache.storm.disruptor$
> consume_batch_when_available.invoke(disruptor.clj:73)
> &gt;&gt; [storm-core-1.0.2.jar:1.0.2]
> &gt;&gt;        at org.apache.storm.daemon.executor$fn__8058$fn__8071$fn_
> _8124.invoke(executor.clj:850)
> &gt;&gt; [storm-core-1.0.2.jar:1.0.2]
> &gt;&gt;        at org.apache.storm.util$async_
> loop$fn__624.invoke(util.clj:484)
> &gt;&gt; [storm-core-1.0.2.jar:1.0.2]
> &gt;&gt;        at clojure.lang.AFn.run(AFn.java:22) [clojure-1.7.0.jar:?]
> &gt;&gt;        at java.lang.Thread.run(Thread.java:745) [?:1.7.0_80]
> &gt;&gt; Caused by: com.google.protobuf.ServiceException: Error calling
> method
> &gt;&gt; MultiRowMutationService.MutateRows
> &gt;&gt;        at org.apache.hadoop.hbase.ipc.CoprocessorRpcChannel.
> &gt;&gt; callBlockingMethod(CoprocessorRpcChannel.java:75)
> ~[stormjar.jar:?]
> &gt;&gt;        at org.apache.hadoop.hbase.protobuf.generated.
> &gt;&gt; MultiRowMutationProtos$MultiRowMutationService$
> BlockingStub.mutateRows(
> &gt;&gt; MultiRowMutationProtos.java:2149) ~[stormjar.jar:?]
> &gt;&gt;        at ...
> &gt;&gt;        ... 13 more
> &gt;&gt; Caused by: org.apache.hadoop.hbase.client.
> RetriesExhaustedException:
> &gt;&gt; Failed after attempts=35, exceptions:
> &gt;&gt; Wed May 23 07:15:57 KST 2018, RpcRetryingCaller{
> globalStartTime=1527027357808,
> &gt;&gt; pause=100, retries=35}, org.apache.hadoop.hbase.
> RegionTooBusyException:
> &gt;&gt; org.apache.hadoop.hbase.RegionTooBusyException: Above memstore
> limit,
> &gt;&gt; regionName={region}, server={server}, memstoreSize=2600502128,
> &gt;&gt; blockingMemStoreSize=2600468480
> &gt;&gt;        at org.apache.hadoop.hbase.regionserver.HRegion.
> &gt;&gt; checkResources(HRegion.java:3649)
> &gt;&gt;        at org.apache.hadoop.hbase.regionserver.HRegion.
> &gt;&gt; processRowsWithLocks(HRegion.java:6935)
> &gt;&gt;        at org.apache.hadoop.hbase.regionserver.HRegion.
> &gt;&gt; mutateRowsWithLocks(HRegion.java:6885)
> &gt;&gt;        at org.apache.hadoop.hbase.coprocessor.
> MultiRowMutationEndpoint.
> &gt;&gt; mutateRows(MultiRowMutationEndpoint.java:116)
> &gt;&gt;        at org.apache.hadoop.hbase.protobuf.generated.
> &gt;&gt; MultiRowMutationProtos$MultiRowMutationService.callMethod(
> &gt;&gt; MultiRowMutationProtos.java:2053)
> &gt;&gt;        at org.apache.hadoop.hbase.regionserver.HRegion.
> &gt;&gt; execService(HRegion.java:7875)
> &gt;&gt;        at org.apache.hadoop.hbase.regionserver.RSRpcServices.
> &gt;&gt; execServiceOnRegion(RSRpcServices.java:2008)
> &gt;&gt;
> &gt;&gt;
> &gt;&gt; Best regards,
> &gt;&gt; Minwoo Kang
> &gt;&gt;
> &gt;&gt; ________________________________________
> &gt;&gt; 보낸 사람: 张铎(Duo Zhang) <pa...@gmail.com>
> &gt;&gt; 보낸 날짜: 2018년 5월 23일 수요일 09:22
> &gt;&gt; 받는 사람: hbase-user
> &gt;&gt; 제목: Re: can not write to HBase
> &gt;&gt;
> &gt;&gt; What is the exception? And the stacktrace?
> &gt;&gt;
> &gt;&gt; 2018-05-23 8:17 GMT+08:00 Kang Minwoo <mi...@outlook.com>:
> &gt;&gt;
> &gt;&gt;&gt; Hello, Users
> &gt;&gt;&gt;
> &gt;&gt;&gt; My HBase client does not work after print the following logs.
> &gt;&gt;&gt;
> &gt;&gt;&gt; Call exception, tries=23, retries=35, started=291277 ms ago,
> &gt;&gt;&gt; cancelled=false, msg=row '{row}' on table '{table}' at
> region={region},
> &gt;&gt;&gt; hostname={hostname}, seqNum=100353531
> &gt;&gt;&gt;
> &gt;&gt;&gt; There are no special logs in the Master and Region Servers.
> &gt;&gt;&gt; Is it something wrong in the client?
> &gt;&gt;&gt;
> &gt;&gt;&gt; Best regards,
> &gt;&gt;&gt; Minwoo Kang
> &gt;&gt;&gt;
> &gt;&gt;
> </m...@gmail.com></minwoo.
> kang@outlook.com></p...@outlook.com></
> anilgupta84@gmail.com></m...@outlook.com>

Re: can not write to HBase

Posted by Kang Minwoo <mi...@outlook.com>.
Next time, If I have a same problem. I will save the jstack of the RS.
(I could not think of saving jstack this time.)

I did not see any special logs.
There was only a warn log that the HBase Scan was slow.

Best regards,
Minwoo Kang

________________________________________
보낸 사람: Yu Li <ca...@gmail.com>
보낸 날짜: 2018년 5월 23일 수요일 15:53
받는 사람: Hbase-User
제목: Re: can not write to HBase

Please save the jstack of the RS when slow flush is ongoing and confirm
whether it stuck at the HDFS writing phase. If so, check log of the local
DN co-located with the RegionServer to see whether any notice-able
exceptions.

Best Regards,
Yu

On 23 May 2018 at 14:19, Kang Minwoo <mi...@outlook.com> wrote:

> @Duo Zhang
> This means that you're writing too fast and memstore has reached its upper
> limit. Is the flush and compaction fine at RS side?
>
> -> No, flush took very long time.
> I attach code that took a long time to run. (about 30min)
>
> https://github.com/apache/hbase/blob/branch-1.2/hbase-
> server/src/main/java/org/apache/hadoop/hbase/regionserver/HRegion.java#
> L2424-L2508
>
> Best regards,
> Minwoo Kang
> ________________________________________
> 보낸 사람: Kang Minwoo <mi...@outlook.com>
> 보낸 날짜: 2018년 5월 23일 수요일 15:16
> 받는 사람: user@hbase.apache.org
> 제목: Re: can not write to HBase
>
> I am using salt, prevent write hotspot.
> And Table has 4000 regions.
>
> HBase version is 1.2.6.
>
> I attach code that took a long time to run. (about 30min)
> https://github.com/apache/hbase/blob/branch-1.2/hbase-
> server/src/main/java/org/apache/hadoop/hbase/regionserver/HRegion.java#
> L2424-L2508
>
> Best regards,
> Minwoo Kang
>
> ________________________________________
> 보낸 사람: Anil Gupta <an...@gmail.com>
> 보낸 날짜: 2018년 5월 23일 수요일 14:13
> 받는 사람: user@hbase.apache.org
> 제목: Re: can not write to HBase
>
> It seems you might have a write hotspot.
> Are your writes evenly distributed across the cluster? Do you have more
> than 15-20 regions for that table?
>
> Sent from my iPhone
>
> &gt; On May 22, 2018, at 9:52 PM, Kang Minwoo <mi...@outlook.com>
> wrote:
> &gt;
> &gt; I think hbase flush is too slow.
> &gt; so memstore reached upper limit.
> &gt;
> &gt; flush took about 30min.
> &gt; I don't know why flush is too long.
> &gt;
> &gt; Best regards,
> &gt; Minwoo Kang
> &gt;
> &gt; ________________________________________
> &gt; 보낸 사람: 张铎(Duo Zhang) <pa...@gmail.com>
> &gt; 보낸 날짜: 2018년 5월 23일 수요일 11:37
> &gt; 받는 사람: hbase-user
> &gt; 제목: Re: can not write to HBase
> &gt;
> &gt; org.apache.hadoop.hbase.RegionTooBusyException:
> &gt; org.apache.hadoop.hbase.RegionTooBusyException:
> &gt; Above memstore limit, regionName={region}, server={server},
> &gt; memstoreSize=2600502128, blockingMemStoreSize=2600468480
> &gt;
> &gt; This means that you're writing too fast and memstore has reached its
> upper
> &gt; limit. Is the flush and compaction fine at RS side?
> &gt;
> &gt; 2018-05-23 10:20 GMT+08:00 Kang Minwoo <mi...@outlook.com>:
> &gt;
> &gt;&gt; attach client exception and stacktrace.
> &gt;&gt;
> &gt;&gt; I've looked more.
> &gt;&gt; It seems to be the reason why it takes 1290 seconds to flush in
> the Region
> &gt;&gt; Server.
> &gt;&gt;
> &gt;&gt; 2018-05-23T07:24:31.202 [INFO] Call exception, tries=34,
> retries=35,
> &gt;&gt; started=513393 ms ago, cancelled=false, msg=row '{row}' on table
> '{table}'
> &gt;&gt; at region={region}, hostname={host}, seqNum=155455658
> &gt;&gt; 2018-05-23T07:24:31.208 [ERROR]
> &gt;&gt; java.lang.RuntimeException: com.google.protobuf.ServiceException:
> Error
> &gt;&gt; calling method MultiRowMutationService.MutateRows
> &gt;&gt;        at com.google.common.base.Throwables.propagate(
> Throwables.java:160)
> &gt;&gt; ~[stormjar.jar:?]
> &gt;&gt;        at ...
> &gt;&gt;        at org.apache.storm.daemon.executor$fn__8058$tuple_
> &gt;&gt; action_fn__8060.invoke(executor.clj:731)
> [storm-core-1.0.2.jar:1.0.2]
> &gt;&gt;        at org.apache.storm.daemon.executor$mk_task_receiver$fn__
> 7979.invoke(executor.clj:464)
> &gt;&gt; [storm-core-1.0.2.jar:1.0.2]
> &gt;&gt;        at org.apache.storm.disruptor$clojure_handler$reify__7492.
> onEvent(disruptor.clj:40)
> &gt;&gt; [storm-core-1.0.2.jar:1.0.2]
> &gt;&gt;        at org.apache.storm.utils.DisruptorQueue.
> consumeBatchToCursor(DisruptorQueue.java:451)
> &gt;&gt; [storm-core-1.0.2.jar:1.0.2]
> &gt;&gt;        at org.apache.storm.utils.DisruptorQueue.
> &gt;&gt; consumeBatchWhenAvailable(DisruptorQueue.java:430)
> &gt;&gt; [storm-core-1.0.2.jar:1.0.2]
> &gt;&gt;        at org.apache.storm.disruptor$
> consume_batch_when_available.invoke(disruptor.clj:73)
> &gt;&gt; [storm-core-1.0.2.jar:1.0.2]
> &gt;&gt;        at org.apache.storm.daemon.executor$fn__8058$fn__8071$fn_
> _8124.invoke(executor.clj:850)
> &gt;&gt; [storm-core-1.0.2.jar:1.0.2]
> &gt;&gt;        at org.apache.storm.util$async_
> loop$fn__624.invoke(util.clj:484)
> &gt;&gt; [storm-core-1.0.2.jar:1.0.2]
> &gt;&gt;        at clojure.lang.AFn.run(AFn.java:22) [clojure-1.7.0.jar:?]
> &gt;&gt;        at java.lang.Thread.run(Thread.java:745) [?:1.7.0_80]
> &gt;&gt; Caused by: com.google.protobuf.ServiceException: Error calling
> method
> &gt;&gt; MultiRowMutationService.MutateRows
> &gt;&gt;        at org.apache.hadoop.hbase.ipc.CoprocessorRpcChannel.
> &gt;&gt; callBlockingMethod(CoprocessorRpcChannel.java:75)
> ~[stormjar.jar:?]
> &gt;&gt;        at org.apache.hadoop.hbase.protobuf.generated.
> &gt;&gt; MultiRowMutationProtos$MultiRowMutationService$
> BlockingStub.mutateRows(
> &gt;&gt; MultiRowMutationProtos.java:2149) ~[stormjar.jar:?]
> &gt;&gt;        at ...
> &gt;&gt;        ... 13 more
> &gt;&gt; Caused by: org.apache.hadoop.hbase.client.
> RetriesExhaustedException:
> &gt;&gt; Failed after attempts=35, exceptions:
> &gt;&gt; Wed May 23 07:15:57 KST 2018, RpcRetryingCaller{
> globalStartTime=1527027357808,
> &gt;&gt; pause=100, retries=35}, org.apache.hadoop.hbase.
> RegionTooBusyException:
> &gt;&gt; org.apache.hadoop.hbase.RegionTooBusyException: Above memstore
> limit,
> &gt;&gt; regionName={region}, server={server}, memstoreSize=2600502128,
> &gt;&gt; blockingMemStoreSize=2600468480
> &gt;&gt;        at org.apache.hadoop.hbase.regionserver.HRegion.
> &gt;&gt; checkResources(HRegion.java:3649)
> &gt;&gt;        at org.apache.hadoop.hbase.regionserver.HRegion.
> &gt;&gt; processRowsWithLocks(HRegion.java:6935)
> &gt;&gt;        at org.apache.hadoop.hbase.regionserver.HRegion.
> &gt;&gt; mutateRowsWithLocks(HRegion.java:6885)
> &gt;&gt;        at org.apache.hadoop.hbase.coprocessor.
> MultiRowMutationEndpoint.
> &gt;&gt; mutateRows(MultiRowMutationEndpoint.java:116)
> &gt;&gt;        at org.apache.hadoop.hbase.protobuf.generated.
> &gt;&gt; MultiRowMutationProtos$MultiRowMutationService.callMethod(
> &gt;&gt; MultiRowMutationProtos.java:2053)
> &gt;&gt;        at org.apache.hadoop.hbase.regionserver.HRegion.
> &gt;&gt; execService(HRegion.java:7875)
> &gt;&gt;        at org.apache.hadoop.hbase.regionserver.RSRpcServices.
> &gt;&gt; execServiceOnRegion(RSRpcServices.java:2008)
> &gt;&gt;
> &gt;&gt;
> &gt;&gt; Best regards,
> &gt;&gt; Minwoo Kang
> &gt;&gt;
> &gt;&gt; ________________________________________
> &gt;&gt; 보낸 사람: 张铎(Duo Zhang) <pa...@gmail.com>
> &gt;&gt; 보낸 날짜: 2018년 5월 23일 수요일 09:22
> &gt;&gt; 받는 사람: hbase-user
> &gt;&gt; 제목: Re: can not write to HBase
> &gt;&gt;
> &gt;&gt; What is the exception? And the stacktrace?
> &gt;&gt;
> &gt;&gt; 2018-05-23 8:17 GMT+08:00 Kang Minwoo <mi...@outlook.com>:
> &gt;&gt;
> &gt;&gt;&gt; Hello, Users
> &gt;&gt;&gt;
> &gt;&gt;&gt; My HBase client does not work after print the following logs.
> &gt;&gt;&gt;
> &gt;&gt;&gt; Call exception, tries=23, retries=35, started=291277 ms ago,
> &gt;&gt;&gt; cancelled=false, msg=row '{row}' on table '{table}' at
> region={region},
> &gt;&gt;&gt; hostname={hostname}, seqNum=100353531
> &gt;&gt;&gt;
> &gt;&gt;&gt; There are no special logs in the Master and Region Servers.
> &gt;&gt;&gt; Is it something wrong in the client?
> &gt;&gt;&gt;
> &gt;&gt;&gt; Best regards,
> &gt;&gt;&gt; Minwoo Kang
> &gt;&gt;&gt;
> &gt;&gt;
> </m...@gmail.com></minwoo.
> kang@outlook.com></p...@outlook.com></
> anilgupta84@gmail.com></m...@outlook.com>

Re: can not write to HBase

Posted by Yu Li <ca...@gmail.com>.
Please save the jstack of the RS when slow flush is ongoing and confirm
whether it stuck at the HDFS writing phase. If so, check log of the local
DN co-located with the RegionServer to see whether any notice-able
exceptions.

Best Regards,
Yu

On 23 May 2018 at 14:19, Kang Minwoo <mi...@outlook.com> wrote:

> @Duo Zhang
> This means that you're writing too fast and memstore has reached its upper
> limit. Is the flush and compaction fine at RS side?
>
> -> No, flush took very long time.
> I attach code that took a long time to run. (about 30min)
>
> https://github.com/apache/hbase/blob/branch-1.2/hbase-
> server/src/main/java/org/apache/hadoop/hbase/regionserver/HRegion.java#
> L2424-L2508
>
> Best regards,
> Minwoo Kang
> ________________________________________
> 보낸 사람: Kang Minwoo <mi...@outlook.com>
> 보낸 날짜: 2018년 5월 23일 수요일 15:16
> 받는 사람: user@hbase.apache.org
> 제목: Re: can not write to HBase
>
> I am using salt, prevent write hotspot.
> And Table has 4000 regions.
>
> HBase version is 1.2.6.
>
> I attach code that took a long time to run. (about 30min)
> https://github.com/apache/hbase/blob/branch-1.2/hbase-
> server/src/main/java/org/apache/hadoop/hbase/regionserver/HRegion.java#
> L2424-L2508
>
> Best regards,
> Minwoo Kang
>
> ________________________________________
> 보낸 사람: Anil Gupta <an...@gmail.com>
> 보낸 날짜: 2018년 5월 23일 수요일 14:13
> 받는 사람: user@hbase.apache.org
> 제목: Re: can not write to HBase
>
> It seems you might have a write hotspot.
> Are your writes evenly distributed across the cluster? Do you have more
> than 15-20 regions for that table?
>
> Sent from my iPhone
>
> &gt; On May 22, 2018, at 9:52 PM, Kang Minwoo <mi...@outlook.com>
> wrote:
> &gt;
> &gt; I think hbase flush is too slow.
> &gt; so memstore reached upper limit.
> &gt;
> &gt; flush took about 30min.
> &gt; I don't know why flush is too long.
> &gt;
> &gt; Best regards,
> &gt; Minwoo Kang
> &gt;
> &gt; ________________________________________
> &gt; 보낸 사람: 张铎(Duo Zhang) <pa...@gmail.com>
> &gt; 보낸 날짜: 2018년 5월 23일 수요일 11:37
> &gt; 받는 사람: hbase-user
> &gt; 제목: Re: can not write to HBase
> &gt;
> &gt; org.apache.hadoop.hbase.RegionTooBusyException:
> &gt; org.apache.hadoop.hbase.RegionTooBusyException:
> &gt; Above memstore limit, regionName={region}, server={server},
> &gt; memstoreSize=2600502128, blockingMemStoreSize=2600468480
> &gt;
> &gt; This means that you're writing too fast and memstore has reached its
> upper
> &gt; limit. Is the flush and compaction fine at RS side?
> &gt;
> &gt; 2018-05-23 10:20 GMT+08:00 Kang Minwoo <mi...@outlook.com>:
> &gt;
> &gt;&gt; attach client exception and stacktrace.
> &gt;&gt;
> &gt;&gt; I've looked more.
> &gt;&gt; It seems to be the reason why it takes 1290 seconds to flush in
> the Region
> &gt;&gt; Server.
> &gt;&gt;
> &gt;&gt; 2018-05-23T07:24:31.202 [INFO] Call exception, tries=34,
> retries=35,
> &gt;&gt; started=513393 ms ago, cancelled=false, msg=row '{row}' on table
> '{table}'
> &gt;&gt; at region={region}, hostname={host}, seqNum=155455658
> &gt;&gt; 2018-05-23T07:24:31.208 [ERROR]
> &gt;&gt; java.lang.RuntimeException: com.google.protobuf.ServiceException:
> Error
> &gt;&gt; calling method MultiRowMutationService.MutateRows
> &gt;&gt;        at com.google.common.base.Throwables.propagate(
> Throwables.java:160)
> &gt;&gt; ~[stormjar.jar:?]
> &gt;&gt;        at ...
> &gt;&gt;        at org.apache.storm.daemon.executor$fn__8058$tuple_
> &gt;&gt; action_fn__8060.invoke(executor.clj:731)
> [storm-core-1.0.2.jar:1.0.2]
> &gt;&gt;        at org.apache.storm.daemon.executor$mk_task_receiver$fn__
> 7979.invoke(executor.clj:464)
> &gt;&gt; [storm-core-1.0.2.jar:1.0.2]
> &gt;&gt;        at org.apache.storm.disruptor$clojure_handler$reify__7492.
> onEvent(disruptor.clj:40)
> &gt;&gt; [storm-core-1.0.2.jar:1.0.2]
> &gt;&gt;        at org.apache.storm.utils.DisruptorQueue.
> consumeBatchToCursor(DisruptorQueue.java:451)
> &gt;&gt; [storm-core-1.0.2.jar:1.0.2]
> &gt;&gt;        at org.apache.storm.utils.DisruptorQueue.
> &gt;&gt; consumeBatchWhenAvailable(DisruptorQueue.java:430)
> &gt;&gt; [storm-core-1.0.2.jar:1.0.2]
> &gt;&gt;        at org.apache.storm.disruptor$
> consume_batch_when_available.invoke(disruptor.clj:73)
> &gt;&gt; [storm-core-1.0.2.jar:1.0.2]
> &gt;&gt;        at org.apache.storm.daemon.executor$fn__8058$fn__8071$fn_
> _8124.invoke(executor.clj:850)
> &gt;&gt; [storm-core-1.0.2.jar:1.0.2]
> &gt;&gt;        at org.apache.storm.util$async_
> loop$fn__624.invoke(util.clj:484)
> &gt;&gt; [storm-core-1.0.2.jar:1.0.2]
> &gt;&gt;        at clojure.lang.AFn.run(AFn.java:22) [clojure-1.7.0.jar:?]
> &gt;&gt;        at java.lang.Thread.run(Thread.java:745) [?:1.7.0_80]
> &gt;&gt; Caused by: com.google.protobuf.ServiceException: Error calling
> method
> &gt;&gt; MultiRowMutationService.MutateRows
> &gt;&gt;        at org.apache.hadoop.hbase.ipc.CoprocessorRpcChannel.
> &gt;&gt; callBlockingMethod(CoprocessorRpcChannel.java:75)
> ~[stormjar.jar:?]
> &gt;&gt;        at org.apache.hadoop.hbase.protobuf.generated.
> &gt;&gt; MultiRowMutationProtos$MultiRowMutationService$
> BlockingStub.mutateRows(
> &gt;&gt; MultiRowMutationProtos.java:2149) ~[stormjar.jar:?]
> &gt;&gt;        at ...
> &gt;&gt;        ... 13 more
> &gt;&gt; Caused by: org.apache.hadoop.hbase.client.
> RetriesExhaustedException:
> &gt;&gt; Failed after attempts=35, exceptions:
> &gt;&gt; Wed May 23 07:15:57 KST 2018, RpcRetryingCaller{
> globalStartTime=1527027357808,
> &gt;&gt; pause=100, retries=35}, org.apache.hadoop.hbase.
> RegionTooBusyException:
> &gt;&gt; org.apache.hadoop.hbase.RegionTooBusyException: Above memstore
> limit,
> &gt;&gt; regionName={region}, server={server}, memstoreSize=2600502128,
> &gt;&gt; blockingMemStoreSize=2600468480
> &gt;&gt;        at org.apache.hadoop.hbase.regionserver.HRegion.
> &gt;&gt; checkResources(HRegion.java:3649)
> &gt;&gt;        at org.apache.hadoop.hbase.regionserver.HRegion.
> &gt;&gt; processRowsWithLocks(HRegion.java:6935)
> &gt;&gt;        at org.apache.hadoop.hbase.regionserver.HRegion.
> &gt;&gt; mutateRowsWithLocks(HRegion.java:6885)
> &gt;&gt;        at org.apache.hadoop.hbase.coprocessor.
> MultiRowMutationEndpoint.
> &gt;&gt; mutateRows(MultiRowMutationEndpoint.java:116)
> &gt;&gt;        at org.apache.hadoop.hbase.protobuf.generated.
> &gt;&gt; MultiRowMutationProtos$MultiRowMutationService.callMethod(
> &gt;&gt; MultiRowMutationProtos.java:2053)
> &gt;&gt;        at org.apache.hadoop.hbase.regionserver.HRegion.
> &gt;&gt; execService(HRegion.java:7875)
> &gt;&gt;        at org.apache.hadoop.hbase.regionserver.RSRpcServices.
> &gt;&gt; execServiceOnRegion(RSRpcServices.java:2008)
> &gt;&gt;
> &gt;&gt;
> &gt;&gt; Best regards,
> &gt;&gt; Minwoo Kang
> &gt;&gt;
> &gt;&gt; ________________________________________
> &gt;&gt; 보낸 사람: 张铎(Duo Zhang) <pa...@gmail.com>
> &gt;&gt; 보낸 날짜: 2018년 5월 23일 수요일 09:22
> &gt;&gt; 받는 사람: hbase-user
> &gt;&gt; 제목: Re: can not write to HBase
> &gt;&gt;
> &gt;&gt; What is the exception? And the stacktrace?
> &gt;&gt;
> &gt;&gt; 2018-05-23 8:17 GMT+08:00 Kang Minwoo <mi...@outlook.com>:
> &gt;&gt;
> &gt;&gt;&gt; Hello, Users
> &gt;&gt;&gt;
> &gt;&gt;&gt; My HBase client does not work after print the following logs.
> &gt;&gt;&gt;
> &gt;&gt;&gt; Call exception, tries=23, retries=35, started=291277 ms ago,
> &gt;&gt;&gt; cancelled=false, msg=row '{row}' on table '{table}' at
> region={region},
> &gt;&gt;&gt; hostname={hostname}, seqNum=100353531
> &gt;&gt;&gt;
> &gt;&gt;&gt; There are no special logs in the Master and Region Servers.
> &gt;&gt;&gt; Is it something wrong in the client?
> &gt;&gt;&gt;
> &gt;&gt;&gt; Best regards,
> &gt;&gt;&gt; Minwoo Kang
> &gt;&gt;&gt;
> &gt;&gt;
> </m...@gmail.com></minwoo.
> kang@outlook.com></p...@outlook.com></
> anilgupta84@gmail.com></m...@outlook.com>

Re: can not write to HBase

Posted by Kang Minwoo <mi...@outlook.com>.
@Duo Zhang
This means that you're writing too fast and memstore has reached its upper limit. Is the flush and compaction fine at RS side?

-> No, flush took very long time.
I attach code that took a long time to run. (about 30min)

https://github.com/apache/hbase/blob/branch-1.2/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/HRegion.java#L2424-L2508

Best regards,
Minwoo Kang
________________________________________
보낸 사람: Kang Minwoo <mi...@outlook.com>
보낸 날짜: 2018년 5월 23일 수요일 15:16
받는 사람: user@hbase.apache.org
제목: Re: can not write to HBase

I am using salt, prevent write hotspot.
And Table has 4000 regions.

HBase version is 1.2.6.

I attach code that took a long time to run. (about 30min)
https://github.com/apache/hbase/blob/branch-1.2/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/HRegion.java#L2424-L2508

Best regards,
Minwoo Kang

________________________________________
보낸 사람: Anil Gupta <an...@gmail.com>
보낸 날짜: 2018년 5월 23일 수요일 14:13
받는 사람: user@hbase.apache.org
제목: Re: can not write to HBase

It seems you might have a write hotspot.
Are your writes evenly distributed across the cluster? Do you have more than 15-20 regions for that table?

Sent from my iPhone

&gt; On May 22, 2018, at 9:52 PM, Kang Minwoo <mi...@outlook.com> wrote:
&gt;
&gt; I think hbase flush is too slow.
&gt; so memstore reached upper limit.
&gt;
&gt; flush took about 30min.
&gt; I don't know why flush is too long.
&gt;
&gt; Best regards,
&gt; Minwoo Kang
&gt;
&gt; ________________________________________
&gt; 보낸 사람: 张铎(Duo Zhang) <pa...@gmail.com>
&gt; 보낸 날짜: 2018년 5월 23일 수요일 11:37
&gt; 받는 사람: hbase-user
&gt; 제목: Re: can not write to HBase
&gt;
&gt; org.apache.hadoop.hbase.RegionTooBusyException:
&gt; org.apache.hadoop.hbase.RegionTooBusyException:
&gt; Above memstore limit, regionName={region}, server={server},
&gt; memstoreSize=2600502128, blockingMemStoreSize=2600468480
&gt;
&gt; This means that you're writing too fast and memstore has reached its upper
&gt; limit. Is the flush and compaction fine at RS side?
&gt;
&gt; 2018-05-23 10:20 GMT+08:00 Kang Minwoo <mi...@outlook.com>:
&gt;
&gt;&gt; attach client exception and stacktrace.
&gt;&gt;
&gt;&gt; I've looked more.
&gt;&gt; It seems to be the reason why it takes 1290 seconds to flush in the Region
&gt;&gt; Server.
&gt;&gt;
&gt;&gt; 2018-05-23T07:24:31.202 [INFO] Call exception, tries=34, retries=35,
&gt;&gt; started=513393 ms ago, cancelled=false, msg=row '{row}' on table '{table}'
&gt;&gt; at region={region}, hostname={host}, seqNum=155455658
&gt;&gt; 2018-05-23T07:24:31.208 [ERROR]
&gt;&gt; java.lang.RuntimeException: com.google.protobuf.ServiceException: Error
&gt;&gt; calling method MultiRowMutationService.MutateRows
&gt;&gt;        at com.google.common.base.Throwables.propagate(Throwables.java:160)
&gt;&gt; ~[stormjar.jar:?]
&gt;&gt;        at ...
&gt;&gt;        at org.apache.storm.daemon.executor$fn__8058$tuple_
&gt;&gt; action_fn__8060.invoke(executor.clj:731) [storm-core-1.0.2.jar:1.0.2]
&gt;&gt;        at org.apache.storm.daemon.executor$mk_task_receiver$fn__7979.invoke(executor.clj:464)
&gt;&gt; [storm-core-1.0.2.jar:1.0.2]
&gt;&gt;        at org.apache.storm.disruptor$clojure_handler$reify__7492.onEvent(disruptor.clj:40)
&gt;&gt; [storm-core-1.0.2.jar:1.0.2]
&gt;&gt;        at org.apache.storm.utils.DisruptorQueue.consumeBatchToCursor(DisruptorQueue.java:451)
&gt;&gt; [storm-core-1.0.2.jar:1.0.2]
&gt;&gt;        at org.apache.storm.utils.DisruptorQueue.
&gt;&gt; consumeBatchWhenAvailable(DisruptorQueue.java:430)
&gt;&gt; [storm-core-1.0.2.jar:1.0.2]
&gt;&gt;        at org.apache.storm.disruptor$consume_batch_when_available.invoke(disruptor.clj:73)
&gt;&gt; [storm-core-1.0.2.jar:1.0.2]
&gt;&gt;        at org.apache.storm.daemon.executor$fn__8058$fn__8071$fn__8124.invoke(executor.clj:850)
&gt;&gt; [storm-core-1.0.2.jar:1.0.2]
&gt;&gt;        at org.apache.storm.util$async_loop$fn__624.invoke(util.clj:484)
&gt;&gt; [storm-core-1.0.2.jar:1.0.2]
&gt;&gt;        at clojure.lang.AFn.run(AFn.java:22) [clojure-1.7.0.jar:?]
&gt;&gt;        at java.lang.Thread.run(Thread.java:745) [?:1.7.0_80]
&gt;&gt; Caused by: com.google.protobuf.ServiceException: Error calling method
&gt;&gt; MultiRowMutationService.MutateRows
&gt;&gt;        at org.apache.hadoop.hbase.ipc.CoprocessorRpcChannel.
&gt;&gt; callBlockingMethod(CoprocessorRpcChannel.java:75) ~[stormjar.jar:?]
&gt;&gt;        at org.apache.hadoop.hbase.protobuf.generated.
&gt;&gt; MultiRowMutationProtos$MultiRowMutationService$BlockingStub.mutateRows(
&gt;&gt; MultiRowMutationProtos.java:2149) ~[stormjar.jar:?]
&gt;&gt;        at ...
&gt;&gt;        ... 13 more
&gt;&gt; Caused by: org.apache.hadoop.hbase.client.RetriesExhaustedException:
&gt;&gt; Failed after attempts=35, exceptions:
&gt;&gt; Wed May 23 07:15:57 KST 2018, RpcRetryingCaller{globalStartTime=1527027357808,
&gt;&gt; pause=100, retries=35}, org.apache.hadoop.hbase.RegionTooBusyException:
&gt;&gt; org.apache.hadoop.hbase.RegionTooBusyException: Above memstore limit,
&gt;&gt; regionName={region}, server={server}, memstoreSize=2600502128,
&gt;&gt; blockingMemStoreSize=2600468480
&gt;&gt;        at org.apache.hadoop.hbase.regionserver.HRegion.
&gt;&gt; checkResources(HRegion.java:3649)
&gt;&gt;        at org.apache.hadoop.hbase.regionserver.HRegion.
&gt;&gt; processRowsWithLocks(HRegion.java:6935)
&gt;&gt;        at org.apache.hadoop.hbase.regionserver.HRegion.
&gt;&gt; mutateRowsWithLocks(HRegion.java:6885)
&gt;&gt;        at org.apache.hadoop.hbase.coprocessor.MultiRowMutationEndpoint.
&gt;&gt; mutateRows(MultiRowMutationEndpoint.java:116)
&gt;&gt;        at org.apache.hadoop.hbase.protobuf.generated.
&gt;&gt; MultiRowMutationProtos$MultiRowMutationService.callMethod(
&gt;&gt; MultiRowMutationProtos.java:2053)
&gt;&gt;        at org.apache.hadoop.hbase.regionserver.HRegion.
&gt;&gt; execService(HRegion.java:7875)
&gt;&gt;        at org.apache.hadoop.hbase.regionserver.RSRpcServices.
&gt;&gt; execServiceOnRegion(RSRpcServices.java:2008)
&gt;&gt;
&gt;&gt;
&gt;&gt; Best regards,
&gt;&gt; Minwoo Kang
&gt;&gt;
&gt;&gt; ________________________________________
&gt;&gt; 보낸 사람: 张铎(Duo Zhang) <pa...@gmail.com>
&gt;&gt; 보낸 날짜: 2018년 5월 23일 수요일 09:22
&gt;&gt; 받는 사람: hbase-user
&gt;&gt; 제목: Re: can not write to HBase
&gt;&gt;
&gt;&gt; What is the exception? And the stacktrace?
&gt;&gt;
&gt;&gt; 2018-05-23 8:17 GMT+08:00 Kang Minwoo <mi...@outlook.com>:
&gt;&gt;
&gt;&gt;&gt; Hello, Users
&gt;&gt;&gt;
&gt;&gt;&gt; My HBase client does not work after print the following logs.
&gt;&gt;&gt;
&gt;&gt;&gt; Call exception, tries=23, retries=35, started=291277 ms ago,
&gt;&gt;&gt; cancelled=false, msg=row '{row}' on table '{table}' at region={region},
&gt;&gt;&gt; hostname={hostname}, seqNum=100353531
&gt;&gt;&gt;
&gt;&gt;&gt; There are no special logs in the Master and Region Servers.
&gt;&gt;&gt; Is it something wrong in the client?
&gt;&gt;&gt;
&gt;&gt;&gt; Best regards,
&gt;&gt;&gt; Minwoo Kang
&gt;&gt;&gt;
&gt;&gt;
</m...@outlook.com>

Re: can not write to HBase

Posted by Kang Minwoo <mi...@outlook.com>.
I am using salt, prevent write hotspot.
And Table has 4000 regions.

HBase version is 1.2.6.

I attach code that took a long time to run. (about 30min)
https://github.com/apache/hbase/blob/branch-1.2/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/HRegion.java#L2424-L2508

Best regards,
Minwoo Kang

________________________________________
보낸 사람: Anil Gupta <an...@gmail.com>
보낸 날짜: 2018년 5월 23일 수요일 14:13
받는 사람: user@hbase.apache.org
제목: Re: can not write to HBase

It seems you might have a write hotspot.
Are your writes evenly distributed across the cluster? Do you have more than 15-20 regions for that table?

Sent from my iPhone

> On May 22, 2018, at 9:52 PM, Kang Minwoo <mi...@outlook.com> wrote:
>
> I think hbase flush is too slow.
> so memstore reached upper limit.
>
> flush took about 30min.
> I don't know why flush is too long.
>
> Best regards,
> Minwoo Kang
>
> ________________________________________
> 보낸 사람: 张铎(Duo Zhang) <pa...@gmail.com>
> 보낸 날짜: 2018년 5월 23일 수요일 11:37
> 받는 사람: hbase-user
> 제목: Re: can not write to HBase
>
> org.apache.hadoop.hbase.RegionTooBusyException:
> org.apache.hadoop.hbase.RegionTooBusyException:
> Above memstore limit, regionName={region}, server={server},
> memstoreSize=2600502128, blockingMemStoreSize=2600468480
>
> This means that you're writing too fast and memstore has reached its upper
> limit. Is the flush and compaction fine at RS side?
>
> 2018-05-23 10:20 GMT+08:00 Kang Minwoo <mi...@outlook.com>:
>
>> attach client exception and stacktrace.
>>
>> I've looked more.
>> It seems to be the reason why it takes 1290 seconds to flush in the Region
>> Server.
>>
>> 2018-05-23T07:24:31.202 [INFO] Call exception, tries=34, retries=35,
>> started=513393 ms ago, cancelled=false, msg=row '{row}' on table '{table}'
>> at region={region}, hostname={host}, seqNum=155455658
>> 2018-05-23T07:24:31.208 [ERROR]
>> java.lang.RuntimeException: com.google.protobuf.ServiceException: Error
>> calling method MultiRowMutationService.MutateRows
>>        at com.google.common.base.Throwables.propagate(Throwables.java:160)
>> ~[stormjar.jar:?]
>>        at ...
>>        at org.apache.storm.daemon.executor$fn__8058$tuple_
>> action_fn__8060.invoke(executor.clj:731) [storm-core-1.0.2.jar:1.0.2]
>>        at org.apache.storm.daemon.executor$mk_task_receiver$fn__7979.invoke(executor.clj:464)
>> [storm-core-1.0.2.jar:1.0.2]
>>        at org.apache.storm.disruptor$clojure_handler$reify__7492.onEvent(disruptor.clj:40)
>> [storm-core-1.0.2.jar:1.0.2]
>>        at org.apache.storm.utils.DisruptorQueue.consumeBatchToCursor(DisruptorQueue.java:451)
>> [storm-core-1.0.2.jar:1.0.2]
>>        at org.apache.storm.utils.DisruptorQueue.
>> consumeBatchWhenAvailable(DisruptorQueue.java:430)
>> [storm-core-1.0.2.jar:1.0.2]
>>        at org.apache.storm.disruptor$consume_batch_when_available.invoke(disruptor.clj:73)
>> [storm-core-1.0.2.jar:1.0.2]
>>        at org.apache.storm.daemon.executor$fn__8058$fn__8071$fn__8124.invoke(executor.clj:850)
>> [storm-core-1.0.2.jar:1.0.2]
>>        at org.apache.storm.util$async_loop$fn__624.invoke(util.clj:484)
>> [storm-core-1.0.2.jar:1.0.2]
>>        at clojure.lang.AFn.run(AFn.java:22) [clojure-1.7.0.jar:?]
>>        at java.lang.Thread.run(Thread.java:745) [?:1.7.0_80]
>> Caused by: com.google.protobuf.ServiceException: Error calling method
>> MultiRowMutationService.MutateRows
>>        at org.apache.hadoop.hbase.ipc.CoprocessorRpcChannel.
>> callBlockingMethod(CoprocessorRpcChannel.java:75) ~[stormjar.jar:?]
>>        at org.apache.hadoop.hbase.protobuf.generated.
>> MultiRowMutationProtos$MultiRowMutationService$BlockingStub.mutateRows(
>> MultiRowMutationProtos.java:2149) ~[stormjar.jar:?]
>>        at ...
>>        ... 13 more
>> Caused by: org.apache.hadoop.hbase.client.RetriesExhaustedException:
>> Failed after attempts=35, exceptions:
>> Wed May 23 07:15:57 KST 2018, RpcRetryingCaller{globalStartTime=1527027357808,
>> pause=100, retries=35}, org.apache.hadoop.hbase.RegionTooBusyException:
>> org.apache.hadoop.hbase.RegionTooBusyException: Above memstore limit,
>> regionName={region}, server={server}, memstoreSize=2600502128,
>> blockingMemStoreSize=2600468480
>>        at org.apache.hadoop.hbase.regionserver.HRegion.
>> checkResources(HRegion.java:3649)
>>        at org.apache.hadoop.hbase.regionserver.HRegion.
>> processRowsWithLocks(HRegion.java:6935)
>>        at org.apache.hadoop.hbase.regionserver.HRegion.
>> mutateRowsWithLocks(HRegion.java:6885)
>>        at org.apache.hadoop.hbase.coprocessor.MultiRowMutationEndpoint.
>> mutateRows(MultiRowMutationEndpoint.java:116)
>>        at org.apache.hadoop.hbase.protobuf.generated.
>> MultiRowMutationProtos$MultiRowMutationService.callMethod(
>> MultiRowMutationProtos.java:2053)
>>        at org.apache.hadoop.hbase.regionserver.HRegion.
>> execService(HRegion.java:7875)
>>        at org.apache.hadoop.hbase.regionserver.RSRpcServices.
>> execServiceOnRegion(RSRpcServices.java:2008)
>>
>>
>> Best regards,
>> Minwoo Kang
>>
>> ________________________________________
>> 보낸 사람: 张铎(Duo Zhang) <pa...@gmail.com>
>> 보낸 날짜: 2018년 5월 23일 수요일 09:22
>> 받는 사람: hbase-user
>> 제목: Re: can not write to HBase
>>
>> What is the exception? And the stacktrace?
>>
>> 2018-05-23 8:17 GMT+08:00 Kang Minwoo <mi...@outlook.com>:
>>
>>> Hello, Users
>>>
>>> My HBase client does not work after print the following logs.
>>>
>>> Call exception, tries=23, retries=35, started=291277 ms ago,
>>> cancelled=false, msg=row '{row}' on table '{table}' at region={region},
>>> hostname={hostname}, seqNum=100353531
>>>
>>> There are no special logs in the Master and Region Servers.
>>> Is it something wrong in the client?
>>>
>>> Best regards,
>>> Minwoo Kang
>>>
>>

Re: can not write to HBase

Posted by Anil Gupta <an...@gmail.com>.
It seems you might have a write hotspot. 
Are your writes evenly distributed across the cluster? Do you have more than 15-20 regions for that table?

Sent from my iPhone

> On May 22, 2018, at 9:52 PM, Kang Minwoo <mi...@outlook.com> wrote:
> 
> I think hbase flush is too slow.
> so memstore reached upper limit.
> 
> flush took about 30min.
> I don't know why flush is too long.
> 
> Best regards,
> Minwoo Kang
> 
> ________________________________________
> 보낸 사람: 张铎(Duo Zhang) <pa...@gmail.com>
> 보낸 날짜: 2018년 5월 23일 수요일 11:37
> 받는 사람: hbase-user
> 제목: Re: can not write to HBase
> 
> org.apache.hadoop.hbase.RegionTooBusyException:
> org.apache.hadoop.hbase.RegionTooBusyException:
> Above memstore limit, regionName={region}, server={server},
> memstoreSize=2600502128, blockingMemStoreSize=2600468480
> 
> This means that you're writing too fast and memstore has reached its upper
> limit. Is the flush and compaction fine at RS side?
> 
> 2018-05-23 10:20 GMT+08:00 Kang Minwoo <mi...@outlook.com>:
> 
>> attach client exception and stacktrace.
>> 
>> I've looked more.
>> It seems to be the reason why it takes 1290 seconds to flush in the Region
>> Server.
>> 
>> 2018-05-23T07:24:31.202 [INFO] Call exception, tries=34, retries=35,
>> started=513393 ms ago, cancelled=false, msg=row '{row}' on table '{table}'
>> at region={region}, hostname={host}, seqNum=155455658
>> 2018-05-23T07:24:31.208 [ERROR]
>> java.lang.RuntimeException: com.google.protobuf.ServiceException: Error
>> calling method MultiRowMutationService.MutateRows
>>        at com.google.common.base.Throwables.propagate(Throwables.java:160)
>> ~[stormjar.jar:?]
>>        at ...
>>        at org.apache.storm.daemon.executor$fn__8058$tuple_
>> action_fn__8060.invoke(executor.clj:731) [storm-core-1.0.2.jar:1.0.2]
>>        at org.apache.storm.daemon.executor$mk_task_receiver$fn__7979.invoke(executor.clj:464)
>> [storm-core-1.0.2.jar:1.0.2]
>>        at org.apache.storm.disruptor$clojure_handler$reify__7492.onEvent(disruptor.clj:40)
>> [storm-core-1.0.2.jar:1.0.2]
>>        at org.apache.storm.utils.DisruptorQueue.consumeBatchToCursor(DisruptorQueue.java:451)
>> [storm-core-1.0.2.jar:1.0.2]
>>        at org.apache.storm.utils.DisruptorQueue.
>> consumeBatchWhenAvailable(DisruptorQueue.java:430)
>> [storm-core-1.0.2.jar:1.0.2]
>>        at org.apache.storm.disruptor$consume_batch_when_available.invoke(disruptor.clj:73)
>> [storm-core-1.0.2.jar:1.0.2]
>>        at org.apache.storm.daemon.executor$fn__8058$fn__8071$fn__8124.invoke(executor.clj:850)
>> [storm-core-1.0.2.jar:1.0.2]
>>        at org.apache.storm.util$async_loop$fn__624.invoke(util.clj:484)
>> [storm-core-1.0.2.jar:1.0.2]
>>        at clojure.lang.AFn.run(AFn.java:22) [clojure-1.7.0.jar:?]
>>        at java.lang.Thread.run(Thread.java:745) [?:1.7.0_80]
>> Caused by: com.google.protobuf.ServiceException: Error calling method
>> MultiRowMutationService.MutateRows
>>        at org.apache.hadoop.hbase.ipc.CoprocessorRpcChannel.
>> callBlockingMethod(CoprocessorRpcChannel.java:75) ~[stormjar.jar:?]
>>        at org.apache.hadoop.hbase.protobuf.generated.
>> MultiRowMutationProtos$MultiRowMutationService$BlockingStub.mutateRows(
>> MultiRowMutationProtos.java:2149) ~[stormjar.jar:?]
>>        at ...
>>        ... 13 more
>> Caused by: org.apache.hadoop.hbase.client.RetriesExhaustedException:
>> Failed after attempts=35, exceptions:
>> Wed May 23 07:15:57 KST 2018, RpcRetryingCaller{globalStartTime=1527027357808,
>> pause=100, retries=35}, org.apache.hadoop.hbase.RegionTooBusyException:
>> org.apache.hadoop.hbase.RegionTooBusyException: Above memstore limit,
>> regionName={region}, server={server}, memstoreSize=2600502128,
>> blockingMemStoreSize=2600468480
>>        at org.apache.hadoop.hbase.regionserver.HRegion.
>> checkResources(HRegion.java:3649)
>>        at org.apache.hadoop.hbase.regionserver.HRegion.
>> processRowsWithLocks(HRegion.java:6935)
>>        at org.apache.hadoop.hbase.regionserver.HRegion.
>> mutateRowsWithLocks(HRegion.java:6885)
>>        at org.apache.hadoop.hbase.coprocessor.MultiRowMutationEndpoint.
>> mutateRows(MultiRowMutationEndpoint.java:116)
>>        at org.apache.hadoop.hbase.protobuf.generated.
>> MultiRowMutationProtos$MultiRowMutationService.callMethod(
>> MultiRowMutationProtos.java:2053)
>>        at org.apache.hadoop.hbase.regionserver.HRegion.
>> execService(HRegion.java:7875)
>>        at org.apache.hadoop.hbase.regionserver.RSRpcServices.
>> execServiceOnRegion(RSRpcServices.java:2008)
>> 
>> 
>> Best regards,
>> Minwoo Kang
>> 
>> ________________________________________
>> 보낸 사람: 张铎(Duo Zhang) <pa...@gmail.com>
>> 보낸 날짜: 2018년 5월 23일 수요일 09:22
>> 받는 사람: hbase-user
>> 제목: Re: can not write to HBase
>> 
>> What is the exception? And the stacktrace?
>> 
>> 2018-05-23 8:17 GMT+08:00 Kang Minwoo <mi...@outlook.com>:
>> 
>>> Hello, Users
>>> 
>>> My HBase client does not work after print the following logs.
>>> 
>>> Call exception, tries=23, retries=35, started=291277 ms ago,
>>> cancelled=false, msg=row '{row}' on table '{table}' at region={region},
>>> hostname={hostname}, seqNum=100353531
>>> 
>>> There are no special logs in the Master and Region Servers.
>>> Is it something wrong in the client?
>>> 
>>> Best regards,
>>> Minwoo Kang
>>> 
>> 

Re: can not write to HBase

Posted by Kang Minwoo <mi...@outlook.com>.
I think hbase flush is too slow.
so memstore reached upper limit.

flush took about 30min.
I don't know why flush is too long.

Best regards,
Minwoo Kang

________________________________________
보낸 사람: 张铎(Duo Zhang) <pa...@gmail.com>
보낸 날짜: 2018년 5월 23일 수요일 11:37
받는 사람: hbase-user
제목: Re: can not write to HBase

 org.apache.hadoop.hbase.RegionTooBusyException:
org.apache.hadoop.hbase.RegionTooBusyException:
Above memstore limit, regionName={region}, server={server},
memstoreSize=2600502128, blockingMemStoreSize=2600468480

This means that you're writing too fast and memstore has reached its upper
limit. Is the flush and compaction fine at RS side?

2018-05-23 10:20 GMT+08:00 Kang Minwoo <mi...@outlook.com>:

> attach client exception and stacktrace.
>
> I've looked more.
> It seems to be the reason why it takes 1290 seconds to flush in the Region
> Server.
>
> 2018-05-23T07:24:31.202 [INFO] Call exception, tries=34, retries=35,
> started=513393 ms ago, cancelled=false, msg=row '{row}' on table '{table}'
> at region={region}, hostname={host}, seqNum=155455658
> 2018-05-23T07:24:31.208 [ERROR]
> java.lang.RuntimeException: com.google.protobuf.ServiceException: Error
> calling method MultiRowMutationService.MutateRows
>         at com.google.common.base.Throwables.propagate(Throwables.java:160)
> ~[stormjar.jar:?]
>         at ...
>         at org.apache.storm.daemon.executor$fn__8058$tuple_
> action_fn__8060.invoke(executor.clj:731) [storm-core-1.0.2.jar:1.0.2]
>         at org.apache.storm.daemon.executor$mk_task_receiver$fn__7979.invoke(executor.clj:464)
> [storm-core-1.0.2.jar:1.0.2]
>         at org.apache.storm.disruptor$clojure_handler$reify__7492.onEvent(disruptor.clj:40)
> [storm-core-1.0.2.jar:1.0.2]
>         at org.apache.storm.utils.DisruptorQueue.consumeBatchToCursor(DisruptorQueue.java:451)
> [storm-core-1.0.2.jar:1.0.2]
>         at org.apache.storm.utils.DisruptorQueue.
> consumeBatchWhenAvailable(DisruptorQueue.java:430)
> [storm-core-1.0.2.jar:1.0.2]
>         at org.apache.storm.disruptor$consume_batch_when_available.invoke(disruptor.clj:73)
> [storm-core-1.0.2.jar:1.0.2]
>         at org.apache.storm.daemon.executor$fn__8058$fn__8071$fn__8124.invoke(executor.clj:850)
> [storm-core-1.0.2.jar:1.0.2]
>         at org.apache.storm.util$async_loop$fn__624.invoke(util.clj:484)
> [storm-core-1.0.2.jar:1.0.2]
>         at clojure.lang.AFn.run(AFn.java:22) [clojure-1.7.0.jar:?]
>         at java.lang.Thread.run(Thread.java:745) [?:1.7.0_80]
> Caused by: com.google.protobuf.ServiceException: Error calling method
> MultiRowMutationService.MutateRows
>         at org.apache.hadoop.hbase.ipc.CoprocessorRpcChannel.
> callBlockingMethod(CoprocessorRpcChannel.java:75) ~[stormjar.jar:?]
>         at org.apache.hadoop.hbase.protobuf.generated.
> MultiRowMutationProtos$MultiRowMutationService$BlockingStub.mutateRows(
> MultiRowMutationProtos.java:2149) ~[stormjar.jar:?]
>         at ...
>         ... 13 more
> Caused by: org.apache.hadoop.hbase.client.RetriesExhaustedException:
> Failed after attempts=35, exceptions:
> Wed May 23 07:15:57 KST 2018, RpcRetryingCaller{globalStartTime=1527027357808,
> pause=100, retries=35}, org.apache.hadoop.hbase.RegionTooBusyException:
> org.apache.hadoop.hbase.RegionTooBusyException: Above memstore limit,
> regionName={region}, server={server}, memstoreSize=2600502128,
> blockingMemStoreSize=2600468480
>         at org.apache.hadoop.hbase.regionserver.HRegion.
> checkResources(HRegion.java:3649)
>         at org.apache.hadoop.hbase.regionserver.HRegion.
> processRowsWithLocks(HRegion.java:6935)
>         at org.apache.hadoop.hbase.regionserver.HRegion.
> mutateRowsWithLocks(HRegion.java:6885)
>         at org.apache.hadoop.hbase.coprocessor.MultiRowMutationEndpoint.
> mutateRows(MultiRowMutationEndpoint.java:116)
>         at org.apache.hadoop.hbase.protobuf.generated.
> MultiRowMutationProtos$MultiRowMutationService.callMethod(
> MultiRowMutationProtos.java:2053)
>         at org.apache.hadoop.hbase.regionserver.HRegion.
> execService(HRegion.java:7875)
>         at org.apache.hadoop.hbase.regionserver.RSRpcServices.
> execServiceOnRegion(RSRpcServices.java:2008)
>
>
> Best regards,
> Minwoo Kang
>
> ________________________________________
> 보낸 사람: 张铎(Duo Zhang) <pa...@gmail.com>
> 보낸 날짜: 2018년 5월 23일 수요일 09:22
> 받는 사람: hbase-user
> 제목: Re: can not write to HBase
>
> What is the exception? And the stacktrace?
>
> 2018-05-23 8:17 GMT+08:00 Kang Minwoo <mi...@outlook.com>:
>
> > Hello, Users
> >
> > My HBase client does not work after print the following logs.
> >
> > Call exception, tries=23, retries=35, started=291277 ms ago,
> > cancelled=false, msg=row '{row}' on table '{table}' at region={region},
> > hostname={hostname}, seqNum=100353531
> >
> > There are no special logs in the Master and Region Servers.
> > Is it something wrong in the client?
> >
> > Best regards,
> > Minwoo Kang
> >
>

Re: can not write to HBase

Posted by "张铎 (Duo Zhang)" <pa...@gmail.com>.
 org.apache.hadoop.hbase.RegionTooBusyException:
org.apache.hadoop.hbase.RegionTooBusyException:
Above memstore limit, regionName={region}, server={server},
memstoreSize=2600502128, blockingMemStoreSize=2600468480

This means that you're writing too fast and memstore has reached its upper
limit. Is the flush and compaction fine at RS side?

2018-05-23 10:20 GMT+08:00 Kang Minwoo <mi...@outlook.com>:

> attach client exception and stacktrace.
>
> I've looked more.
> It seems to be the reason why it takes 1290 seconds to flush in the Region
> Server.
>
> 2018-05-23T07:24:31.202 [INFO] Call exception, tries=34, retries=35,
> started=513393 ms ago, cancelled=false, msg=row '{row}' on table '{table}'
> at region={region}, hostname={host}, seqNum=155455658
> 2018-05-23T07:24:31.208 [ERROR]
> java.lang.RuntimeException: com.google.protobuf.ServiceException: Error
> calling method MultiRowMutationService.MutateRows
>         at com.google.common.base.Throwables.propagate(Throwables.java:160)
> ~[stormjar.jar:?]
>         at ...
>         at org.apache.storm.daemon.executor$fn__8058$tuple_
> action_fn__8060.invoke(executor.clj:731) [storm-core-1.0.2.jar:1.0.2]
>         at org.apache.storm.daemon.executor$mk_task_receiver$fn__7979.invoke(executor.clj:464)
> [storm-core-1.0.2.jar:1.0.2]
>         at org.apache.storm.disruptor$clojure_handler$reify__7492.onEvent(disruptor.clj:40)
> [storm-core-1.0.2.jar:1.0.2]
>         at org.apache.storm.utils.DisruptorQueue.consumeBatchToCursor(DisruptorQueue.java:451)
> [storm-core-1.0.2.jar:1.0.2]
>         at org.apache.storm.utils.DisruptorQueue.
> consumeBatchWhenAvailable(DisruptorQueue.java:430)
> [storm-core-1.0.2.jar:1.0.2]
>         at org.apache.storm.disruptor$consume_batch_when_available.invoke(disruptor.clj:73)
> [storm-core-1.0.2.jar:1.0.2]
>         at org.apache.storm.daemon.executor$fn__8058$fn__8071$fn__8124.invoke(executor.clj:850)
> [storm-core-1.0.2.jar:1.0.2]
>         at org.apache.storm.util$async_loop$fn__624.invoke(util.clj:484)
> [storm-core-1.0.2.jar:1.0.2]
>         at clojure.lang.AFn.run(AFn.java:22) [clojure-1.7.0.jar:?]
>         at java.lang.Thread.run(Thread.java:745) [?:1.7.0_80]
> Caused by: com.google.protobuf.ServiceException: Error calling method
> MultiRowMutationService.MutateRows
>         at org.apache.hadoop.hbase.ipc.CoprocessorRpcChannel.
> callBlockingMethod(CoprocessorRpcChannel.java:75) ~[stormjar.jar:?]
>         at org.apache.hadoop.hbase.protobuf.generated.
> MultiRowMutationProtos$MultiRowMutationService$BlockingStub.mutateRows(
> MultiRowMutationProtos.java:2149) ~[stormjar.jar:?]
>         at ...
>         ... 13 more
> Caused by: org.apache.hadoop.hbase.client.RetriesExhaustedException:
> Failed after attempts=35, exceptions:
> Wed May 23 07:15:57 KST 2018, RpcRetryingCaller{globalStartTime=1527027357808,
> pause=100, retries=35}, org.apache.hadoop.hbase.RegionTooBusyException:
> org.apache.hadoop.hbase.RegionTooBusyException: Above memstore limit,
> regionName={region}, server={server}, memstoreSize=2600502128,
> blockingMemStoreSize=2600468480
>         at org.apache.hadoop.hbase.regionserver.HRegion.
> checkResources(HRegion.java:3649)
>         at org.apache.hadoop.hbase.regionserver.HRegion.
> processRowsWithLocks(HRegion.java:6935)
>         at org.apache.hadoop.hbase.regionserver.HRegion.
> mutateRowsWithLocks(HRegion.java:6885)
>         at org.apache.hadoop.hbase.coprocessor.MultiRowMutationEndpoint.
> mutateRows(MultiRowMutationEndpoint.java:116)
>         at org.apache.hadoop.hbase.protobuf.generated.
> MultiRowMutationProtos$MultiRowMutationService.callMethod(
> MultiRowMutationProtos.java:2053)
>         at org.apache.hadoop.hbase.regionserver.HRegion.
> execService(HRegion.java:7875)
>         at org.apache.hadoop.hbase.regionserver.RSRpcServices.
> execServiceOnRegion(RSRpcServices.java:2008)
>
>
> Best regards,
> Minwoo Kang
>
> ________________________________________
> 보낸 사람: 张铎(Duo Zhang) <pa...@gmail.com>
> 보낸 날짜: 2018년 5월 23일 수요일 09:22
> 받는 사람: hbase-user
> 제목: Re: can not write to HBase
>
> What is the exception? And the stacktrace?
>
> 2018-05-23 8:17 GMT+08:00 Kang Minwoo <mi...@outlook.com>:
>
> > Hello, Users
> >
> > My HBase client does not work after print the following logs.
> >
> > Call exception, tries=23, retries=35, started=291277 ms ago,
> > cancelled=false, msg=row '{row}' on table '{table}' at region={region},
> > hostname={hostname}, seqNum=100353531
> >
> > There are no special logs in the Master and Region Servers.
> > Is it something wrong in the client?
> >
> > Best regards,
> > Minwoo Kang
> >
>

Re: can not write to HBase

Posted by Kang Minwoo <mi...@outlook.com>.
attach client exception and stacktrace.

I've looked more.
It seems to be the reason why it takes 1290 seconds to flush in the Region Server.

2018-05-23T07:24:31.202 [INFO] Call exception, tries=34, retries=35, started=513393 ms ago, cancelled=false, msg=row '{row}' on table '{table}' at region={region}, hostname={host}, seqNum=155455658
2018-05-23T07:24:31.208 [ERROR]
java.lang.RuntimeException: com.google.protobuf.ServiceException: Error calling method MultiRowMutationService.MutateRows
        at com.google.common.base.Throwables.propagate(Throwables.java:160) ~[stormjar.jar:?]
        at ...
        at org.apache.storm.daemon.executor$fn__8058$tuple_action_fn__8060.invoke(executor.clj:731) [storm-core-1.0.2.jar:1.0.2]
        at org.apache.storm.daemon.executor$mk_task_receiver$fn__7979.invoke(executor.clj:464) [storm-core-1.0.2.jar:1.0.2]
        at org.apache.storm.disruptor$clojure_handler$reify__7492.onEvent(disruptor.clj:40) [storm-core-1.0.2.jar:1.0.2]
        at org.apache.storm.utils.DisruptorQueue.consumeBatchToCursor(DisruptorQueue.java:451) [storm-core-1.0.2.jar:1.0.2]
        at org.apache.storm.utils.DisruptorQueue.consumeBatchWhenAvailable(DisruptorQueue.java:430) [storm-core-1.0.2.jar:1.0.2]
        at org.apache.storm.disruptor$consume_batch_when_available.invoke(disruptor.clj:73) [storm-core-1.0.2.jar:1.0.2]
        at org.apache.storm.daemon.executor$fn__8058$fn__8071$fn__8124.invoke(executor.clj:850) [storm-core-1.0.2.jar:1.0.2]
        at org.apache.storm.util$async_loop$fn__624.invoke(util.clj:484) [storm-core-1.0.2.jar:1.0.2]
        at clojure.lang.AFn.run(AFn.java:22) [clojure-1.7.0.jar:?]
        at java.lang.Thread.run(Thread.java:745) [?:1.7.0_80]
Caused by: com.google.protobuf.ServiceException: Error calling method MultiRowMutationService.MutateRows
        at org.apache.hadoop.hbase.ipc.CoprocessorRpcChannel.callBlockingMethod(CoprocessorRpcChannel.java:75) ~[stormjar.jar:?]
        at org.apache.hadoop.hbase.protobuf.generated.MultiRowMutationProtos$MultiRowMutationService$BlockingStub.mutateRows(MultiRowMutationProtos.java:2149) ~[stormjar.jar:?]
        at ...
        ... 13 more
Caused by: org.apache.hadoop.hbase.client.RetriesExhaustedException: Failed after attempts=35, exceptions:
Wed May 23 07:15:57 KST 2018, RpcRetryingCaller{globalStartTime=1527027357808, pause=100, retries=35}, org.apache.hadoop.hbase.RegionTooBusyException: org.apache.hadoop.hbase.RegionTooBusyException: Above memstore limit, regionName={region}, server={server}, memstoreSize=2600502128, blockingMemStoreSize=2600468480
        at org.apache.hadoop.hbase.regionserver.HRegion.checkResources(HRegion.java:3649)
        at org.apache.hadoop.hbase.regionserver.HRegion.processRowsWithLocks(HRegion.java:6935)
        at org.apache.hadoop.hbase.regionserver.HRegion.mutateRowsWithLocks(HRegion.java:6885)
        at org.apache.hadoop.hbase.coprocessor.MultiRowMutationEndpoint.mutateRows(MultiRowMutationEndpoint.java:116)
        at org.apache.hadoop.hbase.protobuf.generated.MultiRowMutationProtos$MultiRowMutationService.callMethod(MultiRowMutationProtos.java:2053)
        at org.apache.hadoop.hbase.regionserver.HRegion.execService(HRegion.java:7875)
        at org.apache.hadoop.hbase.regionserver.RSRpcServices.execServiceOnRegion(RSRpcServices.java:2008)


Best regards,
Minwoo Kang

________________________________________
보낸 사람: 张铎(Duo Zhang) <pa...@gmail.com>
보낸 날짜: 2018년 5월 23일 수요일 09:22
받는 사람: hbase-user
제목: Re: can not write to HBase

What is the exception? And the stacktrace?

2018-05-23 8:17 GMT+08:00 Kang Minwoo <mi...@outlook.com>:

> Hello, Users
>
> My HBase client does not work after print the following logs.
>
> Call exception, tries=23, retries=35, started=291277 ms ago,
> cancelled=false, msg=row '{row}' on table '{table}' at region={region},
> hostname={hostname}, seqNum=100353531
>
> There are no special logs in the Master and Region Servers.
> Is it something wrong in the client?
>
> Best regards,
> Minwoo Kang
>

Re: can not write to HBase

Posted by "张铎 (Duo Zhang)" <pa...@gmail.com>.
What is the exception? And the stacktrace?

2018-05-23 8:17 GMT+08:00 Kang Minwoo <mi...@outlook.com>:

> Hello, Users
>
> My HBase client does not work after print the following logs.
>
> Call exception, tries=23, retries=35, started=291277 ms ago,
> cancelled=false, msg=row '{row}' on table '{table}' at region={region},
> hostname={hostname}, seqNum=100353531
>
> There are no special logs in the Master and Region Servers.
> Is it something wrong in the client?
>
> Best regards,
> Minwoo Kang
>