[jira] [Commented] (HBASE-14443) Add request parameter to the TooSlow/TooLarge warn message of RpcServer

2016-03-30 Thread Tomu Tsuruhara (JIRA)

[ 
https://issues.apache.org/jira/browse/HBASE-14443?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=15217665#comment-15217665
 ] 

Tomu Tsuruhara commented on HBASE-14443:


How is this going?
I also want this feature.
How about introducing a global switch like {{hbase.ipc.warn.response.verbose}}?

> Add request parameter to the TooSlow/TooLarge warn message of RpcServer
> ---
>
> Key: HBASE-14443
> URL: https://issues.apache.org/jira/browse/HBASE-14443
> Project: HBase
>  Issue Type: Improvement
>  Components: IPC/RPC, Operability
>Affects Versions: 1.2.1
>Reporter: Jianwei Cui
>Assignee: Jianwei Cui
>Priority: Minor
> Attachments: HBASE-14443-trunk-v1.patch, HBASE-14443-trunk-v2.patch
>
>
> The RpcServer will log a warn message for TooSlow or TooLarge request as:
> {code}
> logResponse(new Object[]{param},
> md.getName(), md.getName() + "(" + param.getClass().getName() + 
> ")",
> (tooLarge ? "TooLarge" : "TooSlow"),
> status.getClient(), startTime, processingTime, qTime,
> responseSize);
> {code}
> The RpcServer#logResponse will create the warn message as:
> {code}
> if (params.length == 2 && server instanceof HRegionServer &&
> params[0] instanceof byte[] &&
> params[1] instanceof Operation) {
>   ...
>   responseInfo.putAll(((Operation) params[1]).toMap());
>   ...
> } else if (params.length == 1 && server instanceof HRegionServer &&
> params[0] instanceof Operation) {
>   ...
>   responseInfo.putAll(((Operation) params[0]).toMap());
>   ...
> } else {
>   ...
> }
> {code}
> Because the parameter is always a protobuf message, not an instance of 
> Operation, the request parameter will not be added into the warn message. The 
> parameter is helpful to find out the problem, for example, knowing the 
> startRow/endRow is useful for a TooSlow scan. To improve the warn message, we 
> can transform the protobuf request message to corresponding Operation 
> subclass object by ProtobufUtil, so that it can be added the warn message. 
> Suggestion and discussion are welcomed.  



--
This message was sent by Atlassian JIRA
(v6.3.4#6332)


[jira] [Updated] (HBASE-16616) Rpc handlers stuck on ThreadLocalMap.expungeStaleEntry

2016-09-12 Thread Tomu Tsuruhara (JIRA)

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

Tomu Tsuruhara updated HBASE-16616:
---
Description: 
In our HBase 1.2.2 cluster, some regionserver showed too bad 
"QueueCallTime_99th_percentile" exceeding 10 seconds.
Most rpc handler threads stuck on ThreadLocalMap.expungeStaleEntry call at that 
time.

{noformat}
"PriorityRpcServer.handler=18,queue=0,port=16020" #322 daemon prio=5 os_prio=0 
tid=0x7fd422062800 nid=0x19b89 runnable [0x7fcb8a821000]
   java.lang.Thread.State: RUNNABLE
at 
java.lang.ThreadLocal$ThreadLocalMap.expungeStaleEntry(ThreadLocal.java:617)
at java.lang.ThreadLocal$ThreadLocalMap.remove(ThreadLocal.java:499)
at java.lang.ThreadLocal$ThreadLocalMap.access$200(ThreadLocal.java:298)
at java.lang.ThreadLocal.remove(ThreadLocal.java:222)
at 
java.util.concurrent.locks.ReentrantReadWriteLock$Sync.tryReleaseShared(ReentrantReadWriteLock.java:426)
at 
java.util.concurrent.locks.AbstractQueuedSynchronizer.releaseShared(AbstractQueuedSynchronizer.java:1341)
at 
java.util.concurrent.locks.ReentrantReadWriteLock$ReadLock.unlock(ReentrantReadWriteLock.java:881)
at 
com.yammer.metrics.stats.ExponentiallyDecayingSample.unlockForRegularUsage(ExponentiallyDecayingSample.java:196)
at 
com.yammer.metrics.stats.ExponentiallyDecayingSample.update(ExponentiallyDecayingSample.java:113)
at 
com.yammer.metrics.stats.ExponentiallyDecayingSample.update(ExponentiallyDecayingSample.java:81)
at 
org.apache.hadoop.metrics2.lib.MutableHistogram.add(MutableHistogram.java:81)
at 
org.apache.hadoop.metrics2.lib.MutableRangeHistogram.add(MutableRangeHistogram.java:59)
at 
org.apache.hadoop.hbase.ipc.MetricsHBaseServerSourceImpl.dequeuedCall(MetricsHBaseServerSourceImpl.java:194)
at 
org.apache.hadoop.hbase.ipc.MetricsHBaseServer.dequeuedCall(MetricsHBaseServer.java:76)
at org.apache.hadoop.hbase.ipc.RpcServer.call(RpcServer.java:2192)
at org.apache.hadoop.hbase.ipc.CallRunner.run(CallRunner.java:112)
at 
org.apache.hadoop.hbase.ipc.RpcExecutor.consumerLoop(RpcExecutor.java:133)
at org.apache.hadoop.hbase.ipc.RpcExecutor$1.run(RpcExecutor.java:108)
at java.lang.Thread.run(Thread.java:745)
{noformat}

We were using jdk 1.8.0_92 and here is a snippet from ThreadLocal.java.

{code}
616:while (tab[h] != null)
617:h = nextIndex(h, len);
{code}

So I hypothesized that there're too many consecutive entries in {{tab}} array 
and actually I found them in the heapdump.

!ScreenShot 2016-09-09 14.17.53.png|width=50%!

Most of these entries pointed at instance of 
{{org.apache.hadoop.hbase.util.Counter$1}}
which is equivarent to {{indexHolderThreadLocal}} instance-variable in the 
{{Counter}} class.

Because {{RpcServer$Connection}} class creates a {{Counter}} instance 
{{rpcCount}} for every connections,
it is possible to have lots of {{Counter#indexHolderThreadLocal}} instances in 
RegionServer process
when we repeat connect-and-close from client. As a result, a ThreadLocalMap can 
have lots of consecutive
entires.

Usually, since each entry is a {{WeakReference}}, these entries are collected 
and removed
by garbage-collector soon after connection closed.
But if connection's life-time was long enough to survive youngGC, it wouldn't 
be collected until old-gen collector runs.
Furthermore, under G1GC deployment, it is possible not to be collected even by 
old-gen GC(mixed GC)
if entries sit in a region which doesn't have much garbages.
Actually we used G1GC when we encountered this problem.

We should remove the entry from ThreadLocalMap by calling ThreadLocal#remove 
explicitly.

  was:
In our HBase 1.2.2 cluster, some regionserver showed too bad 
"QueueCallTime_99th_percentile" exceeding 10 seconds.
Most rpc handler threads stuck on ThreadLocalMap.expungeStaleEntry call at that 
time.

{noformat}
"PriorityRpcServer.handler=18,queue=0,port=16020" #322 daemon prio=5 os_prio=0 
tid=0x7fd422062800 nid=0x19b89 runnable [0x7fcb8a821000]
   java.lang.Thread.State: RUNNABLE
at 
java.lang.ThreadLocal$ThreadLocalMap.expungeStaleEntry(ThreadLocal.java:617)
at java.lang.ThreadLocal$ThreadLocalMap.remove(ThreadLocal.java:499)
at java.lang.ThreadLocal$ThreadLocalMap.access$200(ThreadLocal.java:298)
at java.lang.ThreadLocal.remove(ThreadLocal.java:222)
at 
java.util.concurrent.locks.ReentrantReadWriteLock$Sync.tryReleaseShared(ReentrantReadWriteLock.java:426)
at 
java.util.concurrent.locks.AbstractQueuedSynchronizer.releaseShared(AbstractQueuedSynchronizer.java:1341)
at 
java.util.concurrent.locks.ReentrantReadWriteLock$ReadLock.unlock(ReentrantReadWriteLock.java:881)
at 
com.yammer.metrics.stats.ExponentiallyDecayingSample.unlockForRegularUsage(ExponentiallyDecayingSample.java:196)
  

[jira] [Created] (HBASE-16616) Rpc handlers stuck on ThreadLocalMap.expungeStaleEntry

2016-09-12 Thread Tomu Tsuruhara (JIRA)
Tomu Tsuruhara created HBASE-16616:
--

 Summary: Rpc handlers stuck on ThreadLocalMap.expungeStaleEntry
 Key: HBASE-16616
 URL: https://issues.apache.org/jira/browse/HBASE-16616
 Project: HBase
  Issue Type: Improvement
  Components: Performance
Affects Versions: 1.2.2
Reporter: Tomu Tsuruhara


In our HBase 1.2.2 cluster, some regionserver showed too bad 
"QueueCallTime_99th_percentile" exceeding 10 seconds.
Most rpc handler threads stuck on ThreadLocalMap.expungeStaleEntry call at that 
time.

{noformat}
"PriorityRpcServer.handler=18,queue=0,port=16020" #322 daemon prio=5 os_prio=0 
tid=0x7fd422062800 nid=0x19b89 runnable [0x7fcb8a821000]
   java.lang.Thread.State: RUNNABLE
at 
java.lang.ThreadLocal$ThreadLocalMap.expungeStaleEntry(ThreadLocal.java:617)
at java.lang.ThreadLocal$ThreadLocalMap.remove(ThreadLocal.java:499)
at java.lang.ThreadLocal$ThreadLocalMap.access$200(ThreadLocal.java:298)
at java.lang.ThreadLocal.remove(ThreadLocal.java:222)
at 
java.util.concurrent.locks.ReentrantReadWriteLock$Sync.tryReleaseShared(ReentrantReadWriteLock.java:426)
at 
java.util.concurrent.locks.AbstractQueuedSynchronizer.releaseShared(AbstractQueuedSynchronizer.java:1341)
at 
java.util.concurrent.locks.ReentrantReadWriteLock$ReadLock.unlock(ReentrantReadWriteLock.java:881)
at 
com.yammer.metrics.stats.ExponentiallyDecayingSample.unlockForRegularUsage(ExponentiallyDecayingSample.java:196)
at 
com.yammer.metrics.stats.ExponentiallyDecayingSample.update(ExponentiallyDecayingSample.java:113)
at 
com.yammer.metrics.stats.ExponentiallyDecayingSample.update(ExponentiallyDecayingSample.java:81)
at 
org.apache.hadoop.metrics2.lib.MutableHistogram.add(MutableHistogram.java:81)
at 
org.apache.hadoop.metrics2.lib.MutableRangeHistogram.add(MutableRangeHistogram.java:59)
at 
org.apache.hadoop.hbase.ipc.MetricsHBaseServerSourceImpl.dequeuedCall(MetricsHBaseServerSourceImpl.java:194)
at 
org.apache.hadoop.hbase.ipc.MetricsHBaseServer.dequeuedCall(MetricsHBaseServer.java:76)
at org.apache.hadoop.hbase.ipc.RpcServer.call(RpcServer.java:2192)
at org.apache.hadoop.hbase.ipc.CallRunner.run(CallRunner.java:112)
at 
org.apache.hadoop.hbase.ipc.RpcExecutor.consumerLoop(RpcExecutor.java:133)
at org.apache.hadoop.hbase.ipc.RpcExecutor$1.run(RpcExecutor.java:108)
at java.lang.Thread.run(Thread.java:745)
{noformat}

We were using jdk 1.8.0_92 and here is a snippet from ThreadLocal.java.

{code}
616:while (tab[h] != null)
617:h = nextIndex(h, len);
{code}

So I hypothesized that there're too many consecutive entries in {{tab}} array 
and actually I found them in the heapdump.

Most of those entries pointed at instance of 
{{org.apache.hadoop.hbase.util.Counter$1}}
which is equivarent to {{indexHolderThreadLocal}} instance-variable in the 
{{Counter}} class.

Because {{RpcServer$Connection}} class creates a {{Counter}} instance 
{{rpcCount}} for every connections,
it is possible to have lots of {{Counter#indexHolderThreadLocal}} instances in 
RegionServer process
when we repeat connect-and-close from client. As a result, a ThreadLocalMap can 
have lots of consecutive
entires.

Usually, since each entry is a {{WeakReference}}, these entries are collected 
and removed
by garbage-collector soon after connection closed.
But if connection's life-time was long enough to survive youngGC, it wouldn't 
be collected until old-gen collector runs.
Furthermore, under G1GC deployment, it is possible not to be collected even by 
old-gen GC(mixed GC)
if entries sit in a region which doesn't have much garbages.
Actually we used G1GC when we encountered this problem.

We should remove the entry from ThreadLocalMap by calling ThreadLocal#remove 
explicitly.



--
This message was sent by Atlassian JIRA
(v6.3.4#6332)


[jira] [Updated] (HBASE-16616) Rpc handlers stuck on ThreadLocalMap.expungeStaleEntry

2016-09-12 Thread Tomu Tsuruhara (JIRA)

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

Tomu Tsuruhara updated HBASE-16616:
---
Description: 
In our HBase 1.2.2 cluster, some regionserver showed too bad 
"QueueCallTime_99th_percentile" exceeding 10 seconds.
Most rpc handler threads stuck on ThreadLocalMap.expungeStaleEntry call at that 
time.

{noformat}
"PriorityRpcServer.handler=18,queue=0,port=16020" #322 daemon prio=5 os_prio=0 
tid=0x7fd422062800 nid=0x19b89 runnable [0x7fcb8a821000]
   java.lang.Thread.State: RUNNABLE
at 
java.lang.ThreadLocal$ThreadLocalMap.expungeStaleEntry(ThreadLocal.java:617)
at java.lang.ThreadLocal$ThreadLocalMap.remove(ThreadLocal.java:499)
at java.lang.ThreadLocal$ThreadLocalMap.access$200(ThreadLocal.java:298)
at java.lang.ThreadLocal.remove(ThreadLocal.java:222)
at 
java.util.concurrent.locks.ReentrantReadWriteLock$Sync.tryReleaseShared(ReentrantReadWriteLock.java:426)
at 
java.util.concurrent.locks.AbstractQueuedSynchronizer.releaseShared(AbstractQueuedSynchronizer.java:1341)
at 
java.util.concurrent.locks.ReentrantReadWriteLock$ReadLock.unlock(ReentrantReadWriteLock.java:881)
at 
com.yammer.metrics.stats.ExponentiallyDecayingSample.unlockForRegularUsage(ExponentiallyDecayingSample.java:196)
at 
com.yammer.metrics.stats.ExponentiallyDecayingSample.update(ExponentiallyDecayingSample.java:113)
at 
com.yammer.metrics.stats.ExponentiallyDecayingSample.update(ExponentiallyDecayingSample.java:81)
at 
org.apache.hadoop.metrics2.lib.MutableHistogram.add(MutableHistogram.java:81)
at 
org.apache.hadoop.metrics2.lib.MutableRangeHistogram.add(MutableRangeHistogram.java:59)
at 
org.apache.hadoop.hbase.ipc.MetricsHBaseServerSourceImpl.dequeuedCall(MetricsHBaseServerSourceImpl.java:194)
at 
org.apache.hadoop.hbase.ipc.MetricsHBaseServer.dequeuedCall(MetricsHBaseServer.java:76)
at org.apache.hadoop.hbase.ipc.RpcServer.call(RpcServer.java:2192)
at org.apache.hadoop.hbase.ipc.CallRunner.run(CallRunner.java:112)
at 
org.apache.hadoop.hbase.ipc.RpcExecutor.consumerLoop(RpcExecutor.java:133)
at org.apache.hadoop.hbase.ipc.RpcExecutor$1.run(RpcExecutor.java:108)
at java.lang.Thread.run(Thread.java:745)
{noformat}

We were using jdk 1.8.0_92 and here is a snippet from ThreadLocal.java.

{code}
616:while (tab[h] != null)
617:h = nextIndex(h, len);
{code}

So I hypothesized that there're too many consecutive entries in {{tab}} array 
and actually I found them in the heapdump.


Most of those entries pointed at instance of 
{{org.apache.hadoop.hbase.util.Counter$1}}
which is equivarent to {{indexHolderThreadLocal}} instance-variable in the 
{{Counter}} class.

Because {{RpcServer$Connection}} class creates a {{Counter}} instance 
{{rpcCount}} for every connections,
it is possible to have lots of {{Counter#indexHolderThreadLocal}} instances in 
RegionServer process
when we repeat connect-and-close from client. As a result, a ThreadLocalMap can 
have lots of consecutive
entires.

Usually, since each entry is a {{WeakReference}}, these entries are collected 
and removed
by garbage-collector soon after connection closed.
But if connection's life-time was long enough to survive youngGC, it wouldn't 
be collected until old-gen collector runs.
Furthermore, under G1GC deployment, it is possible not to be collected even by 
old-gen GC(mixed GC)
if entries sit in a region which doesn't have much garbages.
Actually we used G1GC when we encountered this problem.

We should remove the entry from ThreadLocalMap by calling ThreadLocal#remove 
explicitly.

  was:
In our HBase 1.2.2 cluster, some regionserver showed too bad 
"QueueCallTime_99th_percentile" exceeding 10 seconds.
Most rpc handler threads stuck on ThreadLocalMap.expungeStaleEntry call at that 
time.

{noformat}
"PriorityRpcServer.handler=18,queue=0,port=16020" #322 daemon prio=5 os_prio=0 
tid=0x7fd422062800 nid=0x19b89 runnable [0x7fcb8a821000]
   java.lang.Thread.State: RUNNABLE
at 
java.lang.ThreadLocal$ThreadLocalMap.expungeStaleEntry(ThreadLocal.java:617)
at java.lang.ThreadLocal$ThreadLocalMap.remove(ThreadLocal.java:499)
at java.lang.ThreadLocal$ThreadLocalMap.access$200(ThreadLocal.java:298)
at java.lang.ThreadLocal.remove(ThreadLocal.java:222)
at 
java.util.concurrent.locks.ReentrantReadWriteLock$Sync.tryReleaseShared(ReentrantReadWriteLock.java:426)
at 
java.util.concurrent.locks.AbstractQueuedSynchronizer.releaseShared(AbstractQueuedSynchronizer.java:1341)
at 
java.util.concurrent.locks.ReentrantReadWriteLock$ReadLock.unlock(ReentrantReadWriteLock.java:881)
at 
com.yammer.metrics.stats.ExponentiallyDecayingSample.unlockForRegularUsage(ExponentiallyDecayingSample.java:196)
at 

[jira] [Updated] (HBASE-16616) Rpc handlers stuck on ThreadLocalMap.expungeStaleEntry

2016-09-12 Thread Tomu Tsuruhara (JIRA)

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

Tomu Tsuruhara updated HBASE-16616:
---
Attachment: ScreenShot 2016-09-09 14.17.53.png

> Rpc handlers stuck on ThreadLocalMap.expungeStaleEntry
> --
>
> Key: HBASE-16616
> URL: https://issues.apache.org/jira/browse/HBASE-16616
> Project: HBase
>  Issue Type: Improvement
>  Components: Performance
>Affects Versions: 1.2.2
>Reporter: Tomu Tsuruhara
> Attachments: ScreenShot 2016-09-09 14.17.53.png
>
>
> In our HBase 1.2.2 cluster, some regionserver showed too bad 
> "QueueCallTime_99th_percentile" exceeding 10 seconds.
> Most rpc handler threads stuck on ThreadLocalMap.expungeStaleEntry call at 
> that time.
> {noformat}
> "PriorityRpcServer.handler=18,queue=0,port=16020" #322 daemon prio=5 
> os_prio=0 tid=0x7fd422062800 nid=0x19b89 runnable [0x7fcb8a821000]
>java.lang.Thread.State: RUNNABLE
> at 
> java.lang.ThreadLocal$ThreadLocalMap.expungeStaleEntry(ThreadLocal.java:617)
> at java.lang.ThreadLocal$ThreadLocalMap.remove(ThreadLocal.java:499)
> at 
> java.lang.ThreadLocal$ThreadLocalMap.access$200(ThreadLocal.java:298)
> at java.lang.ThreadLocal.remove(ThreadLocal.java:222)
> at 
> java.util.concurrent.locks.ReentrantReadWriteLock$Sync.tryReleaseShared(ReentrantReadWriteLock.java:426)
> at 
> java.util.concurrent.locks.AbstractQueuedSynchronizer.releaseShared(AbstractQueuedSynchronizer.java:1341)
> at 
> java.util.concurrent.locks.ReentrantReadWriteLock$ReadLock.unlock(ReentrantReadWriteLock.java:881)
> at 
> com.yammer.metrics.stats.ExponentiallyDecayingSample.unlockForRegularUsage(ExponentiallyDecayingSample.java:196)
> at 
> com.yammer.metrics.stats.ExponentiallyDecayingSample.update(ExponentiallyDecayingSample.java:113)
> at 
> com.yammer.metrics.stats.ExponentiallyDecayingSample.update(ExponentiallyDecayingSample.java:81)
> at 
> org.apache.hadoop.metrics2.lib.MutableHistogram.add(MutableHistogram.java:81)
> at 
> org.apache.hadoop.metrics2.lib.MutableRangeHistogram.add(MutableRangeHistogram.java:59)
> at 
> org.apache.hadoop.hbase.ipc.MetricsHBaseServerSourceImpl.dequeuedCall(MetricsHBaseServerSourceImpl.java:194)
> at 
> org.apache.hadoop.hbase.ipc.MetricsHBaseServer.dequeuedCall(MetricsHBaseServer.java:76)
> at org.apache.hadoop.hbase.ipc.RpcServer.call(RpcServer.java:2192)
> at org.apache.hadoop.hbase.ipc.CallRunner.run(CallRunner.java:112)
> at 
> org.apache.hadoop.hbase.ipc.RpcExecutor.consumerLoop(RpcExecutor.java:133)
> at org.apache.hadoop.hbase.ipc.RpcExecutor$1.run(RpcExecutor.java:108)
> at java.lang.Thread.run(Thread.java:745)
> {noformat}
> We were using jdk 1.8.0_92 and here is a snippet from ThreadLocal.java.
> {code}
> 616:while (tab[h] != null)
> 617:h = nextIndex(h, len);
> {code}
> So I hypothesized that there're too many consecutive entries in {{tab}} array 
> and actually I found them in the heapdump.
> Most of those entries pointed at instance of 
> {{org.apache.hadoop.hbase.util.Counter$1}}
> which is equivarent to {{indexHolderThreadLocal}} instance-variable in the 
> {{Counter}} class.
> Because {{RpcServer$Connection}} class creates a {{Counter}} instance 
> {{rpcCount}} for every connections,
> it is possible to have lots of {{Counter#indexHolderThreadLocal}} instances 
> in RegionServer process
> when we repeat connect-and-close from client. As a result, a ThreadLocalMap 
> can have lots of consecutive
> entires.
> Usually, since each entry is a {{WeakReference}}, these entries are collected 
> and removed
> by garbage-collector soon after connection closed.
> But if connection's life-time was long enough to survive youngGC, it wouldn't 
> be collected until old-gen collector runs.
> Furthermore, under G1GC deployment, it is possible not to be collected even 
> by old-gen GC(mixed GC)
> if entries sit in a region which doesn't have much garbages.
> Actually we used G1GC when we encountered this problem.
> We should remove the entry from ThreadLocalMap by calling ThreadLocal#remove 
> explicitly.



--
This message was sent by Atlassian JIRA
(v6.3.4#6332)


[jira] [Updated] (HBASE-16616) Rpc handlers stuck on ThreadLocalMap.expungeStaleEntry

2016-09-12 Thread Tomu Tsuruhara (JIRA)

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

Tomu Tsuruhara updated HBASE-16616:
---
Description: 
In our HBase 1.2.2 cluster, some regionserver showed too bad 
"QueueCallTime_99th_percentile" exceeding 10 seconds.
Most rpc handler threads stuck on ThreadLocalMap.expungeStaleEntry call at that 
time.

{noformat}
"PriorityRpcServer.handler=18,queue=0,port=16020" #322 daemon prio=5 os_prio=0 
tid=0x7fd422062800 nid=0x19b89 runnable [0x7fcb8a821000]
   java.lang.Thread.State: RUNNABLE
at 
java.lang.ThreadLocal$ThreadLocalMap.expungeStaleEntry(ThreadLocal.java:617)
at java.lang.ThreadLocal$ThreadLocalMap.remove(ThreadLocal.java:499)
at java.lang.ThreadLocal$ThreadLocalMap.access$200(ThreadLocal.java:298)
at java.lang.ThreadLocal.remove(ThreadLocal.java:222)
at 
java.util.concurrent.locks.ReentrantReadWriteLock$Sync.tryReleaseShared(ReentrantReadWriteLock.java:426)
at 
java.util.concurrent.locks.AbstractQueuedSynchronizer.releaseShared(AbstractQueuedSynchronizer.java:1341)
at 
java.util.concurrent.locks.ReentrantReadWriteLock$ReadLock.unlock(ReentrantReadWriteLock.java:881)
at 
com.yammer.metrics.stats.ExponentiallyDecayingSample.unlockForRegularUsage(ExponentiallyDecayingSample.java:196)
at 
com.yammer.metrics.stats.ExponentiallyDecayingSample.update(ExponentiallyDecayingSample.java:113)
at 
com.yammer.metrics.stats.ExponentiallyDecayingSample.update(ExponentiallyDecayingSample.java:81)
at 
org.apache.hadoop.metrics2.lib.MutableHistogram.add(MutableHistogram.java:81)
at 
org.apache.hadoop.metrics2.lib.MutableRangeHistogram.add(MutableRangeHistogram.java:59)
at 
org.apache.hadoop.hbase.ipc.MetricsHBaseServerSourceImpl.dequeuedCall(MetricsHBaseServerSourceImpl.java:194)
at 
org.apache.hadoop.hbase.ipc.MetricsHBaseServer.dequeuedCall(MetricsHBaseServer.java:76)
at org.apache.hadoop.hbase.ipc.RpcServer.call(RpcServer.java:2192)
at org.apache.hadoop.hbase.ipc.CallRunner.run(CallRunner.java:112)
at 
org.apache.hadoop.hbase.ipc.RpcExecutor.consumerLoop(RpcExecutor.java:133)
at org.apache.hadoop.hbase.ipc.RpcExecutor$1.run(RpcExecutor.java:108)
at java.lang.Thread.run(Thread.java:745)
{noformat}

We were using jdk 1.8.0_92 and here is a snippet from ThreadLocal.java.

{code}
616:while (tab[h] != null)
617:h = nextIndex(h, len);
{code}

So I hypothesized that there're too many consecutive entries in {{tab}} array 
and actually I found them in the heapdump.

!ScreenShot 2016-09-09 14.17.53.png|thumbnail!

Most of those entries pointed at instance of 
{{org.apache.hadoop.hbase.util.Counter$1}}
which is equivarent to {{indexHolderThreadLocal}} instance-variable in the 
{{Counter}} class.

Because {{RpcServer$Connection}} class creates a {{Counter}} instance 
{{rpcCount}} for every connections,
it is possible to have lots of {{Counter#indexHolderThreadLocal}} instances in 
RegionServer process
when we repeat connect-and-close from client. As a result, a ThreadLocalMap can 
have lots of consecutive
entires.

Usually, since each entry is a {{WeakReference}}, these entries are collected 
and removed
by garbage-collector soon after connection closed.
But if connection's life-time was long enough to survive youngGC, it wouldn't 
be collected until old-gen collector runs.
Furthermore, under G1GC deployment, it is possible not to be collected even by 
old-gen GC(mixed GC)
if entries sit in a region which doesn't have much garbages.
Actually we used G1GC when we encountered this problem.

We should remove the entry from ThreadLocalMap by calling ThreadLocal#remove 
explicitly.

  was:
In our HBase 1.2.2 cluster, some regionserver showed too bad 
"QueueCallTime_99th_percentile" exceeding 10 seconds.
Most rpc handler threads stuck on ThreadLocalMap.expungeStaleEntry call at that 
time.

{noformat}
"PriorityRpcServer.handler=18,queue=0,port=16020" #322 daemon prio=5 os_prio=0 
tid=0x7fd422062800 nid=0x19b89 runnable [0x7fcb8a821000]
   java.lang.Thread.State: RUNNABLE
at 
java.lang.ThreadLocal$ThreadLocalMap.expungeStaleEntry(ThreadLocal.java:617)
at java.lang.ThreadLocal$ThreadLocalMap.remove(ThreadLocal.java:499)
at java.lang.ThreadLocal$ThreadLocalMap.access$200(ThreadLocal.java:298)
at java.lang.ThreadLocal.remove(ThreadLocal.java:222)
at 
java.util.concurrent.locks.ReentrantReadWriteLock$Sync.tryReleaseShared(ReentrantReadWriteLock.java:426)
at 
java.util.concurrent.locks.AbstractQueuedSynchronizer.releaseShared(AbstractQueuedSynchronizer.java:1341)
at 
java.util.concurrent.locks.ReentrantReadWriteLock$ReadLock.unlock(ReentrantReadWriteLock.java:881)
at 
com.yammer.metrics.stats.ExponentiallyDecayingSample.unlockForRegularUsage(ExponentiallyDecayingSample.java:196)
  

[jira] [Updated] (HBASE-16616) Rpc handlers stuck on ThreadLocalMap.expungeStaleEntry

2016-09-12 Thread Tomu Tsuruhara (JIRA)

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

Tomu Tsuruhara updated HBASE-16616:
---
Attachment: HBASE-16616.master.001.patch

> Rpc handlers stuck on ThreadLocalMap.expungeStaleEntry
> --
>
> Key: HBASE-16616
> URL: https://issues.apache.org/jira/browse/HBASE-16616
> Project: HBase
>  Issue Type: Improvement
>  Components: Performance
>Affects Versions: 1.2.2
>Reporter: Tomu Tsuruhara
> Attachments: HBASE-16616.master.001.patch, ScreenShot 2016-09-09 
> 14.17.53.png
>
>
> In our HBase 1.2.2 cluster, some regionserver showed too bad 
> "QueueCallTime_99th_percentile" exceeding 10 seconds.
> Most rpc handler threads stuck on ThreadLocalMap.expungeStaleEntry call at 
> that time.
> {noformat}
> "PriorityRpcServer.handler=18,queue=0,port=16020" #322 daemon prio=5 
> os_prio=0 tid=0x7fd422062800 nid=0x19b89 runnable [0x7fcb8a821000]
>java.lang.Thread.State: RUNNABLE
> at 
> java.lang.ThreadLocal$ThreadLocalMap.expungeStaleEntry(ThreadLocal.java:617)
> at java.lang.ThreadLocal$ThreadLocalMap.remove(ThreadLocal.java:499)
> at 
> java.lang.ThreadLocal$ThreadLocalMap.access$200(ThreadLocal.java:298)
> at java.lang.ThreadLocal.remove(ThreadLocal.java:222)
> at 
> java.util.concurrent.locks.ReentrantReadWriteLock$Sync.tryReleaseShared(ReentrantReadWriteLock.java:426)
> at 
> java.util.concurrent.locks.AbstractQueuedSynchronizer.releaseShared(AbstractQueuedSynchronizer.java:1341)
> at 
> java.util.concurrent.locks.ReentrantReadWriteLock$ReadLock.unlock(ReentrantReadWriteLock.java:881)
> at 
> com.yammer.metrics.stats.ExponentiallyDecayingSample.unlockForRegularUsage(ExponentiallyDecayingSample.java:196)
> at 
> com.yammer.metrics.stats.ExponentiallyDecayingSample.update(ExponentiallyDecayingSample.java:113)
> at 
> com.yammer.metrics.stats.ExponentiallyDecayingSample.update(ExponentiallyDecayingSample.java:81)
> at 
> org.apache.hadoop.metrics2.lib.MutableHistogram.add(MutableHistogram.java:81)
> at 
> org.apache.hadoop.metrics2.lib.MutableRangeHistogram.add(MutableRangeHistogram.java:59)
> at 
> org.apache.hadoop.hbase.ipc.MetricsHBaseServerSourceImpl.dequeuedCall(MetricsHBaseServerSourceImpl.java:194)
> at 
> org.apache.hadoop.hbase.ipc.MetricsHBaseServer.dequeuedCall(MetricsHBaseServer.java:76)
> at org.apache.hadoop.hbase.ipc.RpcServer.call(RpcServer.java:2192)
> at org.apache.hadoop.hbase.ipc.CallRunner.run(CallRunner.java:112)
> at 
> org.apache.hadoop.hbase.ipc.RpcExecutor.consumerLoop(RpcExecutor.java:133)
> at org.apache.hadoop.hbase.ipc.RpcExecutor$1.run(RpcExecutor.java:108)
> at java.lang.Thread.run(Thread.java:745)
> {noformat}
> We were using jdk 1.8.0_92 and here is a snippet from ThreadLocal.java.
> {code}
> 616:while (tab[h] != null)
> 617:h = nextIndex(h, len);
> {code}
> So I hypothesized that there're too many consecutive entries in {{tab}} array 
> and actually I found them in the heapdump.
> !ScreenShot 2016-09-09 14.17.53.png|width=50%!
> Most of these entries pointed at instance of 
> {{org.apache.hadoop.hbase.util.Counter$1}}
> which is equivarent to {{indexHolderThreadLocal}} instance-variable in the 
> {{Counter}} class.
> Because {{RpcServer$Connection}} class creates a {{Counter}} instance 
> {{rpcCount}} for every connections,
> it is possible to have lots of {{Counter#indexHolderThreadLocal}} instances 
> in RegionServer process
> when we repeat connect-and-close from client. As a result, a ThreadLocalMap 
> can have lots of consecutive
> entires.
> Usually, since each entry is a {{WeakReference}}, these entries are collected 
> and removed
> by garbage-collector soon after connection closed.
> But if connection's life-time was long enough to survive youngGC, it wouldn't 
> be collected until old-gen collector runs.
> Furthermore, under G1GC deployment, it is possible not to be collected even 
> by old-gen GC(mixed GC)
> if entries sit in a region which doesn't have much garbages.
> Actually we used G1GC when we encountered this problem.
> We should remove the entry from ThreadLocalMap by calling ThreadLocal#remove 
> explicitly.



--
This message was sent by Atlassian JIRA
(v6.3.4#6332)


[jira] [Commented] (HBASE-16616) Rpc handlers stuck on ThreadLocalMap.expungeStaleEntry

2016-09-12 Thread Tomu Tsuruhara (JIRA)

[ 
https://issues.apache.org/jira/browse/HBASE-16616?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=15485310#comment-15485310
 ] 

Tomu Tsuruhara commented on HBASE-16616:


Ted:
Sure, I will

> Rpc handlers stuck on ThreadLocalMap.expungeStaleEntry
> --
>
> Key: HBASE-16616
> URL: https://issues.apache.org/jira/browse/HBASE-16616
> Project: HBase
>  Issue Type: Improvement
>  Components: Performance
>Affects Versions: 1.2.2
>Reporter: Tomu Tsuruhara
> Attachments: 16616.branch-1.v2.txt, HBASE-16616.master.001.patch, 
> HBASE-16616.master.002.patch, ScreenShot 2016-09-09 14.17.53.png
>
>
> In our HBase 1.2.2 cluster, some regionserver showed too bad 
> "QueueCallTime_99th_percentile" exceeding 10 seconds.
> Most rpc handler threads stuck on ThreadLocalMap.expungeStaleEntry call at 
> that time.
> {noformat}
> "PriorityRpcServer.handler=18,queue=0,port=16020" #322 daemon prio=5 
> os_prio=0 tid=0x7fd422062800 nid=0x19b89 runnable [0x7fcb8a821000]
>java.lang.Thread.State: RUNNABLE
> at 
> java.lang.ThreadLocal$ThreadLocalMap.expungeStaleEntry(ThreadLocal.java:617)
> at java.lang.ThreadLocal$ThreadLocalMap.remove(ThreadLocal.java:499)
> at 
> java.lang.ThreadLocal$ThreadLocalMap.access$200(ThreadLocal.java:298)
> at java.lang.ThreadLocal.remove(ThreadLocal.java:222)
> at 
> java.util.concurrent.locks.ReentrantReadWriteLock$Sync.tryReleaseShared(ReentrantReadWriteLock.java:426)
> at 
> java.util.concurrent.locks.AbstractQueuedSynchronizer.releaseShared(AbstractQueuedSynchronizer.java:1341)
> at 
> java.util.concurrent.locks.ReentrantReadWriteLock$ReadLock.unlock(ReentrantReadWriteLock.java:881)
> at 
> com.yammer.metrics.stats.ExponentiallyDecayingSample.unlockForRegularUsage(ExponentiallyDecayingSample.java:196)
> at 
> com.yammer.metrics.stats.ExponentiallyDecayingSample.update(ExponentiallyDecayingSample.java:113)
> at 
> com.yammer.metrics.stats.ExponentiallyDecayingSample.update(ExponentiallyDecayingSample.java:81)
> at 
> org.apache.hadoop.metrics2.lib.MutableHistogram.add(MutableHistogram.java:81)
> at 
> org.apache.hadoop.metrics2.lib.MutableRangeHistogram.add(MutableRangeHistogram.java:59)
> at 
> org.apache.hadoop.hbase.ipc.MetricsHBaseServerSourceImpl.dequeuedCall(MetricsHBaseServerSourceImpl.java:194)
> at 
> org.apache.hadoop.hbase.ipc.MetricsHBaseServer.dequeuedCall(MetricsHBaseServer.java:76)
> at org.apache.hadoop.hbase.ipc.RpcServer.call(RpcServer.java:2192)
> at org.apache.hadoop.hbase.ipc.CallRunner.run(CallRunner.java:112)
> at 
> org.apache.hadoop.hbase.ipc.RpcExecutor.consumerLoop(RpcExecutor.java:133)
> at org.apache.hadoop.hbase.ipc.RpcExecutor$1.run(RpcExecutor.java:108)
> at java.lang.Thread.run(Thread.java:745)
> {noformat}
> We were using jdk 1.8.0_92 and here is a snippet from ThreadLocal.java.
> {code}
> 616:while (tab[h] != null)
> 617:h = nextIndex(h, len);
> {code}
> So I hypothesized that there're too many consecutive entries in {{tab}} array 
> and actually I found them in the heapdump.
> !ScreenShot 2016-09-09 14.17.53.png|width=50%!
> Most of these entries pointed at instance of 
> {{org.apache.hadoop.hbase.util.Counter$1}}
> which is equivarent to {{indexHolderThreadLocal}} instance-variable in the 
> {{Counter}} class.
> Because {{RpcServer$Connection}} class creates a {{Counter}} instance 
> {{rpcCount}} for every connections,
> it is possible to have lots of {{Counter#indexHolderThreadLocal}} instances 
> in RegionServer process
> when we repeat connect-and-close from client. As a result, a ThreadLocalMap 
> can have lots of consecutive
> entires.
> Usually, since each entry is a {{WeakReference}}, these entries are collected 
> and removed
> by garbage-collector soon after connection closed.
> But if connection's life-time was long enough to survive youngGC, it wouldn't 
> be collected until old-gen collector runs.
> Furthermore, under G1GC deployment, it is possible not to be collected even 
> by old-gen GC(mixed GC)
> if entries sit in a region which doesn't have much garbages.
> Actually we used G1GC when we encountered this problem.
> We should remove the entry from ThreadLocalMap by calling ThreadLocal#remove 
> explicitly.



--
This message was sent by Atlassian JIRA
(v6.3.4#6332)


[jira] [Commented] (HBASE-16616) Rpc handlers stuck on ThreadLocalMap.expungeStaleEntry

2016-09-12 Thread Tomu Tsuruhara (JIRA)

[ 
https://issues.apache.org/jira/browse/HBASE-16616?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=15485892#comment-15485892
 ] 

Tomu Tsuruhara commented on HBASE-16616:


Then, what's next for this issue?

> Rpc handlers stuck on ThreadLocalMap.expungeStaleEntry
> --
>
> Key: HBASE-16616
> URL: https://issues.apache.org/jira/browse/HBASE-16616
> Project: HBase
>  Issue Type: Improvement
>  Components: Performance
>Affects Versions: 1.2.2
>Reporter: Tomu Tsuruhara
>Assignee: Tomu Tsuruhara
> Fix For: 2.0.0, 1.4.0
>
> Attachments: 16616.branch-1.v2.txt, HBASE-16616.master.001.patch, 
> HBASE-16616.master.002.patch, ScreenShot 2016-09-09 14.17.53.png
>
>
> In our HBase 1.2.2 cluster, some regionserver showed too bad 
> "QueueCallTime_99th_percentile" exceeding 10 seconds.
> Most rpc handler threads stuck on ThreadLocalMap.expungeStaleEntry call at 
> that time.
> {noformat}
> "PriorityRpcServer.handler=18,queue=0,port=16020" #322 daemon prio=5 
> os_prio=0 tid=0x7fd422062800 nid=0x19b89 runnable [0x7fcb8a821000]
>java.lang.Thread.State: RUNNABLE
> at 
> java.lang.ThreadLocal$ThreadLocalMap.expungeStaleEntry(ThreadLocal.java:617)
> at java.lang.ThreadLocal$ThreadLocalMap.remove(ThreadLocal.java:499)
> at 
> java.lang.ThreadLocal$ThreadLocalMap.access$200(ThreadLocal.java:298)
> at java.lang.ThreadLocal.remove(ThreadLocal.java:222)
> at 
> java.util.concurrent.locks.ReentrantReadWriteLock$Sync.tryReleaseShared(ReentrantReadWriteLock.java:426)
> at 
> java.util.concurrent.locks.AbstractQueuedSynchronizer.releaseShared(AbstractQueuedSynchronizer.java:1341)
> at 
> java.util.concurrent.locks.ReentrantReadWriteLock$ReadLock.unlock(ReentrantReadWriteLock.java:881)
> at 
> com.yammer.metrics.stats.ExponentiallyDecayingSample.unlockForRegularUsage(ExponentiallyDecayingSample.java:196)
> at 
> com.yammer.metrics.stats.ExponentiallyDecayingSample.update(ExponentiallyDecayingSample.java:113)
> at 
> com.yammer.metrics.stats.ExponentiallyDecayingSample.update(ExponentiallyDecayingSample.java:81)
> at 
> org.apache.hadoop.metrics2.lib.MutableHistogram.add(MutableHistogram.java:81)
> at 
> org.apache.hadoop.metrics2.lib.MutableRangeHistogram.add(MutableRangeHistogram.java:59)
> at 
> org.apache.hadoop.hbase.ipc.MetricsHBaseServerSourceImpl.dequeuedCall(MetricsHBaseServerSourceImpl.java:194)
> at 
> org.apache.hadoop.hbase.ipc.MetricsHBaseServer.dequeuedCall(MetricsHBaseServer.java:76)
> at org.apache.hadoop.hbase.ipc.RpcServer.call(RpcServer.java:2192)
> at org.apache.hadoop.hbase.ipc.CallRunner.run(CallRunner.java:112)
> at 
> org.apache.hadoop.hbase.ipc.RpcExecutor.consumerLoop(RpcExecutor.java:133)
> at org.apache.hadoop.hbase.ipc.RpcExecutor$1.run(RpcExecutor.java:108)
> at java.lang.Thread.run(Thread.java:745)
> {noformat}
> We were using jdk 1.8.0_92 and here is a snippet from ThreadLocal.java.
> {code}
> 616:while (tab[h] != null)
> 617:h = nextIndex(h, len);
> {code}
> So I hypothesized that there're too many consecutive entries in {{tab}} array 
> and actually I found them in the heapdump.
> !ScreenShot 2016-09-09 14.17.53.png|width=50%!
> Most of these entries pointed at instance of 
> {{org.apache.hadoop.hbase.util.Counter$1}}
> which is equivarent to {{indexHolderThreadLocal}} instance-variable in the 
> {{Counter}} class.
> Because {{RpcServer$Connection}} class creates a {{Counter}} instance 
> {{rpcCount}} for every connections,
> it is possible to have lots of {{Counter#indexHolderThreadLocal}} instances 
> in RegionServer process
> when we repeat connect-and-close from client. As a result, a ThreadLocalMap 
> can have lots of consecutive
> entires.
> Usually, since each entry is a {{WeakReference}}, these entries are collected 
> and removed
> by garbage-collector soon after connection closed.
> But if connection's life-time was long enough to survive youngGC, it wouldn't 
> be collected until old-gen collector runs.
> Furthermore, under G1GC deployment, it is possible not to be collected even 
> by old-gen GC(mixed GC)
> if entries sit in a region which doesn't have much garbages.
> Actually we used G1GC when we encountered this problem.
> We should remove the entry from ThreadLocalMap by calling ThreadLocal#remove 
> explicitly.



--
This message was sent by Atlassian JIRA
(v6.3.4#6332)


[jira] [Updated] (HBASE-16616) Rpc handlers stuck on ThreadLocalMap.expungeStaleEntry

2016-09-12 Thread Tomu Tsuruhara (JIRA)

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

Tomu Tsuruhara updated HBASE-16616:
---
Attachment: HBASE-16616.master.002.patch

> Rpc handlers stuck on ThreadLocalMap.expungeStaleEntry
> --
>
> Key: HBASE-16616
> URL: https://issues.apache.org/jira/browse/HBASE-16616
> Project: HBase
>  Issue Type: Improvement
>  Components: Performance
>Affects Versions: 1.2.2
>Reporter: Tomu Tsuruhara
> Attachments: HBASE-16616.master.001.patch, 
> HBASE-16616.master.002.patch, ScreenShot 2016-09-09 14.17.53.png
>
>
> In our HBase 1.2.2 cluster, some regionserver showed too bad 
> "QueueCallTime_99th_percentile" exceeding 10 seconds.
> Most rpc handler threads stuck on ThreadLocalMap.expungeStaleEntry call at 
> that time.
> {noformat}
> "PriorityRpcServer.handler=18,queue=0,port=16020" #322 daemon prio=5 
> os_prio=0 tid=0x7fd422062800 nid=0x19b89 runnable [0x7fcb8a821000]
>java.lang.Thread.State: RUNNABLE
> at 
> java.lang.ThreadLocal$ThreadLocalMap.expungeStaleEntry(ThreadLocal.java:617)
> at java.lang.ThreadLocal$ThreadLocalMap.remove(ThreadLocal.java:499)
> at 
> java.lang.ThreadLocal$ThreadLocalMap.access$200(ThreadLocal.java:298)
> at java.lang.ThreadLocal.remove(ThreadLocal.java:222)
> at 
> java.util.concurrent.locks.ReentrantReadWriteLock$Sync.tryReleaseShared(ReentrantReadWriteLock.java:426)
> at 
> java.util.concurrent.locks.AbstractQueuedSynchronizer.releaseShared(AbstractQueuedSynchronizer.java:1341)
> at 
> java.util.concurrent.locks.ReentrantReadWriteLock$ReadLock.unlock(ReentrantReadWriteLock.java:881)
> at 
> com.yammer.metrics.stats.ExponentiallyDecayingSample.unlockForRegularUsage(ExponentiallyDecayingSample.java:196)
> at 
> com.yammer.metrics.stats.ExponentiallyDecayingSample.update(ExponentiallyDecayingSample.java:113)
> at 
> com.yammer.metrics.stats.ExponentiallyDecayingSample.update(ExponentiallyDecayingSample.java:81)
> at 
> org.apache.hadoop.metrics2.lib.MutableHistogram.add(MutableHistogram.java:81)
> at 
> org.apache.hadoop.metrics2.lib.MutableRangeHistogram.add(MutableRangeHistogram.java:59)
> at 
> org.apache.hadoop.hbase.ipc.MetricsHBaseServerSourceImpl.dequeuedCall(MetricsHBaseServerSourceImpl.java:194)
> at 
> org.apache.hadoop.hbase.ipc.MetricsHBaseServer.dequeuedCall(MetricsHBaseServer.java:76)
> at org.apache.hadoop.hbase.ipc.RpcServer.call(RpcServer.java:2192)
> at org.apache.hadoop.hbase.ipc.CallRunner.run(CallRunner.java:112)
> at 
> org.apache.hadoop.hbase.ipc.RpcExecutor.consumerLoop(RpcExecutor.java:133)
> at org.apache.hadoop.hbase.ipc.RpcExecutor$1.run(RpcExecutor.java:108)
> at java.lang.Thread.run(Thread.java:745)
> {noformat}
> We were using jdk 1.8.0_92 and here is a snippet from ThreadLocal.java.
> {code}
> 616:while (tab[h] != null)
> 617:h = nextIndex(h, len);
> {code}
> So I hypothesized that there're too many consecutive entries in {{tab}} array 
> and actually I found them in the heapdump.
> !ScreenShot 2016-09-09 14.17.53.png|width=50%!
> Most of these entries pointed at instance of 
> {{org.apache.hadoop.hbase.util.Counter$1}}
> which is equivarent to {{indexHolderThreadLocal}} instance-variable in the 
> {{Counter}} class.
> Because {{RpcServer$Connection}} class creates a {{Counter}} instance 
> {{rpcCount}} for every connections,
> it is possible to have lots of {{Counter#indexHolderThreadLocal}} instances 
> in RegionServer process
> when we repeat connect-and-close from client. As a result, a ThreadLocalMap 
> can have lots of consecutive
> entires.
> Usually, since each entry is a {{WeakReference}}, these entries are collected 
> and removed
> by garbage-collector soon after connection closed.
> But if connection's life-time was long enough to survive youngGC, it wouldn't 
> be collected until old-gen collector runs.
> Furthermore, under G1GC deployment, it is possible not to be collected even 
> by old-gen GC(mixed GC)
> if entries sit in a region which doesn't have much garbages.
> Actually we used G1GC when we encountered this problem.
> We should remove the entry from ThreadLocalMap by calling ThreadLocal#remove 
> explicitly.



--
This message was sent by Atlassian JIRA
(v6.3.4#6332)


[jira] [Commented] (HBASE-17072) CPU usage starts to climb up to 90-100% when using G1GC

2016-11-17 Thread Tomu Tsuruhara (JIRA)

[ 
https://issues.apache.org/jira/browse/HBASE-17072?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=15673741#comment-15673741
 ] 

Tomu Tsuruhara commented on HBASE-17072:


How about simply calling {{ThreadLocal#remove}} on closeStreams?

{code}
diff --git 
a/hbase-server/src/main/java/org/apache/hadoop/hbase/io/hfile/HFileBlock.java 
b/hbase-server/src/main/java/org/apache/hadoop/hbase/io/hfile/HFileBlock.java
index 0dc0f80..6e9f61a 100644
--- 
a/hbase-server/src/main/java/org/apache/hadoop/hbase/io/hfile/HFileBlock.java
+++ 
b/hbase-server/src/main/java/org/apache/hadoop/hbase/io/hfile/HFileBlock.java
@@ -1760,6 +1760,7 @@ public class HFileBlock implements Cacheable {

 @Override
 public void closeStreams() throws IOException {
+  prefetchedHeaderForThread.remove();
   streamWrapper.close();
 }
{code}

In this way, we can avoid generating too large ThreadLocalMap while keeping 
prefetch benefit.


> CPU usage starts to climb up to 90-100% when using G1GC
> ---
>
> Key: HBASE-17072
> URL: https://issues.apache.org/jira/browse/HBASE-17072
> Project: HBase
>  Issue Type: Bug
>  Components: Performance, regionserver
>Affects Versions: 1.0.0, 1.2.0
>Reporter: Eiichi Sato
> Attachments: disable-block-header-cache.patch, mat-threadlocals.png, 
> mat-threads.png, metrics.png, slave1.svg, slave2.svg, slave3.svg, slave4.svg
>
>
> h5. Problem
> CPU usage of a region server in our CDH 5.4.5 cluster, at some point, starts 
> to gradually get higher up to nearly 90-100% when using G1GC.  We've also run 
> into this problem on CDH 5.7.3 and CDH 5.8.2.
> In our production cluster, it normally takes a few weeks for this to happen 
> after restarting a RS.  We reproduced this on our test cluster and attached 
> the results.  Please note that, to make it easy to reproduce, we did some 
> "anti-tuning" on a table when running tests.
> In metrics.png, soon after we started running some workloads against a test 
> cluster (CDH 5.8.2) at about 7 p.m. CPU usage of the two RSs started to rise. 
>  Flame Graphs (slave1.svg to slave4.svg) are generated from jstack dumps of 
> each RS process around 10:30 a.m. the next day.
> After investigating heapdumps from another occurrence on a test cluster 
> running CDH 5.7.3, we found that the ThreadLocalMap contain a lot of 
> contiguous entries of {{HFileBlock$PrefetchedHeader}} probably due to primary 
> clustering.  This caused more loops in 
> {{ThreadLocalMap#expungeStaleEntries()}}, consuming a certain amount of CPU 
> time.  What is worse is that the method is called from RPC metrics code, 
> which means even a small amount of per-RPC time soon adds up to a huge amount 
> of CPU time.
> This is very similar to the issue in HBASE-16616, but we have many 
> {{HFileBlock$PrefetchedHeader}} not only {{Counter$IndexHolder}} instances.  
> Here are some OQL counts from Eclipse Memory Analyzer (MAT).  This shows a 
> number of ThreadLocal instances in the ThreadLocalMap of a single handler 
> thread.
> {code}
> SELECT *
> FROM OBJECTS (SELECT AS RETAINED SET OBJECTS value
> FROM OBJECTS 0x4ee380430) obj
> WHERE obj.@clazz.@name = 
> "org.apache.hadoop.hbase.io.hfile.HFileBlock$PrefetchedHeader"
> #=> 10980 instances
> {code}
> {code}
> SELECT *
> FROM OBJECTS (SELECT AS RETAINED SET OBJECTS value
> FROM OBJECTS 0x4ee380430) obj
> WHERE obj.@clazz.@name = "org.apache.hadoop.hbase.util.Counter$IndexHolder"
> #=> 2052 instances
> {code}
> Although as described in HBASE-16616 this somewhat seems to be an issue in 
> G1GC side regarding weakly-reachable objects, we should keep ThreadLocal 
> usage minimal and avoid creating an indefinite number (in this case, a number 
> of HFiles) of ThreadLocal instances.
> HBASE-16146 removes ThreadLocals from the RPC metrics code.  That may solve 
> the issue (I just saw the patch, never tested it at all), but the 
> {{HFileBlock$PrefetchedHeader}} are still there in the ThreadLocalMap, which 
> may cause issues in the future again.
> h5. Our Solution
> We simply removed the whole {{HFileBlock$PrefetchedHeader}} caching and 
> fortunately we didn't notice any performance degradation for our production 
> workloads.
> Because the PrefetchedHeader caching uses ThreadLocal and because RPCs are 
> handled randomly in any of the handlers, small Get or small Scan RPCs do not 
> benefit from the caching (See HBASE-10676 and HBASE-11402 for the details).  
> Probably, we need to see how well reads are saved by the caching for large 
> Scan or Get RPCs and especially for compactions if we really remove the 
> caching. It's probably better if we can remove ThreadLocals without breaking 
> the current caching behavior.
> FWIW, I'm attaching the patch we applied. It's for CDH 5.4.5.



--
This 

[jira] [Commented] (HBASE-17072) CPU usage starts to climb up to 90-100% when using G1GC

2016-11-17 Thread Tomu Tsuruhara (JIRA)

[ 
https://issues.apache.org/jira/browse/HBASE-17072?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=15675448#comment-15675448
 ] 

Tomu Tsuruhara commented on HBASE-17072:


ah, I see. Thanks.

> CPU usage starts to climb up to 90-100% when using G1GC
> ---
>
> Key: HBASE-17072
> URL: https://issues.apache.org/jira/browse/HBASE-17072
> Project: HBase
>  Issue Type: Bug
>  Components: Performance, regionserver
>Affects Versions: 1.0.0, 1.2.0
>Reporter: Eiichi Sato
> Attachments: disable-block-header-cache.patch, mat-threadlocals.png, 
> mat-threads.png, metrics.png, slave1.svg, slave2.svg, slave3.svg, slave4.svg
>
>
> h5. Problem
> CPU usage of a region server in our CDH 5.4.5 cluster, at some point, starts 
> to gradually get higher up to nearly 90-100% when using G1GC.  We've also run 
> into this problem on CDH 5.7.3 and CDH 5.8.2.
> In our production cluster, it normally takes a few weeks for this to happen 
> after restarting a RS.  We reproduced this on our test cluster and attached 
> the results.  Please note that, to make it easy to reproduce, we did some 
> "anti-tuning" on a table when running tests.
> In metrics.png, soon after we started running some workloads against a test 
> cluster (CDH 5.8.2) at about 7 p.m. CPU usage of the two RSs started to rise. 
>  Flame Graphs (slave1.svg to slave4.svg) are generated from jstack dumps of 
> each RS process around 10:30 a.m. the next day.
> After investigating heapdumps from another occurrence on a test cluster 
> running CDH 5.7.3, we found that the ThreadLocalMap contain a lot of 
> contiguous entries of {{HFileBlock$PrefetchedHeader}} probably due to primary 
> clustering.  This caused more loops in 
> {{ThreadLocalMap#expungeStaleEntries()}}, consuming a certain amount of CPU 
> time.  What is worse is that the method is called from RPC metrics code, 
> which means even a small amount of per-RPC time soon adds up to a huge amount 
> of CPU time.
> This is very similar to the issue in HBASE-16616, but we have many 
> {{HFileBlock$PrefetchedHeader}} not only {{Counter$IndexHolder}} instances.  
> Here are some OQL counts from Eclipse Memory Analyzer (MAT).  This shows a 
> number of ThreadLocal instances in the ThreadLocalMap of a single handler 
> thread.
> {code}
> SELECT *
> FROM OBJECTS (SELECT AS RETAINED SET OBJECTS value
> FROM OBJECTS 0x4ee380430) obj
> WHERE obj.@clazz.@name = 
> "org.apache.hadoop.hbase.io.hfile.HFileBlock$PrefetchedHeader"
> #=> 10980 instances
> {code}
> {code}
> SELECT *
> FROM OBJECTS (SELECT AS RETAINED SET OBJECTS value
> FROM OBJECTS 0x4ee380430) obj
> WHERE obj.@clazz.@name = "org.apache.hadoop.hbase.util.Counter$IndexHolder"
> #=> 2052 instances
> {code}
> Although as described in HBASE-16616 this somewhat seems to be an issue in 
> G1GC side regarding weakly-reachable objects, we should keep ThreadLocal 
> usage minimal and avoid creating an indefinite number (in this case, a number 
> of HFiles) of ThreadLocal instances.
> HBASE-16146 removes ThreadLocals from the RPC metrics code.  That may solve 
> the issue (I just saw the patch, never tested it at all), but the 
> {{HFileBlock$PrefetchedHeader}} are still there in the ThreadLocalMap, which 
> may cause issues in the future again.
> h5. Our Solution
> We simply removed the whole {{HFileBlock$PrefetchedHeader}} caching and 
> fortunately we didn't notice any performance degradation for our production 
> workloads.
> Because the PrefetchedHeader caching uses ThreadLocal and because RPCs are 
> handled randomly in any of the handlers, small Get or small Scan RPCs do not 
> benefit from the caching (See HBASE-10676 and HBASE-11402 for the details).  
> Probably, we need to see how well reads are saved by the caching for large 
> Scan or Get RPCs and especially for compactions if we really remove the 
> caching. It's probably better if we can remove ThreadLocals without breaking 
> the current caching behavior.
> FWIW, I'm attaching the patch we applied. It's for CDH 5.4.5.



--
This message was sent by Atlassian JIRA
(v6.3.4#6332)


[jira] [Commented] (HBASE-16616) Rpc handlers stuck on ThreadLocalMap.expungeStaleEntry

2016-12-01 Thread Tomu Tsuruhara (JIRA)

[ 
https://issues.apache.org/jira/browse/HBASE-16616?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=15713580#comment-15713580
 ] 

Tomu Tsuruhara commented on HBASE-16616:


Actually, a patch here was not enough to resolve the issue because of the same 
reason mentioned here 
https://issues.apache.org/jira/browse/HBASE-17072?focusedCommentId=15675394=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel#comment-15675394

HBASE-16146 seems more essential. It removes {{ThreadLocal}} usage from 
{[Counter}} entirely.

> Rpc handlers stuck on ThreadLocalMap.expungeStaleEntry
> --
>
> Key: HBASE-16616
> URL: https://issues.apache.org/jira/browse/HBASE-16616
> Project: HBase
>  Issue Type: Improvement
>  Components: Performance
>Affects Versions: 1.2.2
>Reporter: Tomu Tsuruhara
>Assignee: Tomu Tsuruhara
> Fix For: 2.0.0, 1.4.0
>
> Attachments: 16616.branch-1.v2.txt, HBASE-16616.master.001.patch, 
> HBASE-16616.master.002.patch, ScreenShot 2016-09-09 14.17.53.png
>
>
> In our HBase 1.2.2 cluster, some regionserver showed too bad 
> "QueueCallTime_99th_percentile" exceeding 10 seconds.
> Most rpc handler threads stuck on ThreadLocalMap.expungeStaleEntry call at 
> that time.
> {noformat}
> "PriorityRpcServer.handler=18,queue=0,port=16020" #322 daemon prio=5 
> os_prio=0 tid=0x7fd422062800 nid=0x19b89 runnable [0x7fcb8a821000]
>java.lang.Thread.State: RUNNABLE
> at 
> java.lang.ThreadLocal$ThreadLocalMap.expungeStaleEntry(ThreadLocal.java:617)
> at java.lang.ThreadLocal$ThreadLocalMap.remove(ThreadLocal.java:499)
> at 
> java.lang.ThreadLocal$ThreadLocalMap.access$200(ThreadLocal.java:298)
> at java.lang.ThreadLocal.remove(ThreadLocal.java:222)
> at 
> java.util.concurrent.locks.ReentrantReadWriteLock$Sync.tryReleaseShared(ReentrantReadWriteLock.java:426)
> at 
> java.util.concurrent.locks.AbstractQueuedSynchronizer.releaseShared(AbstractQueuedSynchronizer.java:1341)
> at 
> java.util.concurrent.locks.ReentrantReadWriteLock$ReadLock.unlock(ReentrantReadWriteLock.java:881)
> at 
> com.yammer.metrics.stats.ExponentiallyDecayingSample.unlockForRegularUsage(ExponentiallyDecayingSample.java:196)
> at 
> com.yammer.metrics.stats.ExponentiallyDecayingSample.update(ExponentiallyDecayingSample.java:113)
> at 
> com.yammer.metrics.stats.ExponentiallyDecayingSample.update(ExponentiallyDecayingSample.java:81)
> at 
> org.apache.hadoop.metrics2.lib.MutableHistogram.add(MutableHistogram.java:81)
> at 
> org.apache.hadoop.metrics2.lib.MutableRangeHistogram.add(MutableRangeHistogram.java:59)
> at 
> org.apache.hadoop.hbase.ipc.MetricsHBaseServerSourceImpl.dequeuedCall(MetricsHBaseServerSourceImpl.java:194)
> at 
> org.apache.hadoop.hbase.ipc.MetricsHBaseServer.dequeuedCall(MetricsHBaseServer.java:76)
> at org.apache.hadoop.hbase.ipc.RpcServer.call(RpcServer.java:2192)
> at org.apache.hadoop.hbase.ipc.CallRunner.run(CallRunner.java:112)
> at 
> org.apache.hadoop.hbase.ipc.RpcExecutor.consumerLoop(RpcExecutor.java:133)
> at org.apache.hadoop.hbase.ipc.RpcExecutor$1.run(RpcExecutor.java:108)
> at java.lang.Thread.run(Thread.java:745)
> {noformat}
> We were using jdk 1.8.0_92 and here is a snippet from ThreadLocal.java.
> {code}
> 616:while (tab[h] != null)
> 617:h = nextIndex(h, len);
> {code}
> So I hypothesized that there're too many consecutive entries in {{tab}} array 
> and actually I found them in the heapdump.
> !ScreenShot 2016-09-09 14.17.53.png|width=50%!
> Most of these entries pointed at instance of 
> {{org.apache.hadoop.hbase.util.Counter$1}}
> which is equivarent to {{indexHolderThreadLocal}} instance-variable in the 
> {{Counter}} class.
> Because {{RpcServer$Connection}} class creates a {{Counter}} instance 
> {{rpcCount}} for every connections,
> it is possible to have lots of {{Counter#indexHolderThreadLocal}} instances 
> in RegionServer process
> when we repeat connect-and-close from client. As a result, a ThreadLocalMap 
> can have lots of consecutive
> entires.
> Usually, since each entry is a {{WeakReference}}, these entries are collected 
> and removed
> by garbage-collector soon after connection closed.
> But if connection's life-time was long enough to survive youngGC, it wouldn't 
> be collected until old-gen collector runs.
> Furthermore, under G1GC deployment, it is possible not to be collected even 
> by old-gen GC(mixed GC)
> if entries sit in a region which doesn't have much garbages.
> Actually we used G1GC when we encountered this problem.
> We should remove the entry from ThreadLocalMap by calling ThreadLocal#remove 
> explicitly.



--
This message was sent by Atlassian JIRA
(v6.3.4#6332)


[jira] [Comment Edited] (HBASE-16616) Rpc handlers stuck on ThreadLocalMap.expungeStaleEntry

2016-12-01 Thread Tomu Tsuruhara (JIRA)

[ 
https://issues.apache.org/jira/browse/HBASE-16616?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=15713580#comment-15713580
 ] 

Tomu Tsuruhara edited comment on HBASE-16616 at 12/2/16 12:49 AM:
--

Actually, a patch here was not enough to resolve the issue because of the same 
reason mentioned here 
https://issues.apache.org/jira/browse/HBASE-17072?focusedCommentId=15675394=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel#comment-15675394

HBASE-16146 seems more essential. It removes {{ThreadLocal}} usage from 
{{Counter}} entirely.


was (Author: tomu.tsuruhara):
Actually, a patch here was not enough to resolve the issue because of the same 
reason mentioned here 
https://issues.apache.org/jira/browse/HBASE-17072?focusedCommentId=15675394=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel#comment-15675394

HBASE-16146 seems more essential. It removes {{ThreadLocal}} usage from 
{[Counter}} entirely.

> Rpc handlers stuck on ThreadLocalMap.expungeStaleEntry
> --
>
> Key: HBASE-16616
> URL: https://issues.apache.org/jira/browse/HBASE-16616
> Project: HBase
>  Issue Type: Improvement
>  Components: Performance
>Affects Versions: 1.2.2
>Reporter: Tomu Tsuruhara
>Assignee: Tomu Tsuruhara
> Fix For: 2.0.0, 1.4.0
>
> Attachments: 16616.branch-1.v2.txt, HBASE-16616.master.001.patch, 
> HBASE-16616.master.002.patch, ScreenShot 2016-09-09 14.17.53.png
>
>
> In our HBase 1.2.2 cluster, some regionserver showed too bad 
> "QueueCallTime_99th_percentile" exceeding 10 seconds.
> Most rpc handler threads stuck on ThreadLocalMap.expungeStaleEntry call at 
> that time.
> {noformat}
> "PriorityRpcServer.handler=18,queue=0,port=16020" #322 daemon prio=5 
> os_prio=0 tid=0x7fd422062800 nid=0x19b89 runnable [0x7fcb8a821000]
>java.lang.Thread.State: RUNNABLE
> at 
> java.lang.ThreadLocal$ThreadLocalMap.expungeStaleEntry(ThreadLocal.java:617)
> at java.lang.ThreadLocal$ThreadLocalMap.remove(ThreadLocal.java:499)
> at 
> java.lang.ThreadLocal$ThreadLocalMap.access$200(ThreadLocal.java:298)
> at java.lang.ThreadLocal.remove(ThreadLocal.java:222)
> at 
> java.util.concurrent.locks.ReentrantReadWriteLock$Sync.tryReleaseShared(ReentrantReadWriteLock.java:426)
> at 
> java.util.concurrent.locks.AbstractQueuedSynchronizer.releaseShared(AbstractQueuedSynchronizer.java:1341)
> at 
> java.util.concurrent.locks.ReentrantReadWriteLock$ReadLock.unlock(ReentrantReadWriteLock.java:881)
> at 
> com.yammer.metrics.stats.ExponentiallyDecayingSample.unlockForRegularUsage(ExponentiallyDecayingSample.java:196)
> at 
> com.yammer.metrics.stats.ExponentiallyDecayingSample.update(ExponentiallyDecayingSample.java:113)
> at 
> com.yammer.metrics.stats.ExponentiallyDecayingSample.update(ExponentiallyDecayingSample.java:81)
> at 
> org.apache.hadoop.metrics2.lib.MutableHistogram.add(MutableHistogram.java:81)
> at 
> org.apache.hadoop.metrics2.lib.MutableRangeHistogram.add(MutableRangeHistogram.java:59)
> at 
> org.apache.hadoop.hbase.ipc.MetricsHBaseServerSourceImpl.dequeuedCall(MetricsHBaseServerSourceImpl.java:194)
> at 
> org.apache.hadoop.hbase.ipc.MetricsHBaseServer.dequeuedCall(MetricsHBaseServer.java:76)
> at org.apache.hadoop.hbase.ipc.RpcServer.call(RpcServer.java:2192)
> at org.apache.hadoop.hbase.ipc.CallRunner.run(CallRunner.java:112)
> at 
> org.apache.hadoop.hbase.ipc.RpcExecutor.consumerLoop(RpcExecutor.java:133)
> at org.apache.hadoop.hbase.ipc.RpcExecutor$1.run(RpcExecutor.java:108)
> at java.lang.Thread.run(Thread.java:745)
> {noformat}
> We were using jdk 1.8.0_92 and here is a snippet from ThreadLocal.java.
> {code}
> 616:while (tab[h] != null)
> 617:h = nextIndex(h, len);
> {code}
> So I hypothesized that there're too many consecutive entries in {{tab}} array 
> and actually I found them in the heapdump.
> !ScreenShot 2016-09-09 14.17.53.png|width=50%!
> Most of these entries pointed at instance of 
> {{org.apache.hadoop.hbase.util.Counter$1}}
> which is equivarent to {{indexHolderThreadLocal}} instance-variable in the 
> {{Counter}} class.
> Because {{RpcServer$Connection}} class creates a {{Counter}} instance 
> {{rpcCount}} for every connections,
> it is possible to have lots of {{Counter#indexHolderThreadLocal}} instances 
> in RegionServer process
> when we repeat connect-and-close from client. As a result, a ThreadLocalMap 
> can have lots of consecutive
> entires.
> Usually, since each entry is a {{WeakReference}}, these entries are collected 
> and removed
> by garbage-collector soon after connection closed.
> But if connection's life-time was long enough to survive 

[jira] [Created] (HBASE-17871) scan#setBatch(int) call leads wrong result of VerifyReplication

2017-04-03 Thread Tomu Tsuruhara (JIRA)
Tomu Tsuruhara created HBASE-17871:
--

 Summary: scan#setBatch(int) call leads wrong result of 
VerifyReplication
 Key: HBASE-17871
 URL: https://issues.apache.org/jira/browse/HBASE-17871
 Project: HBase
  Issue Type: Bug
Affects Versions: 2.0.0, 1.4.0
Reporter: Tomu Tsuruhara
Assignee: Tomu Tsuruhara
Priority: Minor


VerifyReplication tool printed weird logs.

{noformat}
2017-04-03 23:30:50,252 ERROR [main] 
org.apache.hadoop.hbase.mapreduce.replication.VerifyReplication: 
CONTENT_DIFFERENT_ROWS, rowkey=a100193
2017-04-03 23:30:50,280 ERROR [main] 
org.apache.hadoop.hbase.mapreduce.replication.VerifyReplication: 
ONLY_IN_PEER_TABLE_ROWS, rowkey=a100193
2017-04-03 23:30:50,387 ERROR [main] 
org.apache.hadoop.hbase.mapreduce.replication.VerifyReplication: 
CONTENT_DIFFERENT_ROWS, rowkey=a100385
2017-04-03 23:30:50,414 ERROR [main] 
org.apache.hadoop.hbase.mapreduce.replication.VerifyReplication: 
ONLY_IN_PEER_TABLE_ROWS, rowkey=a100385
2017-04-03 23:30:50,480 ERROR [main] 
org.apache.hadoop.hbase.mapreduce.replication.VerifyReplication: 
CONTENT_DIFFERENT_ROWS, rowkey=a100532
2017-04-03 23:30:50,508 ERROR [main] 
org.apache.hadoop.hbase.mapreduce.replication.VerifyReplication: 
ONLY_IN_PEER_TABLE_ROWS, rowkey=a100532
{noformat}

Here, each bad rows were marked as both {{CONTENT_DIFFERENT_ROWS}} and 
{{ONLY_IN_PEER_TABLE_ROWS}}.
This should never happen so I took a look at code and found scan.setBatch call.

{code}
@Override
public void map(ImmutableBytesWritable row, final Result value,
Context context)
throws IOException {
  if (replicatedScanner == null) {
...
final Scan scan = new Scan();
scan.setBatch(batch);
{code}

As stated in HBASE-16376, {{scan#setBatch(int)}} call implicitly allows scan 
results to be partial.

Since {{VerifyReplication}} is assuming each {{scanner.next()}} call returns 
entire row,
partial results break compare logic.

We should avoid setBatch call here.
Thanks to RPC chunking (explained in this blog 
https://blogs.apache.org/hbase/entry/scan_improvements_in_hbase_1),
it's safe and acceptable I think.




--
This message was sent by Atlassian JIRA
(v6.3.15#6346)


[jira] [Updated] (HBASE-17871) scan#setBatch(int) call leads wrong result of VerifyReplication

2017-04-03 Thread Tomu Tsuruhara (JIRA)

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

Tomu Tsuruhara updated HBASE-17871:
---
Status: Patch Available  (was: Open)

> scan#setBatch(int) call leads wrong result of VerifyReplication
> ---
>
> Key: HBASE-17871
> URL: https://issues.apache.org/jira/browse/HBASE-17871
> Project: HBase
>  Issue Type: Bug
>Affects Versions: 2.0.0, 1.4.0
>Reporter: Tomu Tsuruhara
>Assignee: Tomu Tsuruhara
>Priority: Minor
> Attachments: HBASE-17871.master.001.patch
>
>
> VerifyReplication tool printed weird logs.
> {noformat}
> 2017-04-03 23:30:50,252 ERROR [main] 
> org.apache.hadoop.hbase.mapreduce.replication.VerifyReplication: 
> CONTENT_DIFFERENT_ROWS, rowkey=a100193
> 2017-04-03 23:30:50,280 ERROR [main] 
> org.apache.hadoop.hbase.mapreduce.replication.VerifyReplication: 
> ONLY_IN_PEER_TABLE_ROWS, rowkey=a100193
> 2017-04-03 23:30:50,387 ERROR [main] 
> org.apache.hadoop.hbase.mapreduce.replication.VerifyReplication: 
> CONTENT_DIFFERENT_ROWS, rowkey=a100385
> 2017-04-03 23:30:50,414 ERROR [main] 
> org.apache.hadoop.hbase.mapreduce.replication.VerifyReplication: 
> ONLY_IN_PEER_TABLE_ROWS, rowkey=a100385
> 2017-04-03 23:30:50,480 ERROR [main] 
> org.apache.hadoop.hbase.mapreduce.replication.VerifyReplication: 
> CONTENT_DIFFERENT_ROWS, rowkey=a100532
> 2017-04-03 23:30:50,508 ERROR [main] 
> org.apache.hadoop.hbase.mapreduce.replication.VerifyReplication: 
> ONLY_IN_PEER_TABLE_ROWS, rowkey=a100532
> {noformat}
> Here, each bad rows were marked as both {{CONTENT_DIFFERENT_ROWS}} and 
> {{ONLY_IN_PEER_TABLE_ROWS}}.
> This should never happen so I took a look at code and found scan.setBatch 
> call.
> {code}
> @Override
> public void map(ImmutableBytesWritable row, final Result value,
> Context context)
> throws IOException {
>   if (replicatedScanner == null) {
>   ...
> final Scan scan = new Scan();
> scan.setBatch(batch);
> {code}
> As stated in HBASE-16376, {{scan#setBatch(int)}} call implicitly allows scan 
> results to be partial.
> Since {{VerifyReplication}} is assuming each {{scanner.next()}} call returns 
> entire row,
> partial results break compare logic.
> We should avoid setBatch call here.
> Thanks to RPC chunking (explained in this blog 
> https://blogs.apache.org/hbase/entry/scan_improvements_in_hbase_1),
> it's safe and acceptable I think.



--
This message was sent by Atlassian JIRA
(v6.3.15#6346)


[jira] [Updated] (HBASE-17871) scan#setBatch(int) call leads wrong result of VerifyReplication

2017-04-03 Thread Tomu Tsuruhara (JIRA)

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

Tomu Tsuruhara updated HBASE-17871:
---
Attachment: HBASE-17871.master.001.patch

> scan#setBatch(int) call leads wrong result of VerifyReplication
> ---
>
> Key: HBASE-17871
> URL: https://issues.apache.org/jira/browse/HBASE-17871
> Project: HBase
>  Issue Type: Bug
>Affects Versions: 2.0.0, 1.4.0
>Reporter: Tomu Tsuruhara
>Assignee: Tomu Tsuruhara
>Priority: Minor
> Attachments: HBASE-17871.master.001.patch
>
>
> VerifyReplication tool printed weird logs.
> {noformat}
> 2017-04-03 23:30:50,252 ERROR [main] 
> org.apache.hadoop.hbase.mapreduce.replication.VerifyReplication: 
> CONTENT_DIFFERENT_ROWS, rowkey=a100193
> 2017-04-03 23:30:50,280 ERROR [main] 
> org.apache.hadoop.hbase.mapreduce.replication.VerifyReplication: 
> ONLY_IN_PEER_TABLE_ROWS, rowkey=a100193
> 2017-04-03 23:30:50,387 ERROR [main] 
> org.apache.hadoop.hbase.mapreduce.replication.VerifyReplication: 
> CONTENT_DIFFERENT_ROWS, rowkey=a100385
> 2017-04-03 23:30:50,414 ERROR [main] 
> org.apache.hadoop.hbase.mapreduce.replication.VerifyReplication: 
> ONLY_IN_PEER_TABLE_ROWS, rowkey=a100385
> 2017-04-03 23:30:50,480 ERROR [main] 
> org.apache.hadoop.hbase.mapreduce.replication.VerifyReplication: 
> CONTENT_DIFFERENT_ROWS, rowkey=a100532
> 2017-04-03 23:30:50,508 ERROR [main] 
> org.apache.hadoop.hbase.mapreduce.replication.VerifyReplication: 
> ONLY_IN_PEER_TABLE_ROWS, rowkey=a100532
> {noformat}
> Here, each bad rows were marked as both {{CONTENT_DIFFERENT_ROWS}} and 
> {{ONLY_IN_PEER_TABLE_ROWS}}.
> This should never happen so I took a look at code and found scan.setBatch 
> call.
> {code}
> @Override
> public void map(ImmutableBytesWritable row, final Result value,
> Context context)
> throws IOException {
>   if (replicatedScanner == null) {
>   ...
> final Scan scan = new Scan();
> scan.setBatch(batch);
> {code}
> As stated in HBASE-16376, {{scan#setBatch(int)}} call implicitly allows scan 
> results to be partial.
> Since {{VerifyReplication}} is assuming each {{scanner.next()}} call returns 
> entire row,
> partial results break compare logic.
> We should avoid setBatch call here.
> Thanks to RPC chunking (explained in this blog 
> https://blogs.apache.org/hbase/entry/scan_improvements_in_hbase_1),
> it's safe and acceptable I think.



--
This message was sent by Atlassian JIRA
(v6.3.15#6346)


[jira] [Updated] (HBASE-17871) scan#setBatch(int) call leads wrong result of VerifyReplication

2017-04-05 Thread Tomu Tsuruhara (JIRA)

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

Tomu Tsuruhara updated HBASE-17871:
---
Attachment: HBASE-17871.master.003.patch

> scan#setBatch(int) call leads wrong result of VerifyReplication
> ---
>
> Key: HBASE-17871
> URL: https://issues.apache.org/jira/browse/HBASE-17871
> Project: HBase
>  Issue Type: Bug
>Affects Versions: 2.0.0, 1.4.0
>Reporter: Tomu Tsuruhara
>Assignee: Tomu Tsuruhara
>Priority: Minor
> Attachments: HBASE-17871.master.001.patch, 
> HBASE-17871.master.002.patch, HBASE-17871.master.003.patch
>
>
> VerifyReplication tool printed weird logs.
> {noformat}
> 2017-04-03 23:30:50,252 ERROR [main] 
> org.apache.hadoop.hbase.mapreduce.replication.VerifyReplication: 
> CONTENT_DIFFERENT_ROWS, rowkey=a100193
> 2017-04-03 23:30:50,280 ERROR [main] 
> org.apache.hadoop.hbase.mapreduce.replication.VerifyReplication: 
> ONLY_IN_PEER_TABLE_ROWS, rowkey=a100193
> 2017-04-03 23:30:50,387 ERROR [main] 
> org.apache.hadoop.hbase.mapreduce.replication.VerifyReplication: 
> CONTENT_DIFFERENT_ROWS, rowkey=a100385
> 2017-04-03 23:30:50,414 ERROR [main] 
> org.apache.hadoop.hbase.mapreduce.replication.VerifyReplication: 
> ONLY_IN_PEER_TABLE_ROWS, rowkey=a100385
> 2017-04-03 23:30:50,480 ERROR [main] 
> org.apache.hadoop.hbase.mapreduce.replication.VerifyReplication: 
> CONTENT_DIFFERENT_ROWS, rowkey=a100532
> 2017-04-03 23:30:50,508 ERROR [main] 
> org.apache.hadoop.hbase.mapreduce.replication.VerifyReplication: 
> ONLY_IN_PEER_TABLE_ROWS, rowkey=a100532
> {noformat}
> Here, each bad rows were marked as both {{CONTENT_DIFFERENT_ROWS}} and 
> {{ONLY_IN_PEER_TABLE_ROWS}}.
> This should never happen so I took a look at code and found scan.setBatch 
> call.
> {code}
> @Override
> public void map(ImmutableBytesWritable row, final Result value,
> Context context)
> throws IOException {
>   if (replicatedScanner == null) {
>   ...
> final Scan scan = new Scan();
> scan.setBatch(batch);
> {code}
> As stated in HBASE-16376, {{scan#setBatch(int)}} call implicitly allows scan 
> results to be partial.
> Since {{VerifyReplication}} is assuming each {{scanner.next()}} call returns 
> entire row,
> partial results break compare logic.
> We should avoid setBatch call here.
> Thanks to RPC chunking (explained in this blog 
> https://blogs.apache.org/hbase/entry/scan_improvements_in_hbase_1),
> it's safe and acceptable I think.



--
This message was sent by Atlassian JIRA
(v6.3.15#6346)


[jira] [Updated] (HBASE-17871) scan#setBatch(int) call leads wrong result of VerifyReplication

2017-04-05 Thread Tomu Tsuruhara (JIRA)

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

Tomu Tsuruhara updated HBASE-17871:
---
Attachment: スクリーンショット 2017-04-06 7.21.53.png
スクリーンショット 2017-04-06 7.19.53.png

> scan#setBatch(int) call leads wrong result of VerifyReplication
> ---
>
> Key: HBASE-17871
> URL: https://issues.apache.org/jira/browse/HBASE-17871
> Project: HBase
>  Issue Type: Bug
>Affects Versions: 2.0.0, 1.4.0
>Reporter: Tomu Tsuruhara
>Assignee: Tomu Tsuruhara
>Priority: Minor
> Attachments: スクリーンショット 2017-04-06 7.19.53.png, スクリーンショット 2017-04-06 
> 7.21.53.png, HBASE-17871.master.001.patch, HBASE-17871.master.002.patch, 
> HBASE-17871.master.003.patch
>
>
> VerifyReplication tool printed weird logs.
> {noformat}
> 2017-04-03 23:30:50,252 ERROR [main] 
> org.apache.hadoop.hbase.mapreduce.replication.VerifyReplication: 
> CONTENT_DIFFERENT_ROWS, rowkey=a100193
> 2017-04-03 23:30:50,280 ERROR [main] 
> org.apache.hadoop.hbase.mapreduce.replication.VerifyReplication: 
> ONLY_IN_PEER_TABLE_ROWS, rowkey=a100193
> 2017-04-03 23:30:50,387 ERROR [main] 
> org.apache.hadoop.hbase.mapreduce.replication.VerifyReplication: 
> CONTENT_DIFFERENT_ROWS, rowkey=a100385
> 2017-04-03 23:30:50,414 ERROR [main] 
> org.apache.hadoop.hbase.mapreduce.replication.VerifyReplication: 
> ONLY_IN_PEER_TABLE_ROWS, rowkey=a100385
> 2017-04-03 23:30:50,480 ERROR [main] 
> org.apache.hadoop.hbase.mapreduce.replication.VerifyReplication: 
> CONTENT_DIFFERENT_ROWS, rowkey=a100532
> 2017-04-03 23:30:50,508 ERROR [main] 
> org.apache.hadoop.hbase.mapreduce.replication.VerifyReplication: 
> ONLY_IN_PEER_TABLE_ROWS, rowkey=a100532
> {noformat}
> Here, each bad rows were marked as both {{CONTENT_DIFFERENT_ROWS}} and 
> {{ONLY_IN_PEER_TABLE_ROWS}}.
> This should never happen so I took a look at code and found scan.setBatch 
> call.
> {code}
> @Override
> public void map(ImmutableBytesWritable row, final Result value,
> Context context)
> throws IOException {
>   if (replicatedScanner == null) {
>   ...
> final Scan scan = new Scan();
> scan.setBatch(batch);
> {code}
> As stated in HBASE-16376, {{scan#setBatch(int)}} call implicitly allows scan 
> results to be partial.
> Since {{VerifyReplication}} is assuming each {{scanner.next()}} call returns 
> entire row,
> partial results break compare logic.
> We should avoid setBatch call here.
> Thanks to RPC chunking (explained in this blog 
> https://blogs.apache.org/hbase/entry/scan_improvements_in_hbase_1),
> it's safe and acceptable I think.



--
This message was sent by Atlassian JIRA
(v6.3.15#6346)


[jira] [Updated] (HBASE-17871) scan#setBatch(int) call leads wrong result of VerifyReplication

2017-04-06 Thread Tomu Tsuruhara (JIRA)

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

Tomu Tsuruhara updated HBASE-17871:
---
Attachment: HBASE-17871.master.004.patch

> scan#setBatch(int) call leads wrong result of VerifyReplication
> ---
>
> Key: HBASE-17871
> URL: https://issues.apache.org/jira/browse/HBASE-17871
> Project: HBase
>  Issue Type: Bug
>Affects Versions: 2.0.0, 1.4.0
>Reporter: Tomu Tsuruhara
>Assignee: Tomu Tsuruhara
>Priority: Minor
> Attachments: after.png, beforethepatch.png, 
> HBASE-17871.master.001.patch, HBASE-17871.master.002.patch, 
> HBASE-17871.master.003.patch, HBASE-17871.master.003.patch, 
> HBASE-17871.master.004.patch
>
>
> VerifyReplication tool printed weird logs.
> {noformat}
> 2017-04-03 23:30:50,252 ERROR [main] 
> org.apache.hadoop.hbase.mapreduce.replication.VerifyReplication: 
> CONTENT_DIFFERENT_ROWS, rowkey=a100193
> 2017-04-03 23:30:50,280 ERROR [main] 
> org.apache.hadoop.hbase.mapreduce.replication.VerifyReplication: 
> ONLY_IN_PEER_TABLE_ROWS, rowkey=a100193
> 2017-04-03 23:30:50,387 ERROR [main] 
> org.apache.hadoop.hbase.mapreduce.replication.VerifyReplication: 
> CONTENT_DIFFERENT_ROWS, rowkey=a100385
> 2017-04-03 23:30:50,414 ERROR [main] 
> org.apache.hadoop.hbase.mapreduce.replication.VerifyReplication: 
> ONLY_IN_PEER_TABLE_ROWS, rowkey=a100385
> 2017-04-03 23:30:50,480 ERROR [main] 
> org.apache.hadoop.hbase.mapreduce.replication.VerifyReplication: 
> CONTENT_DIFFERENT_ROWS, rowkey=a100532
> 2017-04-03 23:30:50,508 ERROR [main] 
> org.apache.hadoop.hbase.mapreduce.replication.VerifyReplication: 
> ONLY_IN_PEER_TABLE_ROWS, rowkey=a100532
> {noformat}
> Here, each bad rows were marked as both {{CONTENT_DIFFERENT_ROWS}} and 
> {{ONLY_IN_PEER_TABLE_ROWS}}.
> This should never happen so I took a look at code and found scan.setBatch 
> call.
> {code}
> @Override
> public void map(ImmutableBytesWritable row, final Result value,
> Context context)
> throws IOException {
>   if (replicatedScanner == null) {
>   ...
> final Scan scan = new Scan();
> scan.setBatch(batch);
> {code}
> As stated in HBASE-16376, {{scan#setBatch(int)}} call implicitly allows scan 
> results to be partial.
> Since {{VerifyReplication}} is assuming each {{scanner.next()}} call returns 
> entire row,
> partial results break compare logic.
> We should avoid setBatch call here.
> Thanks to RPC chunking (explained in this blog 
> https://blogs.apache.org/hbase/entry/scan_improvements_in_hbase_1),
> it's safe and acceptable I think.



--
This message was sent by Atlassian JIRA
(v6.3.15#6346)


[jira] [Commented] (HBASE-17871) scan#setBatch(int) call leads wrong result of VerifyReplication

2017-04-06 Thread Tomu Tsuruhara (JIRA)

[ 
https://issues.apache.org/jira/browse/HBASE-17871?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=15958761#comment-15958761
 ] 

Tomu Tsuruhara commented on HBASE-17871:


Oops.. sorry about that, attaching order.

And again, hadoop QA looks like failed ..
https://builds.apache.org/job/PreCommit-HBASE-Build/6348/console

{noformat}
Modes:  MultiJDK  Jenkins  Robot  Docker  ResetRepo  UnitTests 
Processing: HBASE-17871
ERROR: Unsure how to process HBASE-17871.
{noformat}

I'll attach the same patch again as v4.

> scan#setBatch(int) call leads wrong result of VerifyReplication
> ---
>
> Key: HBASE-17871
> URL: https://issues.apache.org/jira/browse/HBASE-17871
> Project: HBase
>  Issue Type: Bug
>Affects Versions: 2.0.0, 1.4.0
>Reporter: Tomu Tsuruhara
>Assignee: Tomu Tsuruhara
>Priority: Minor
> Attachments: after.png, beforethepatch.png, 
> HBASE-17871.master.001.patch, HBASE-17871.master.002.patch, 
> HBASE-17871.master.003.patch, HBASE-17871.master.003.patch, 
> HBASE-17871.master.004.patch
>
>
> VerifyReplication tool printed weird logs.
> {noformat}
> 2017-04-03 23:30:50,252 ERROR [main] 
> org.apache.hadoop.hbase.mapreduce.replication.VerifyReplication: 
> CONTENT_DIFFERENT_ROWS, rowkey=a100193
> 2017-04-03 23:30:50,280 ERROR [main] 
> org.apache.hadoop.hbase.mapreduce.replication.VerifyReplication: 
> ONLY_IN_PEER_TABLE_ROWS, rowkey=a100193
> 2017-04-03 23:30:50,387 ERROR [main] 
> org.apache.hadoop.hbase.mapreduce.replication.VerifyReplication: 
> CONTENT_DIFFERENT_ROWS, rowkey=a100385
> 2017-04-03 23:30:50,414 ERROR [main] 
> org.apache.hadoop.hbase.mapreduce.replication.VerifyReplication: 
> ONLY_IN_PEER_TABLE_ROWS, rowkey=a100385
> 2017-04-03 23:30:50,480 ERROR [main] 
> org.apache.hadoop.hbase.mapreduce.replication.VerifyReplication: 
> CONTENT_DIFFERENT_ROWS, rowkey=a100532
> 2017-04-03 23:30:50,508 ERROR [main] 
> org.apache.hadoop.hbase.mapreduce.replication.VerifyReplication: 
> ONLY_IN_PEER_TABLE_ROWS, rowkey=a100532
> {noformat}
> Here, each bad rows were marked as both {{CONTENT_DIFFERENT_ROWS}} and 
> {{ONLY_IN_PEER_TABLE_ROWS}}.
> This should never happen so I took a look at code and found scan.setBatch 
> call.
> {code}
> @Override
> public void map(ImmutableBytesWritable row, final Result value,
> Context context)
> throws IOException {
>   if (replicatedScanner == null) {
>   ...
> final Scan scan = new Scan();
> scan.setBatch(batch);
> {code}
> As stated in HBASE-16376, {{scan#setBatch(int)}} call implicitly allows scan 
> results to be partial.
> Since {{VerifyReplication}} is assuming each {{scanner.next()}} call returns 
> entire row,
> partial results break compare logic.
> We should avoid setBatch call here.
> Thanks to RPC chunking (explained in this blog 
> https://blogs.apache.org/hbase/entry/scan_improvements_in_hbase_1),
> it's safe and acceptable I think.



--
This message was sent by Atlassian JIRA
(v6.3.15#6346)


[jira] [Commented] (HBASE-17871) scan#setBatch(int) call leads wrong result of VerifyReplication

2017-04-06 Thread Tomu Tsuruhara (JIRA)

[ 
https://issues.apache.org/jira/browse/HBASE-17871?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=15958766#comment-15958766
 ] 

Tomu Tsuruhara commented on HBASE-17871:


[~yangzhe1991] Thanks! Yes, the patch can be applied to the branch-1 cleanly.

> scan#setBatch(int) call leads wrong result of VerifyReplication
> ---
>
> Key: HBASE-17871
> URL: https://issues.apache.org/jira/browse/HBASE-17871
> Project: HBase
>  Issue Type: Bug
>Affects Versions: 2.0.0, 1.4.0
>Reporter: Tomu Tsuruhara
>Assignee: Tomu Tsuruhara
>Priority: Minor
> Attachments: after.png, beforethepatch.png, 
> HBASE-17871.master.001.patch, HBASE-17871.master.002.patch, 
> HBASE-17871.master.003.patch, HBASE-17871.master.003.patch, 
> HBASE-17871.master.004.patch
>
>
> VerifyReplication tool printed weird logs.
> {noformat}
> 2017-04-03 23:30:50,252 ERROR [main] 
> org.apache.hadoop.hbase.mapreduce.replication.VerifyReplication: 
> CONTENT_DIFFERENT_ROWS, rowkey=a100193
> 2017-04-03 23:30:50,280 ERROR [main] 
> org.apache.hadoop.hbase.mapreduce.replication.VerifyReplication: 
> ONLY_IN_PEER_TABLE_ROWS, rowkey=a100193
> 2017-04-03 23:30:50,387 ERROR [main] 
> org.apache.hadoop.hbase.mapreduce.replication.VerifyReplication: 
> CONTENT_DIFFERENT_ROWS, rowkey=a100385
> 2017-04-03 23:30:50,414 ERROR [main] 
> org.apache.hadoop.hbase.mapreduce.replication.VerifyReplication: 
> ONLY_IN_PEER_TABLE_ROWS, rowkey=a100385
> 2017-04-03 23:30:50,480 ERROR [main] 
> org.apache.hadoop.hbase.mapreduce.replication.VerifyReplication: 
> CONTENT_DIFFERENT_ROWS, rowkey=a100532
> 2017-04-03 23:30:50,508 ERROR [main] 
> org.apache.hadoop.hbase.mapreduce.replication.VerifyReplication: 
> ONLY_IN_PEER_TABLE_ROWS, rowkey=a100532
> {noformat}
> Here, each bad rows were marked as both {{CONTENT_DIFFERENT_ROWS}} and 
> {{ONLY_IN_PEER_TABLE_ROWS}}.
> This should never happen so I took a look at code and found scan.setBatch 
> call.
> {code}
> @Override
> public void map(ImmutableBytesWritable row, final Result value,
> Context context)
> throws IOException {
>   if (replicatedScanner == null) {
>   ...
> final Scan scan = new Scan();
> scan.setBatch(batch);
> {code}
> As stated in HBASE-16376, {{scan#setBatch(int)}} call implicitly allows scan 
> results to be partial.
> Since {{VerifyReplication}} is assuming each {{scanner.next()}} call returns 
> entire row,
> partial results break compare logic.
> We should avoid setBatch call here.
> Thanks to RPC chunking (explained in this blog 
> https://blogs.apache.org/hbase/entry/scan_improvements_in_hbase_1),
> it's safe and acceptable I think.



--
This message was sent by Atlassian JIRA
(v6.3.15#6346)


[jira] [Created] (HBASE-17773) VerifyReplication tool wrongly emits warning "ERROR: Invalid argument '--recomparesleep=xx'"

2017-03-11 Thread Tomu Tsuruhara (JIRA)
Tomu Tsuruhara created HBASE-17773:
--

 Summary: VerifyReplication tool wrongly emits warning "ERROR: 
Invalid argument '--recomparesleep=xx'"
 Key: HBASE-17773
 URL: https://issues.apache.org/jira/browse/HBASE-17773
 Project: HBase
  Issue Type: Bug
Affects Versions: 2.0.0
Reporter: Tomu Tsuruhara
Priority: Trivial


Even though it's completely valid, VerifyReplication tool says "Invalid 
argument" when specifying {{\-\-recomparesleep}} or {{\-\-delimiter}} option.

{noformat}
$ bin/hbase org.apache.hadoop.hbase.mapreduce.replication.VerifyReplication 
--recomparesleep=5 1 foo
ERROR: Invalid argument '--recomparesleep=5'
Usage: verifyrep [--starttime=X] [--endtime=Y] [--families=A] 
[--row-prefixes=B] [--delimiter=] [--recomparesleep=] [--verbose]  


Options:
 starttimebeginning of the time range
  without endtime means from starttime to forever
 endtime  end of the time range
 versions number of cell versions to verify
 raw  includes raw scan if given in options
 families comma-separated list of families to copy
 row-prefixes comma-separated list of row key prefixes to filter on
 delimiterthe delimiter used in display around rowkey
 recomparesleep   milliseconds to sleep before recompare row, default value is 
0 which disables the recompare.
 verbose  logs row keys of good rows

Args:
 peerid   Id of the peer used for verification, must match the one given 
for replication
 tablenameName of the table to verify

Examples:
 To verify the data replicated from TestTable for a 1 hour window with peer #5
 $ hbase org.apache.hadoop.hbase.mapreduce.replication.VerifyReplication 
--starttime=1265875194289 --endtime=1265878794289 5 TestTable
{noformat}



--
This message was sent by Atlassian JIRA
(v6.3.15#6346)


[jira] [Updated] (HBASE-17773) VerifyReplication tool wrongly emits warning "ERROR: Invalid argument '--recomparesleep=xx'"

2017-03-11 Thread Tomu Tsuruhara (JIRA)

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

Tomu Tsuruhara updated HBASE-17773:
---
Attachment: HBASE-17773.master.001.patch

> VerifyReplication tool wrongly emits warning "ERROR: Invalid argument 
> '--recomparesleep=xx'"
> 
>
> Key: HBASE-17773
> URL: https://issues.apache.org/jira/browse/HBASE-17773
> Project: HBase
>  Issue Type: Bug
>Affects Versions: 2.0.0
>Reporter: Tomu Tsuruhara
>Priority: Trivial
> Attachments: HBASE-17773.master.001.patch
>
>
> Even though it's completely valid, VerifyReplication tool says "Invalid 
> argument" when specifying {{\-\-recomparesleep}} or {{\-\-delimiter}} option.
> {noformat}
> $ bin/hbase org.apache.hadoop.hbase.mapreduce.replication.VerifyReplication 
> --recomparesleep=5 1 foo
> ERROR: Invalid argument '--recomparesleep=5'
> Usage: verifyrep [--starttime=X] [--endtime=Y] [--families=A] 
> [--row-prefixes=B] [--delimiter=] [--recomparesleep=] [--verbose]  
> 
> Options:
>  starttimebeginning of the time range
>   without endtime means from starttime to forever
>  endtime  end of the time range
>  versions number of cell versions to verify
>  raw  includes raw scan if given in options
>  families comma-separated list of families to copy
>  row-prefixes comma-separated list of row key prefixes to filter on
>  delimiterthe delimiter used in display around rowkey
>  recomparesleep   milliseconds to sleep before recompare row, default value 
> is 0 which disables the recompare.
>  verbose  logs row keys of good rows
> Args:
>  peerid   Id of the peer used for verification, must match the one given 
> for replication
>  tablenameName of the table to verify
> Examples:
>  To verify the data replicated from TestTable for a 1 hour window with peer #5
>  $ hbase org.apache.hadoop.hbase.mapreduce.replication.VerifyReplication 
> --starttime=1265875194289 --endtime=1265878794289 5 TestTable
> {noformat}



--
This message was sent by Atlassian JIRA
(v6.3.15#6346)


[jira] [Work started] (HBASE-17773) VerifyReplication tool wrongly emits warning "ERROR: Invalid argument '--recomparesleep=xx'"

2017-03-11 Thread Tomu Tsuruhara (JIRA)

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

Work on HBASE-17773 started by Tomu Tsuruhara.
--
> VerifyReplication tool wrongly emits warning "ERROR: Invalid argument 
> '--recomparesleep=xx'"
> 
>
> Key: HBASE-17773
> URL: https://issues.apache.org/jira/browse/HBASE-17773
> Project: HBase
>  Issue Type: Bug
>Affects Versions: 2.0.0
>Reporter: Tomu Tsuruhara
>Assignee: Tomu Tsuruhara
>Priority: Trivial
> Attachments: HBASE-17773.master.001.patch
>
>
> Even though it's completely valid, VerifyReplication tool says "Invalid 
> argument" when specifying {{\-\-recomparesleep}} or {{\-\-delimiter}} option.
> {noformat}
> $ bin/hbase org.apache.hadoop.hbase.mapreduce.replication.VerifyReplication 
> --recomparesleep=5 1 foo
> ERROR: Invalid argument '--recomparesleep=5'
> Usage: verifyrep [--starttime=X] [--endtime=Y] [--families=A] 
> [--row-prefixes=B] [--delimiter=] [--recomparesleep=] [--verbose]  
> 
> Options:
>  starttimebeginning of the time range
>   without endtime means from starttime to forever
>  endtime  end of the time range
>  versions number of cell versions to verify
>  raw  includes raw scan if given in options
>  families comma-separated list of families to copy
>  row-prefixes comma-separated list of row key prefixes to filter on
>  delimiterthe delimiter used in display around rowkey
>  recomparesleep   milliseconds to sleep before recompare row, default value 
> is 0 which disables the recompare.
>  verbose  logs row keys of good rows
> Args:
>  peerid   Id of the peer used for verification, must match the one given 
> for replication
>  tablenameName of the table to verify
> Examples:
>  To verify the data replicated from TestTable for a 1 hour window with peer #5
>  $ hbase org.apache.hadoop.hbase.mapreduce.replication.VerifyReplication 
> --starttime=1265875194289 --endtime=1265878794289 5 TestTable
> {noformat}



--
This message was sent by Atlassian JIRA
(v6.3.15#6346)


[jira] [Updated] (HBASE-17773) VerifyReplication tool wrongly emits warning "ERROR: Invalid argument '--recomparesleep=xx'"

2017-03-11 Thread Tomu Tsuruhara (JIRA)

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

Tomu Tsuruhara updated HBASE-17773:
---
Status: Patch Available  (was: In Progress)

> VerifyReplication tool wrongly emits warning "ERROR: Invalid argument 
> '--recomparesleep=xx'"
> 
>
> Key: HBASE-17773
> URL: https://issues.apache.org/jira/browse/HBASE-17773
> Project: HBase
>  Issue Type: Bug
>Affects Versions: 2.0.0
>Reporter: Tomu Tsuruhara
>Assignee: Tomu Tsuruhara
>Priority: Trivial
> Attachments: HBASE-17773.master.001.patch
>
>
> Even though it's completely valid, VerifyReplication tool says "Invalid 
> argument" when specifying {{\-\-recomparesleep}} or {{\-\-delimiter}} option.
> {noformat}
> $ bin/hbase org.apache.hadoop.hbase.mapreduce.replication.VerifyReplication 
> --recomparesleep=5 1 foo
> ERROR: Invalid argument '--recomparesleep=5'
> Usage: verifyrep [--starttime=X] [--endtime=Y] [--families=A] 
> [--row-prefixes=B] [--delimiter=] [--recomparesleep=] [--verbose]  
> 
> Options:
>  starttimebeginning of the time range
>   without endtime means from starttime to forever
>  endtime  end of the time range
>  versions number of cell versions to verify
>  raw  includes raw scan if given in options
>  families comma-separated list of families to copy
>  row-prefixes comma-separated list of row key prefixes to filter on
>  delimiterthe delimiter used in display around rowkey
>  recomparesleep   milliseconds to sleep before recompare row, default value 
> is 0 which disables the recompare.
>  verbose  logs row keys of good rows
> Args:
>  peerid   Id of the peer used for verification, must match the one given 
> for replication
>  tablenameName of the table to verify
> Examples:
>  To verify the data replicated from TestTable for a 1 hour window with peer #5
>  $ hbase org.apache.hadoop.hbase.mapreduce.replication.VerifyReplication 
> --starttime=1265875194289 --endtime=1265878794289 5 TestTable
> {noformat}



--
This message was sent by Atlassian JIRA
(v6.3.15#6346)


[jira] [Assigned] (HBASE-17773) VerifyReplication tool wrongly emits warning "ERROR: Invalid argument '--recomparesleep=xx'"

2017-03-11 Thread Tomu Tsuruhara (JIRA)

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

Tomu Tsuruhara reassigned HBASE-17773:
--

Assignee: Tomu Tsuruhara

> VerifyReplication tool wrongly emits warning "ERROR: Invalid argument 
> '--recomparesleep=xx'"
> 
>
> Key: HBASE-17773
> URL: https://issues.apache.org/jira/browse/HBASE-17773
> Project: HBase
>  Issue Type: Bug
>Affects Versions: 2.0.0
>Reporter: Tomu Tsuruhara
>Assignee: Tomu Tsuruhara
>Priority: Trivial
> Attachments: HBASE-17773.master.001.patch
>
>
> Even though it's completely valid, VerifyReplication tool says "Invalid 
> argument" when specifying {{\-\-recomparesleep}} or {{\-\-delimiter}} option.
> {noformat}
> $ bin/hbase org.apache.hadoop.hbase.mapreduce.replication.VerifyReplication 
> --recomparesleep=5 1 foo
> ERROR: Invalid argument '--recomparesleep=5'
> Usage: verifyrep [--starttime=X] [--endtime=Y] [--families=A] 
> [--row-prefixes=B] [--delimiter=] [--recomparesleep=] [--verbose]  
> 
> Options:
>  starttimebeginning of the time range
>   without endtime means from starttime to forever
>  endtime  end of the time range
>  versions number of cell versions to verify
>  raw  includes raw scan if given in options
>  families comma-separated list of families to copy
>  row-prefixes comma-separated list of row key prefixes to filter on
>  delimiterthe delimiter used in display around rowkey
>  recomparesleep   milliseconds to sleep before recompare row, default value 
> is 0 which disables the recompare.
>  verbose  logs row keys of good rows
> Args:
>  peerid   Id of the peer used for verification, must match the one given 
> for replication
>  tablenameName of the table to verify
> Examples:
>  To verify the data replicated from TestTable for a 1 hour window with peer #5
>  $ hbase org.apache.hadoop.hbase.mapreduce.replication.VerifyReplication 
> --starttime=1265875194289 --endtime=1265878794289 5 TestTable
> {noformat}



--
This message was sent by Atlassian JIRA
(v6.3.15#6346)


[jira] [Updated] (HBASE-17871) scan#setBatch(int) call leads wrong result of VerifyReplication

2017-04-05 Thread Tomu Tsuruhara (JIRA)

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

Tomu Tsuruhara updated HBASE-17871:
---
Attachment: after.png
beforethepatch.png

> scan#setBatch(int) call leads wrong result of VerifyReplication
> ---
>
> Key: HBASE-17871
> URL: https://issues.apache.org/jira/browse/HBASE-17871
> Project: HBase
>  Issue Type: Bug
>Affects Versions: 2.0.0, 1.4.0
>Reporter: Tomu Tsuruhara
>Assignee: Tomu Tsuruhara
>Priority: Minor
> Attachments: after.png, beforethepatch.png, 
> HBASE-17871.master.001.patch, HBASE-17871.master.002.patch, 
> HBASE-17871.master.003.patch
>
>
> VerifyReplication tool printed weird logs.
> {noformat}
> 2017-04-03 23:30:50,252 ERROR [main] 
> org.apache.hadoop.hbase.mapreduce.replication.VerifyReplication: 
> CONTENT_DIFFERENT_ROWS, rowkey=a100193
> 2017-04-03 23:30:50,280 ERROR [main] 
> org.apache.hadoop.hbase.mapreduce.replication.VerifyReplication: 
> ONLY_IN_PEER_TABLE_ROWS, rowkey=a100193
> 2017-04-03 23:30:50,387 ERROR [main] 
> org.apache.hadoop.hbase.mapreduce.replication.VerifyReplication: 
> CONTENT_DIFFERENT_ROWS, rowkey=a100385
> 2017-04-03 23:30:50,414 ERROR [main] 
> org.apache.hadoop.hbase.mapreduce.replication.VerifyReplication: 
> ONLY_IN_PEER_TABLE_ROWS, rowkey=a100385
> 2017-04-03 23:30:50,480 ERROR [main] 
> org.apache.hadoop.hbase.mapreduce.replication.VerifyReplication: 
> CONTENT_DIFFERENT_ROWS, rowkey=a100532
> 2017-04-03 23:30:50,508 ERROR [main] 
> org.apache.hadoop.hbase.mapreduce.replication.VerifyReplication: 
> ONLY_IN_PEER_TABLE_ROWS, rowkey=a100532
> {noformat}
> Here, each bad rows were marked as both {{CONTENT_DIFFERENT_ROWS}} and 
> {{ONLY_IN_PEER_TABLE_ROWS}}.
> This should never happen so I took a look at code and found scan.setBatch 
> call.
> {code}
> @Override
> public void map(ImmutableBytesWritable row, final Result value,
> Context context)
> throws IOException {
>   if (replicatedScanner == null) {
>   ...
> final Scan scan = new Scan();
> scan.setBatch(batch);
> {code}
> As stated in HBASE-16376, {{scan#setBatch(int)}} call implicitly allows scan 
> results to be partial.
> Since {{VerifyReplication}} is assuming each {{scanner.next()}} call returns 
> entire row,
> partial results break compare logic.
> We should avoid setBatch call here.
> Thanks to RPC chunking (explained in this blog 
> https://blogs.apache.org/hbase/entry/scan_improvements_in_hbase_1),
> it's safe and acceptable I think.



--
This message was sent by Atlassian JIRA
(v6.3.15#6346)


[jira] [Comment Edited] (HBASE-17871) scan#setBatch(int) call leads wrong result of VerifyReplication

2017-04-05 Thread Tomu Tsuruhara (JIRA)

[ 
https://issues.apache.org/jira/browse/HBASE-17871?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=15957920#comment-15957920
 ] 

Tomu Tsuruhara edited comment on HBASE-17871 at 4/5/17 10:38 PM:
-

Thanks! Addressed findbugs warning in v3 patch.

"--batch" parsing code was introduced in HBASE-15191.
We don't need to put extra code this time.

> Have you verified the change fixes the inconsistency ?

Yes, I confirmed that in my environment.

before the patch: 
!beforethepatch.png|width=600!

after:
!after.png|width=600!



was (Author: tomu.tsuruhara):
Thanks! Addressed findbugs warning in v3 patch.

"--batch" parsing code was introduced in HBASE-15191.
We don't need to put extra code this time.

> Have you verified the change fixes the inconsistency ?

Yes, I confirmed that in my environment.

before the patch: 
!beforethepatch.png|width=100!

after:
!after.png|width=100!


> scan#setBatch(int) call leads wrong result of VerifyReplication
> ---
>
> Key: HBASE-17871
> URL: https://issues.apache.org/jira/browse/HBASE-17871
> Project: HBase
>  Issue Type: Bug
>Affects Versions: 2.0.0, 1.4.0
>Reporter: Tomu Tsuruhara
>Assignee: Tomu Tsuruhara
>Priority: Minor
> Attachments: after.png, beforethepatch.png, 
> HBASE-17871.master.001.patch, HBASE-17871.master.002.patch, 
> HBASE-17871.master.003.patch
>
>
> VerifyReplication tool printed weird logs.
> {noformat}
> 2017-04-03 23:30:50,252 ERROR [main] 
> org.apache.hadoop.hbase.mapreduce.replication.VerifyReplication: 
> CONTENT_DIFFERENT_ROWS, rowkey=a100193
> 2017-04-03 23:30:50,280 ERROR [main] 
> org.apache.hadoop.hbase.mapreduce.replication.VerifyReplication: 
> ONLY_IN_PEER_TABLE_ROWS, rowkey=a100193
> 2017-04-03 23:30:50,387 ERROR [main] 
> org.apache.hadoop.hbase.mapreduce.replication.VerifyReplication: 
> CONTENT_DIFFERENT_ROWS, rowkey=a100385
> 2017-04-03 23:30:50,414 ERROR [main] 
> org.apache.hadoop.hbase.mapreduce.replication.VerifyReplication: 
> ONLY_IN_PEER_TABLE_ROWS, rowkey=a100385
> 2017-04-03 23:30:50,480 ERROR [main] 
> org.apache.hadoop.hbase.mapreduce.replication.VerifyReplication: 
> CONTENT_DIFFERENT_ROWS, rowkey=a100532
> 2017-04-03 23:30:50,508 ERROR [main] 
> org.apache.hadoop.hbase.mapreduce.replication.VerifyReplication: 
> ONLY_IN_PEER_TABLE_ROWS, rowkey=a100532
> {noformat}
> Here, each bad rows were marked as both {{CONTENT_DIFFERENT_ROWS}} and 
> {{ONLY_IN_PEER_TABLE_ROWS}}.
> This should never happen so I took a look at code and found scan.setBatch 
> call.
> {code}
> @Override
> public void map(ImmutableBytesWritable row, final Result value,
> Context context)
> throws IOException {
>   if (replicatedScanner == null) {
>   ...
> final Scan scan = new Scan();
> scan.setBatch(batch);
> {code}
> As stated in HBASE-16376, {{scan#setBatch(int)}} call implicitly allows scan 
> results to be partial.
> Since {{VerifyReplication}} is assuming each {{scanner.next()}} call returns 
> entire row,
> partial results break compare logic.
> We should avoid setBatch call here.
> Thanks to RPC chunking (explained in this blog 
> https://blogs.apache.org/hbase/entry/scan_improvements_in_hbase_1),
> it's safe and acceptable I think.



--
This message was sent by Atlassian JIRA
(v6.3.15#6346)


[jira] [Comment Edited] (HBASE-17871) scan#setBatch(int) call leads wrong result of VerifyReplication

2017-04-05 Thread Tomu Tsuruhara (JIRA)

[ 
https://issues.apache.org/jira/browse/HBASE-17871?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=15957920#comment-15957920
 ] 

Tomu Tsuruhara edited comment on HBASE-17871 at 4/5/17 10:38 PM:
-

Thanks! Addressed findbugs warning in v3 patch.

"--batch" parsing code was introduced in HBASE-15191.
We don't need to put extra code this time.

> Have you verified the change fixes the inconsistency ?

Yes, I confirmed that in my environment.

before the patch: 
!beforethepatch.png|width=100!

after:
!after.png|width=100!



was (Author: tomu.tsuruhara):
Thanks! Addressed findbugs warning in v3 patch.

"--batch" parsing code was introduced in HBASE-15191.
We don't need to put extra code this time.

> Have you verified the change fixes the inconsistency ?

Yes, I confirmed that in my environment.

before the patch: 
!beforethepatch.png|thumbnail!

after:
!after.png|thumbnail!

> scan#setBatch(int) call leads wrong result of VerifyReplication
> ---
>
> Key: HBASE-17871
> URL: https://issues.apache.org/jira/browse/HBASE-17871
> Project: HBase
>  Issue Type: Bug
>Affects Versions: 2.0.0, 1.4.0
>Reporter: Tomu Tsuruhara
>Assignee: Tomu Tsuruhara
>Priority: Minor
> Attachments: after.png, beforethepatch.png, 
> HBASE-17871.master.001.patch, HBASE-17871.master.002.patch, 
> HBASE-17871.master.003.patch
>
>
> VerifyReplication tool printed weird logs.
> {noformat}
> 2017-04-03 23:30:50,252 ERROR [main] 
> org.apache.hadoop.hbase.mapreduce.replication.VerifyReplication: 
> CONTENT_DIFFERENT_ROWS, rowkey=a100193
> 2017-04-03 23:30:50,280 ERROR [main] 
> org.apache.hadoop.hbase.mapreduce.replication.VerifyReplication: 
> ONLY_IN_PEER_TABLE_ROWS, rowkey=a100193
> 2017-04-03 23:30:50,387 ERROR [main] 
> org.apache.hadoop.hbase.mapreduce.replication.VerifyReplication: 
> CONTENT_DIFFERENT_ROWS, rowkey=a100385
> 2017-04-03 23:30:50,414 ERROR [main] 
> org.apache.hadoop.hbase.mapreduce.replication.VerifyReplication: 
> ONLY_IN_PEER_TABLE_ROWS, rowkey=a100385
> 2017-04-03 23:30:50,480 ERROR [main] 
> org.apache.hadoop.hbase.mapreduce.replication.VerifyReplication: 
> CONTENT_DIFFERENT_ROWS, rowkey=a100532
> 2017-04-03 23:30:50,508 ERROR [main] 
> org.apache.hadoop.hbase.mapreduce.replication.VerifyReplication: 
> ONLY_IN_PEER_TABLE_ROWS, rowkey=a100532
> {noformat}
> Here, each bad rows were marked as both {{CONTENT_DIFFERENT_ROWS}} and 
> {{ONLY_IN_PEER_TABLE_ROWS}}.
> This should never happen so I took a look at code and found scan.setBatch 
> call.
> {code}
> @Override
> public void map(ImmutableBytesWritable row, final Result value,
> Context context)
> throws IOException {
>   if (replicatedScanner == null) {
>   ...
> final Scan scan = new Scan();
> scan.setBatch(batch);
> {code}
> As stated in HBASE-16376, {{scan#setBatch(int)}} call implicitly allows scan 
> results to be partial.
> Since {{VerifyReplication}} is assuming each {{scanner.next()}} call returns 
> entire row,
> partial results break compare logic.
> We should avoid setBatch call here.
> Thanks to RPC chunking (explained in this blog 
> https://blogs.apache.org/hbase/entry/scan_improvements_in_hbase_1),
> it's safe and acceptable I think.



--
This message was sent by Atlassian JIRA
(v6.3.15#6346)


[jira] [Updated] (HBASE-17871) scan#setBatch(int) call leads wrong result of VerifyReplication

2017-04-05 Thread Tomu Tsuruhara (JIRA)

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

Tomu Tsuruhara updated HBASE-17871:
---
Attachment: (was: スクリーンショット 2017-04-06 7.21.53.png)

> scan#setBatch(int) call leads wrong result of VerifyReplication
> ---
>
> Key: HBASE-17871
> URL: https://issues.apache.org/jira/browse/HBASE-17871
> Project: HBase
>  Issue Type: Bug
>Affects Versions: 2.0.0, 1.4.0
>Reporter: Tomu Tsuruhara
>Assignee: Tomu Tsuruhara
>Priority: Minor
> Attachments: HBASE-17871.master.001.patch, 
> HBASE-17871.master.002.patch, HBASE-17871.master.003.patch
>
>
> VerifyReplication tool printed weird logs.
> {noformat}
> 2017-04-03 23:30:50,252 ERROR [main] 
> org.apache.hadoop.hbase.mapreduce.replication.VerifyReplication: 
> CONTENT_DIFFERENT_ROWS, rowkey=a100193
> 2017-04-03 23:30:50,280 ERROR [main] 
> org.apache.hadoop.hbase.mapreduce.replication.VerifyReplication: 
> ONLY_IN_PEER_TABLE_ROWS, rowkey=a100193
> 2017-04-03 23:30:50,387 ERROR [main] 
> org.apache.hadoop.hbase.mapreduce.replication.VerifyReplication: 
> CONTENT_DIFFERENT_ROWS, rowkey=a100385
> 2017-04-03 23:30:50,414 ERROR [main] 
> org.apache.hadoop.hbase.mapreduce.replication.VerifyReplication: 
> ONLY_IN_PEER_TABLE_ROWS, rowkey=a100385
> 2017-04-03 23:30:50,480 ERROR [main] 
> org.apache.hadoop.hbase.mapreduce.replication.VerifyReplication: 
> CONTENT_DIFFERENT_ROWS, rowkey=a100532
> 2017-04-03 23:30:50,508 ERROR [main] 
> org.apache.hadoop.hbase.mapreduce.replication.VerifyReplication: 
> ONLY_IN_PEER_TABLE_ROWS, rowkey=a100532
> {noformat}
> Here, each bad rows were marked as both {{CONTENT_DIFFERENT_ROWS}} and 
> {{ONLY_IN_PEER_TABLE_ROWS}}.
> This should never happen so I took a look at code and found scan.setBatch 
> call.
> {code}
> @Override
> public void map(ImmutableBytesWritable row, final Result value,
> Context context)
> throws IOException {
>   if (replicatedScanner == null) {
>   ...
> final Scan scan = new Scan();
> scan.setBatch(batch);
> {code}
> As stated in HBASE-16376, {{scan#setBatch(int)}} call implicitly allows scan 
> results to be partial.
> Since {{VerifyReplication}} is assuming each {{scanner.next()}} call returns 
> entire row,
> partial results break compare logic.
> We should avoid setBatch call here.
> Thanks to RPC chunking (explained in this blog 
> https://blogs.apache.org/hbase/entry/scan_improvements_in_hbase_1),
> it's safe and acceptable I think.



--
This message was sent by Atlassian JIRA
(v6.3.15#6346)


[jira] [Commented] (HBASE-17871) scan#setBatch(int) call leads wrong result of VerifyReplication

2017-04-05 Thread Tomu Tsuruhara (JIRA)

[ 
https://issues.apache.org/jira/browse/HBASE-17871?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=15957920#comment-15957920
 ] 

Tomu Tsuruhara commented on HBASE-17871:


@Ted Yu Thanks! Addressed findbugs warning in v3 patch.

"--batch" parsing code was introduced in HBASE-15191.
We don't need to put extra code this time.

> Have you verified the change fixes the inconsistency ?

Yes, I confirmed that in my environment.

before the patch: 
!スクリーンショット 2017-04-06 7.19.53.png|thumbnail=1!

after:
!スクリーンショット 2017-04-06 7.21.53.png|thumbnail=1!

> scan#setBatch(int) call leads wrong result of VerifyReplication
> ---
>
> Key: HBASE-17871
> URL: https://issues.apache.org/jira/browse/HBASE-17871
> Project: HBase
>  Issue Type: Bug
>Affects Versions: 2.0.0, 1.4.0
>Reporter: Tomu Tsuruhara
>Assignee: Tomu Tsuruhara
>Priority: Minor
> Attachments: HBASE-17871.master.001.patch, 
> HBASE-17871.master.002.patch, HBASE-17871.master.003.patch
>
>
> VerifyReplication tool printed weird logs.
> {noformat}
> 2017-04-03 23:30:50,252 ERROR [main] 
> org.apache.hadoop.hbase.mapreduce.replication.VerifyReplication: 
> CONTENT_DIFFERENT_ROWS, rowkey=a100193
> 2017-04-03 23:30:50,280 ERROR [main] 
> org.apache.hadoop.hbase.mapreduce.replication.VerifyReplication: 
> ONLY_IN_PEER_TABLE_ROWS, rowkey=a100193
> 2017-04-03 23:30:50,387 ERROR [main] 
> org.apache.hadoop.hbase.mapreduce.replication.VerifyReplication: 
> CONTENT_DIFFERENT_ROWS, rowkey=a100385
> 2017-04-03 23:30:50,414 ERROR [main] 
> org.apache.hadoop.hbase.mapreduce.replication.VerifyReplication: 
> ONLY_IN_PEER_TABLE_ROWS, rowkey=a100385
> 2017-04-03 23:30:50,480 ERROR [main] 
> org.apache.hadoop.hbase.mapreduce.replication.VerifyReplication: 
> CONTENT_DIFFERENT_ROWS, rowkey=a100532
> 2017-04-03 23:30:50,508 ERROR [main] 
> org.apache.hadoop.hbase.mapreduce.replication.VerifyReplication: 
> ONLY_IN_PEER_TABLE_ROWS, rowkey=a100532
> {noformat}
> Here, each bad rows were marked as both {{CONTENT_DIFFERENT_ROWS}} and 
> {{ONLY_IN_PEER_TABLE_ROWS}}.
> This should never happen so I took a look at code and found scan.setBatch 
> call.
> {code}
> @Override
> public void map(ImmutableBytesWritable row, final Result value,
> Context context)
> throws IOException {
>   if (replicatedScanner == null) {
>   ...
> final Scan scan = new Scan();
> scan.setBatch(batch);
> {code}
> As stated in HBASE-16376, {{scan#setBatch(int)}} call implicitly allows scan 
> results to be partial.
> Since {{VerifyReplication}} is assuming each {{scanner.next()}} call returns 
> entire row,
> partial results break compare logic.
> We should avoid setBatch call here.
> Thanks to RPC chunking (explained in this blog 
> https://blogs.apache.org/hbase/entry/scan_improvements_in_hbase_1),
> it's safe and acceptable I think.



--
This message was sent by Atlassian JIRA
(v6.3.15#6346)


[jira] [Updated] (HBASE-17871) scan#setBatch(int) call leads wrong result of VerifyReplication

2017-04-05 Thread Tomu Tsuruhara (JIRA)

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

Tomu Tsuruhara updated HBASE-17871:
---
Attachment: (was: スクリーンショット 2017-04-06 7.19.53.png)

> scan#setBatch(int) call leads wrong result of VerifyReplication
> ---
>
> Key: HBASE-17871
> URL: https://issues.apache.org/jira/browse/HBASE-17871
> Project: HBase
>  Issue Type: Bug
>Affects Versions: 2.0.0, 1.4.0
>Reporter: Tomu Tsuruhara
>Assignee: Tomu Tsuruhara
>Priority: Minor
> Attachments: HBASE-17871.master.001.patch, 
> HBASE-17871.master.002.patch, HBASE-17871.master.003.patch
>
>
> VerifyReplication tool printed weird logs.
> {noformat}
> 2017-04-03 23:30:50,252 ERROR [main] 
> org.apache.hadoop.hbase.mapreduce.replication.VerifyReplication: 
> CONTENT_DIFFERENT_ROWS, rowkey=a100193
> 2017-04-03 23:30:50,280 ERROR [main] 
> org.apache.hadoop.hbase.mapreduce.replication.VerifyReplication: 
> ONLY_IN_PEER_TABLE_ROWS, rowkey=a100193
> 2017-04-03 23:30:50,387 ERROR [main] 
> org.apache.hadoop.hbase.mapreduce.replication.VerifyReplication: 
> CONTENT_DIFFERENT_ROWS, rowkey=a100385
> 2017-04-03 23:30:50,414 ERROR [main] 
> org.apache.hadoop.hbase.mapreduce.replication.VerifyReplication: 
> ONLY_IN_PEER_TABLE_ROWS, rowkey=a100385
> 2017-04-03 23:30:50,480 ERROR [main] 
> org.apache.hadoop.hbase.mapreduce.replication.VerifyReplication: 
> CONTENT_DIFFERENT_ROWS, rowkey=a100532
> 2017-04-03 23:30:50,508 ERROR [main] 
> org.apache.hadoop.hbase.mapreduce.replication.VerifyReplication: 
> ONLY_IN_PEER_TABLE_ROWS, rowkey=a100532
> {noformat}
> Here, each bad rows were marked as both {{CONTENT_DIFFERENT_ROWS}} and 
> {{ONLY_IN_PEER_TABLE_ROWS}}.
> This should never happen so I took a look at code and found scan.setBatch 
> call.
> {code}
> @Override
> public void map(ImmutableBytesWritable row, final Result value,
> Context context)
> throws IOException {
>   if (replicatedScanner == null) {
>   ...
> final Scan scan = new Scan();
> scan.setBatch(batch);
> {code}
> As stated in HBASE-16376, {{scan#setBatch(int)}} call implicitly allows scan 
> results to be partial.
> Since {{VerifyReplication}} is assuming each {{scanner.next()}} call returns 
> entire row,
> partial results break compare logic.
> We should avoid setBatch call here.
> Thanks to RPC chunking (explained in this blog 
> https://blogs.apache.org/hbase/entry/scan_improvements_in_hbase_1),
> it's safe and acceptable I think.



--
This message was sent by Atlassian JIRA
(v6.3.15#6346)


[jira] [Comment Edited] (HBASE-17871) scan#setBatch(int) call leads wrong result of VerifyReplication

2017-04-05 Thread Tomu Tsuruhara (JIRA)

[ 
https://issues.apache.org/jira/browse/HBASE-17871?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=15957920#comment-15957920
 ] 

Tomu Tsuruhara edited comment on HBASE-17871 at 4/5/17 10:36 PM:
-

Thanks! Addressed findbugs warning in v3 patch.

"--batch" parsing code was introduced in HBASE-15191.
We don't need to put extra code this time.

> Have you verified the change fixes the inconsistency ?

Yes, I confirmed that in my environment.

before the patch: 
!beforethepatch.png|thumbnail!

after:
!after.png|thumbnail!


was (Author: tomu.tsuruhara):
@Ted Yu Thanks! Addressed findbugs warning in v3 patch.

"--batch" parsing code was introduced in HBASE-15191.
We don't need to put extra code this time.

> Have you verified the change fixes the inconsistency ?

Yes, I confirmed that in my environment.

before the patch: 
!スクリーンショット 2017-04-06 7.19.53.png|thumbnail=1!

after:
!スクリーンショット 2017-04-06 7.21.53.png|thumbnail=1!

> scan#setBatch(int) call leads wrong result of VerifyReplication
> ---
>
> Key: HBASE-17871
> URL: https://issues.apache.org/jira/browse/HBASE-17871
> Project: HBase
>  Issue Type: Bug
>Affects Versions: 2.0.0, 1.4.0
>Reporter: Tomu Tsuruhara
>Assignee: Tomu Tsuruhara
>Priority: Minor
> Attachments: after.png, beforethepatch.png, 
> HBASE-17871.master.001.patch, HBASE-17871.master.002.patch, 
> HBASE-17871.master.003.patch
>
>
> VerifyReplication tool printed weird logs.
> {noformat}
> 2017-04-03 23:30:50,252 ERROR [main] 
> org.apache.hadoop.hbase.mapreduce.replication.VerifyReplication: 
> CONTENT_DIFFERENT_ROWS, rowkey=a100193
> 2017-04-03 23:30:50,280 ERROR [main] 
> org.apache.hadoop.hbase.mapreduce.replication.VerifyReplication: 
> ONLY_IN_PEER_TABLE_ROWS, rowkey=a100193
> 2017-04-03 23:30:50,387 ERROR [main] 
> org.apache.hadoop.hbase.mapreduce.replication.VerifyReplication: 
> CONTENT_DIFFERENT_ROWS, rowkey=a100385
> 2017-04-03 23:30:50,414 ERROR [main] 
> org.apache.hadoop.hbase.mapreduce.replication.VerifyReplication: 
> ONLY_IN_PEER_TABLE_ROWS, rowkey=a100385
> 2017-04-03 23:30:50,480 ERROR [main] 
> org.apache.hadoop.hbase.mapreduce.replication.VerifyReplication: 
> CONTENT_DIFFERENT_ROWS, rowkey=a100532
> 2017-04-03 23:30:50,508 ERROR [main] 
> org.apache.hadoop.hbase.mapreduce.replication.VerifyReplication: 
> ONLY_IN_PEER_TABLE_ROWS, rowkey=a100532
> {noformat}
> Here, each bad rows were marked as both {{CONTENT_DIFFERENT_ROWS}} and 
> {{ONLY_IN_PEER_TABLE_ROWS}}.
> This should never happen so I took a look at code and found scan.setBatch 
> call.
> {code}
> @Override
> public void map(ImmutableBytesWritable row, final Result value,
> Context context)
> throws IOException {
>   if (replicatedScanner == null) {
>   ...
> final Scan scan = new Scan();
> scan.setBatch(batch);
> {code}
> As stated in HBASE-16376, {{scan#setBatch(int)}} call implicitly allows scan 
> results to be partial.
> Since {{VerifyReplication}} is assuming each {{scanner.next()}} call returns 
> entire row,
> partial results break compare logic.
> We should avoid setBatch call here.
> Thanks to RPC chunking (explained in this blog 
> https://blogs.apache.org/hbase/entry/scan_improvements_in_hbase_1),
> it's safe and acceptable I think.



--
This message was sent by Atlassian JIRA
(v6.3.15#6346)


[jira] [Comment Edited] (HBASE-17871) scan#setBatch(int) call leads wrong result of VerifyReplication

2017-04-05 Thread Tomu Tsuruhara (JIRA)

[ 
https://issues.apache.org/jira/browse/HBASE-17871?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=15957920#comment-15957920
 ] 

Tomu Tsuruhara edited comment on HBASE-17871 at 4/5/17 10:40 PM:
-

[~yuzhih...@gmail.com] Thanks! Addressed findbugs warning in v3 patch.

"--batch" parsing code was introduced in HBASE-15191.
We don't need to put extra code this time.

> Have you verified the change fixes the inconsistency ?

Yes, I confirmed that in my environment.

before the patch: 
!beforethepatch.png|width=600!

after:
!after.png|width=600!



was (Author: tomu.tsuruhara):
[~tedyu] Thanks! Addressed findbugs warning in v3 patch.

"--batch" parsing code was introduced in HBASE-15191.
We don't need to put extra code this time.

> Have you verified the change fixes the inconsistency ?

Yes, I confirmed that in my environment.

before the patch: 
!beforethepatch.png|width=600!

after:
!after.png|width=600!


> scan#setBatch(int) call leads wrong result of VerifyReplication
> ---
>
> Key: HBASE-17871
> URL: https://issues.apache.org/jira/browse/HBASE-17871
> Project: HBase
>  Issue Type: Bug
>Affects Versions: 2.0.0, 1.4.0
>Reporter: Tomu Tsuruhara
>Assignee: Tomu Tsuruhara
>Priority: Minor
> Attachments: after.png, beforethepatch.png, 
> HBASE-17871.master.001.patch, HBASE-17871.master.002.patch, 
> HBASE-17871.master.003.patch
>
>
> VerifyReplication tool printed weird logs.
> {noformat}
> 2017-04-03 23:30:50,252 ERROR [main] 
> org.apache.hadoop.hbase.mapreduce.replication.VerifyReplication: 
> CONTENT_DIFFERENT_ROWS, rowkey=a100193
> 2017-04-03 23:30:50,280 ERROR [main] 
> org.apache.hadoop.hbase.mapreduce.replication.VerifyReplication: 
> ONLY_IN_PEER_TABLE_ROWS, rowkey=a100193
> 2017-04-03 23:30:50,387 ERROR [main] 
> org.apache.hadoop.hbase.mapreduce.replication.VerifyReplication: 
> CONTENT_DIFFERENT_ROWS, rowkey=a100385
> 2017-04-03 23:30:50,414 ERROR [main] 
> org.apache.hadoop.hbase.mapreduce.replication.VerifyReplication: 
> ONLY_IN_PEER_TABLE_ROWS, rowkey=a100385
> 2017-04-03 23:30:50,480 ERROR [main] 
> org.apache.hadoop.hbase.mapreduce.replication.VerifyReplication: 
> CONTENT_DIFFERENT_ROWS, rowkey=a100532
> 2017-04-03 23:30:50,508 ERROR [main] 
> org.apache.hadoop.hbase.mapreduce.replication.VerifyReplication: 
> ONLY_IN_PEER_TABLE_ROWS, rowkey=a100532
> {noformat}
> Here, each bad rows were marked as both {{CONTENT_DIFFERENT_ROWS}} and 
> {{ONLY_IN_PEER_TABLE_ROWS}}.
> This should never happen so I took a look at code and found scan.setBatch 
> call.
> {code}
> @Override
> public void map(ImmutableBytesWritable row, final Result value,
> Context context)
> throws IOException {
>   if (replicatedScanner == null) {
>   ...
> final Scan scan = new Scan();
> scan.setBatch(batch);
> {code}
> As stated in HBASE-16376, {{scan#setBatch(int)}} call implicitly allows scan 
> results to be partial.
> Since {{VerifyReplication}} is assuming each {{scanner.next()}} call returns 
> entire row,
> partial results break compare logic.
> We should avoid setBatch call here.
> Thanks to RPC chunking (explained in this blog 
> https://blogs.apache.org/hbase/entry/scan_improvements_in_hbase_1),
> it's safe and acceptable I think.



--
This message was sent by Atlassian JIRA
(v6.3.15#6346)


[jira] [Comment Edited] (HBASE-17871) scan#setBatch(int) call leads wrong result of VerifyReplication

2017-04-05 Thread Tomu Tsuruhara (JIRA)

[ 
https://issues.apache.org/jira/browse/HBASE-17871?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=15957920#comment-15957920
 ] 

Tomu Tsuruhara edited comment on HBASE-17871 at 4/5/17 10:39 PM:
-

[~tedyu] Thanks! Addressed findbugs warning in v3 patch.

"--batch" parsing code was introduced in HBASE-15191.
We don't need to put extra code this time.

> Have you verified the change fixes the inconsistency ?

Yes, I confirmed that in my environment.

before the patch: 
!beforethepatch.png|width=600!

after:
!after.png|width=600!



was (Author: tomu.tsuruhara):
Thanks! Addressed findbugs warning in v3 patch.

"--batch" parsing code was introduced in HBASE-15191.
We don't need to put extra code this time.

> Have you verified the change fixes the inconsistency ?

Yes, I confirmed that in my environment.

before the patch: 
!beforethepatch.png|width=600!

after:
!after.png|width=600!


> scan#setBatch(int) call leads wrong result of VerifyReplication
> ---
>
> Key: HBASE-17871
> URL: https://issues.apache.org/jira/browse/HBASE-17871
> Project: HBase
>  Issue Type: Bug
>Affects Versions: 2.0.0, 1.4.0
>Reporter: Tomu Tsuruhara
>Assignee: Tomu Tsuruhara
>Priority: Minor
> Attachments: after.png, beforethepatch.png, 
> HBASE-17871.master.001.patch, HBASE-17871.master.002.patch, 
> HBASE-17871.master.003.patch
>
>
> VerifyReplication tool printed weird logs.
> {noformat}
> 2017-04-03 23:30:50,252 ERROR [main] 
> org.apache.hadoop.hbase.mapreduce.replication.VerifyReplication: 
> CONTENT_DIFFERENT_ROWS, rowkey=a100193
> 2017-04-03 23:30:50,280 ERROR [main] 
> org.apache.hadoop.hbase.mapreduce.replication.VerifyReplication: 
> ONLY_IN_PEER_TABLE_ROWS, rowkey=a100193
> 2017-04-03 23:30:50,387 ERROR [main] 
> org.apache.hadoop.hbase.mapreduce.replication.VerifyReplication: 
> CONTENT_DIFFERENT_ROWS, rowkey=a100385
> 2017-04-03 23:30:50,414 ERROR [main] 
> org.apache.hadoop.hbase.mapreduce.replication.VerifyReplication: 
> ONLY_IN_PEER_TABLE_ROWS, rowkey=a100385
> 2017-04-03 23:30:50,480 ERROR [main] 
> org.apache.hadoop.hbase.mapreduce.replication.VerifyReplication: 
> CONTENT_DIFFERENT_ROWS, rowkey=a100532
> 2017-04-03 23:30:50,508 ERROR [main] 
> org.apache.hadoop.hbase.mapreduce.replication.VerifyReplication: 
> ONLY_IN_PEER_TABLE_ROWS, rowkey=a100532
> {noformat}
> Here, each bad rows were marked as both {{CONTENT_DIFFERENT_ROWS}} and 
> {{ONLY_IN_PEER_TABLE_ROWS}}.
> This should never happen so I took a look at code and found scan.setBatch 
> call.
> {code}
> @Override
> public void map(ImmutableBytesWritable row, final Result value,
> Context context)
> throws IOException {
>   if (replicatedScanner == null) {
>   ...
> final Scan scan = new Scan();
> scan.setBatch(batch);
> {code}
> As stated in HBASE-16376, {{scan#setBatch(int)}} call implicitly allows scan 
> results to be partial.
> Since {{VerifyReplication}} is assuming each {{scanner.next()}} call returns 
> entire row,
> partial results break compare logic.
> We should avoid setBatch call here.
> Thanks to RPC chunking (explained in this blog 
> https://blogs.apache.org/hbase/entry/scan_improvements_in_hbase_1),
> it's safe and acceptable I think.



--
This message was sent by Atlassian JIRA
(v6.3.15#6346)


[jira] [Updated] (HBASE-17871) scan#setBatch(int) call leads wrong result of VerifyReplication

2017-04-05 Thread Tomu Tsuruhara (JIRA)

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

Tomu Tsuruhara updated HBASE-17871:
---
Attachment: HBASE-17871.master.002.patch

> scan#setBatch(int) call leads wrong result of VerifyReplication
> ---
>
> Key: HBASE-17871
> URL: https://issues.apache.org/jira/browse/HBASE-17871
> Project: HBase
>  Issue Type: Bug
>Affects Versions: 2.0.0, 1.4.0
>Reporter: Tomu Tsuruhara
>Assignee: Tomu Tsuruhara
>Priority: Minor
> Attachments: HBASE-17871.master.001.patch, 
> HBASE-17871.master.002.patch
>
>
> VerifyReplication tool printed weird logs.
> {noformat}
> 2017-04-03 23:30:50,252 ERROR [main] 
> org.apache.hadoop.hbase.mapreduce.replication.VerifyReplication: 
> CONTENT_DIFFERENT_ROWS, rowkey=a100193
> 2017-04-03 23:30:50,280 ERROR [main] 
> org.apache.hadoop.hbase.mapreduce.replication.VerifyReplication: 
> ONLY_IN_PEER_TABLE_ROWS, rowkey=a100193
> 2017-04-03 23:30:50,387 ERROR [main] 
> org.apache.hadoop.hbase.mapreduce.replication.VerifyReplication: 
> CONTENT_DIFFERENT_ROWS, rowkey=a100385
> 2017-04-03 23:30:50,414 ERROR [main] 
> org.apache.hadoop.hbase.mapreduce.replication.VerifyReplication: 
> ONLY_IN_PEER_TABLE_ROWS, rowkey=a100385
> 2017-04-03 23:30:50,480 ERROR [main] 
> org.apache.hadoop.hbase.mapreduce.replication.VerifyReplication: 
> CONTENT_DIFFERENT_ROWS, rowkey=a100532
> 2017-04-03 23:30:50,508 ERROR [main] 
> org.apache.hadoop.hbase.mapreduce.replication.VerifyReplication: 
> ONLY_IN_PEER_TABLE_ROWS, rowkey=a100532
> {noformat}
> Here, each bad rows were marked as both {{CONTENT_DIFFERENT_ROWS}} and 
> {{ONLY_IN_PEER_TABLE_ROWS}}.
> This should never happen so I took a look at code and found scan.setBatch 
> call.
> {code}
> @Override
> public void map(ImmutableBytesWritable row, final Result value,
> Context context)
> throws IOException {
>   if (replicatedScanner == null) {
>   ...
> final Scan scan = new Scan();
> scan.setBatch(batch);
> {code}
> As stated in HBASE-16376, {{scan#setBatch(int)}} call implicitly allows scan 
> results to be partial.
> Since {{VerifyReplication}} is assuming each {{scanner.next()}} call returns 
> entire row,
> partial results break compare logic.
> We should avoid setBatch call here.
> Thanks to RPC chunking (explained in this blog 
> https://blogs.apache.org/hbase/entry/scan_improvements_in_hbase_1),
> it's safe and acceptable I think.



--
This message was sent by Atlassian JIRA
(v6.3.15#6346)


[jira] [Commented] (HBASE-17871) scan#setBatch(int) call leads wrong result of VerifyReplication

2017-04-05 Thread Tomu Tsuruhara (JIRA)

[ 
https://issues.apache.org/jira/browse/HBASE-17871?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=15956802#comment-15956802
 ] 

Tomu Tsuruhara commented on HBASE-17871:


[~yangzhe1991] Thanks for your comment!

You're right. {{--batch}} is helpful in some cases.
I changed my mind.

I just attached second patch.
Could you take a look? 

In the second patch, I
 * added {{setBatch}} call in {{createSubmittableJob}}
 * added {{setCacheBlocks}} call to prevent unexpected block cache churn on 
source cluster
 * changed default {{batch}} value to avoid implicit {{allowPartialResults}} 
effect

Thanks!

> scan#setBatch(int) call leads wrong result of VerifyReplication
> ---
>
> Key: HBASE-17871
> URL: https://issues.apache.org/jira/browse/HBASE-17871
> Project: HBase
>  Issue Type: Bug
>Affects Versions: 2.0.0, 1.4.0
>Reporter: Tomu Tsuruhara
>Assignee: Tomu Tsuruhara
>Priority: Minor
> Attachments: HBASE-17871.master.001.patch, 
> HBASE-17871.master.002.patch
>
>
> VerifyReplication tool printed weird logs.
> {noformat}
> 2017-04-03 23:30:50,252 ERROR [main] 
> org.apache.hadoop.hbase.mapreduce.replication.VerifyReplication: 
> CONTENT_DIFFERENT_ROWS, rowkey=a100193
> 2017-04-03 23:30:50,280 ERROR [main] 
> org.apache.hadoop.hbase.mapreduce.replication.VerifyReplication: 
> ONLY_IN_PEER_TABLE_ROWS, rowkey=a100193
> 2017-04-03 23:30:50,387 ERROR [main] 
> org.apache.hadoop.hbase.mapreduce.replication.VerifyReplication: 
> CONTENT_DIFFERENT_ROWS, rowkey=a100385
> 2017-04-03 23:30:50,414 ERROR [main] 
> org.apache.hadoop.hbase.mapreduce.replication.VerifyReplication: 
> ONLY_IN_PEER_TABLE_ROWS, rowkey=a100385
> 2017-04-03 23:30:50,480 ERROR [main] 
> org.apache.hadoop.hbase.mapreduce.replication.VerifyReplication: 
> CONTENT_DIFFERENT_ROWS, rowkey=a100532
> 2017-04-03 23:30:50,508 ERROR [main] 
> org.apache.hadoop.hbase.mapreduce.replication.VerifyReplication: 
> ONLY_IN_PEER_TABLE_ROWS, rowkey=a100532
> {noformat}
> Here, each bad rows were marked as both {{CONTENT_DIFFERENT_ROWS}} and 
> {{ONLY_IN_PEER_TABLE_ROWS}}.
> This should never happen so I took a look at code and found scan.setBatch 
> call.
> {code}
> @Override
> public void map(ImmutableBytesWritable row, final Result value,
> Context context)
> throws IOException {
>   if (replicatedScanner == null) {
>   ...
> final Scan scan = new Scan();
> scan.setBatch(batch);
> {code}
> As stated in HBASE-16376, {{scan#setBatch(int)}} call implicitly allows scan 
> results to be partial.
> Since {{VerifyReplication}} is assuming each {{scanner.next()}} call returns 
> entire row,
> partial results break compare logic.
> We should avoid setBatch call here.
> Thanks to RPC chunking (explained in this blog 
> https://blogs.apache.org/hbase/entry/scan_improvements_in_hbase_1),
> it's safe and acceptable I think.



--
This message was sent by Atlassian JIRA
(v6.3.15#6346)


[jira] [Commented] (HBASE-16616) Rpc handlers stuck on ThreadLocalMap.expungeStaleEntry

2017-06-20 Thread Tomu Tsuruhara (JIRA)

[ 
https://issues.apache.org/jira/browse/HBASE-16616?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=16056756#comment-16056756
 ] 

Tomu Tsuruhara commented on HBASE-16616:


[~dvdreddy] You're right. The patch here is not enough. We needed the patch in 
the HBASE-16146 to resolve the issue. 


> Rpc handlers stuck on ThreadLocalMap.expungeStaleEntry
> --
>
> Key: HBASE-16616
> URL: https://issues.apache.org/jira/browse/HBASE-16616
> Project: HBase
>  Issue Type: Improvement
>  Components: Performance
>Affects Versions: 1.2.2
>Reporter: Tomu Tsuruhara
>Assignee: Tomu Tsuruhara
> Fix For: 2.0.0, 1.4.0
>
> Attachments: 16616.branch-1.v2.txt, HBASE-16616.master.001.patch, 
> HBASE-16616.master.002.patch, ScreenShot 2016-09-09 14.17.53.png
>
>
> In our HBase 1.2.2 cluster, some regionserver showed too bad 
> "QueueCallTime_99th_percentile" exceeding 10 seconds.
> Most rpc handler threads stuck on ThreadLocalMap.expungeStaleEntry call at 
> that time.
> {noformat}
> "PriorityRpcServer.handler=18,queue=0,port=16020" #322 daemon prio=5 
> os_prio=0 tid=0x7fd422062800 nid=0x19b89 runnable [0x7fcb8a821000]
>java.lang.Thread.State: RUNNABLE
> at 
> java.lang.ThreadLocal$ThreadLocalMap.expungeStaleEntry(ThreadLocal.java:617)
> at java.lang.ThreadLocal$ThreadLocalMap.remove(ThreadLocal.java:499)
> at 
> java.lang.ThreadLocal$ThreadLocalMap.access$200(ThreadLocal.java:298)
> at java.lang.ThreadLocal.remove(ThreadLocal.java:222)
> at 
> java.util.concurrent.locks.ReentrantReadWriteLock$Sync.tryReleaseShared(ReentrantReadWriteLock.java:426)
> at 
> java.util.concurrent.locks.AbstractQueuedSynchronizer.releaseShared(AbstractQueuedSynchronizer.java:1341)
> at 
> java.util.concurrent.locks.ReentrantReadWriteLock$ReadLock.unlock(ReentrantReadWriteLock.java:881)
> at 
> com.yammer.metrics.stats.ExponentiallyDecayingSample.unlockForRegularUsage(ExponentiallyDecayingSample.java:196)
> at 
> com.yammer.metrics.stats.ExponentiallyDecayingSample.update(ExponentiallyDecayingSample.java:113)
> at 
> com.yammer.metrics.stats.ExponentiallyDecayingSample.update(ExponentiallyDecayingSample.java:81)
> at 
> org.apache.hadoop.metrics2.lib.MutableHistogram.add(MutableHistogram.java:81)
> at 
> org.apache.hadoop.metrics2.lib.MutableRangeHistogram.add(MutableRangeHistogram.java:59)
> at 
> org.apache.hadoop.hbase.ipc.MetricsHBaseServerSourceImpl.dequeuedCall(MetricsHBaseServerSourceImpl.java:194)
> at 
> org.apache.hadoop.hbase.ipc.MetricsHBaseServer.dequeuedCall(MetricsHBaseServer.java:76)
> at org.apache.hadoop.hbase.ipc.RpcServer.call(RpcServer.java:2192)
> at org.apache.hadoop.hbase.ipc.CallRunner.run(CallRunner.java:112)
> at 
> org.apache.hadoop.hbase.ipc.RpcExecutor.consumerLoop(RpcExecutor.java:133)
> at org.apache.hadoop.hbase.ipc.RpcExecutor$1.run(RpcExecutor.java:108)
> at java.lang.Thread.run(Thread.java:745)
> {noformat}
> We were using jdk 1.8.0_92 and here is a snippet from ThreadLocal.java.
> {code}
> 616:while (tab[h] != null)
> 617:h = nextIndex(h, len);
> {code}
> So I hypothesized that there're too many consecutive entries in {{tab}} array 
> and actually I found them in the heapdump.
> !ScreenShot 2016-09-09 14.17.53.png|width=50%!
> Most of these entries pointed at instance of 
> {{org.apache.hadoop.hbase.util.Counter$1}}
> which is equivarent to {{indexHolderThreadLocal}} instance-variable in the 
> {{Counter}} class.
> Because {{RpcServer$Connection}} class creates a {{Counter}} instance 
> {{rpcCount}} for every connections,
> it is possible to have lots of {{Counter#indexHolderThreadLocal}} instances 
> in RegionServer process
> when we repeat connect-and-close from client. As a result, a ThreadLocalMap 
> can have lots of consecutive
> entires.
> Usually, since each entry is a {{WeakReference}}, these entries are collected 
> and removed
> by garbage-collector soon after connection closed.
> But if connection's life-time was long enough to survive youngGC, it wouldn't 
> be collected until old-gen collector runs.
> Furthermore, under G1GC deployment, it is possible not to be collected even 
> by old-gen GC(mixed GC)
> if entries sit in a region which doesn't have much garbages.
> Actually we used G1GC when we encountered this problem.
> We should remove the entry from ThreadLocalMap by calling ThreadLocal#remove 
> explicitly.



--
This message was sent by Atlassian JIRA
(v6.4.14#64029)


[jira] [Commented] (HBASE-26517) Add auth method information to AccessChecker audit log

2021-12-04 Thread Tomu Tsuruhara (Jira)


[ 
https://issues.apache.org/jira/browse/HBASE-26517?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=17453509#comment-17453509
 ] 

Tomu Tsuruhara commented on HBASE-26517:


Thank you for the review and merging.

> Add auth method information to AccessChecker audit log
> --
>
> Key: HBASE-26517
> URL: https://issues.apache.org/jira/browse/HBASE-26517
> Project: HBase
>  Issue Type: Improvement
>  Components: security
>Reporter: Tomu Tsuruhara
>Assignee: Tomu Tsuruhara
>Priority: Trivial
> Fix For: 2.5.0, 3.0.0-alpha-2, 2.4.9
>
>
> If we turn on audit logging, authentication events are logged with auth 
> method information  (e.g. KERBEROS, TOKEN etc) like below.
> {noformat}
> 2021-11-30 14:15:07,417 INFO SecurityLogger.org.apache.hadoop.hbase.Server: 
> Auth successful for PRINCIPAL@REALM (auth:KERBEROS)
> {noformat}
> However, authorization event logs which are emitted by AccessChecker class 
> don't contain auth method information.
> I need this info to filter out audit log lines generated by MapReduce jobs, 
> auth method = TOKEN case.



--
This message was sent by Atlassian Jira
(v8.20.1#820001)


[jira] [Assigned] (HBASE-26517) Add auth method information to AccessChecker audit log

2021-11-29 Thread Tomu Tsuruhara (Jira)


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

Tomu Tsuruhara reassigned HBASE-26517:
--

Assignee: Tomu Tsuruhara

> Add auth method information to AccessChecker audit log
> --
>
> Key: HBASE-26517
> URL: https://issues.apache.org/jira/browse/HBASE-26517
> Project: HBase
>  Issue Type: Improvement
>  Components: security
>Reporter: Tomu Tsuruhara
>Assignee: Tomu Tsuruhara
>Priority: Trivial
>
> If we turn on audit logging, authentication events are logged with auth 
> method information  (e.g. KERBEROS, TOKEN etc) like below.
> {noformat}
> 2021-11-30 14:15:07,417 INFO SecurityLogger.org.apache.hadoop.hbase.Server: 
> Auth successful for PRINCIPAL@REALM (auth:KERBEROS)
> {noformat}
> However, authorization event logs which are emitted by AccessChecker class 
> don't contain auth method information.
> I need this info to filter out audit log lines generated by MapReduce jobs, 
> auth method = TOKEN case.



--
This message was sent by Atlassian Jira
(v8.20.1#820001)


[jira] [Created] (HBASE-26517) Add auth method information to AccessChecker audit log

2021-11-29 Thread Tomu Tsuruhara (Jira)
Tomu Tsuruhara created HBASE-26517:
--

 Summary: Add auth method information to AccessChecker audit log
 Key: HBASE-26517
 URL: https://issues.apache.org/jira/browse/HBASE-26517
 Project: HBase
  Issue Type: Improvement
  Components: security
Reporter: Tomu Tsuruhara


If we turn on audit logging, authentication events are logged with auth method 
information  (e.g. KERBEROS, TOKEN etc) like below.

{noformat}
2021-11-30 14:15:07,417 INFO SecurityLogger.org.apache.hadoop.hbase.Server: 
Auth successful for PRINCIPAL@REALM (auth:KERBEROS)
{noformat}

However, authorization event logs which are emitted by AccessChecker class 
don't contain auth method information.

I need this info to filter out audit log lines generated by MapReduce jobs, 
auth method = TOKEN case.




--
This message was sent by Atlassian Jira
(v8.20.1#820001)