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

binlijin commented on HBASE-19338:
----------------------------------

V3 is ok with me.

YCSB : 
1 regionserver.
4 client machine, every machine run 10 ycsb process and every yscb use 100 
threads.

Regionserver config is :

"-Xmx36864m -Xms36864m -Xmn10240m -XX:SurvivorRatio=3 
-XX:MaxTenuringThreshold=15 -Xss256k -XX:+UseParNewGC 
-XX:+UseCMSInitiatingOccupancyOnly -XX:CMSInitiatingOccupancyFraction=63  
-XX:ParallelGCThreads=16"
 
   <property>
      <name>dfs.client.block.write.replace-datanode-on-failure.enable</name>
      <value>true</value>
    </property>

    <property>
      <name>dfs.client.read.shortcircuit</name>
      <value>true</value>
    </property>

    <property>
      <name>dfs.client.socket-timeout</name>
      <value>30000</value>
    </property>

    <property>
      <name>dfs.datanode.socket.write.timeout</name>
      <value>30000</value>
    </property>

    <property>
      <name>dfs.domain.socket.path</name>
      <value>/var/run/hadoop-hdfs/dn._PORT</value>
    </property>

    <property>
      <name>hbase.balancer.period</name>
      <value>3600000</value>
    </property>

    <property>
      <name>hbase.block.data.cachecompressed</name>
      <value>false</value>
    </property>

    <property>
      <name>hbase.bucketcache.combinedcache.enabled</name>
      <value>true</value>
    </property>

    <property>
      <name>hbase.bucketcache.ioengine</name>
      <value>offheap</value>
    </property>

    <property>
      <name>hbase.bucketcache.size</name>
      <value>12288</value>
    </property>

    <property>
      <name>hbase.bucketcache.writer.queuelength</name>
      <value>64</value>
    </property>

    <property>
      <name>hbase.bucketcache.writer.threads</name>
      <value>3</value>
    </property>

    <property>
      <name>hbase.bulk.assignment.threadpool.size</name>
      <value>100</value>
    </property>

    <property>
      <name>hbase.bulkload.wait.duration</name>
      <value>10000</value>
    </property>

    <property>
      <name>hbase.client.keyvalue.maxsize</name>
      <value>41943040</value>
    </property>

    <property>
      <name>hbase.client.log.batcherrors.details</name>
      <value>true</value>
    </property>

    <property>
      <name>hbase.client.operation.timeout</name>
      <value>180000</value>
    </property>

    <property>
      <name>hbase.client.pause</name>
      <value>1000</value>
    </property>

    <property>
      <name>hbase.client.retries.number</name>
      <value>34</value>
    </property>

    <property>
      <name>hbase.client.rpc.codec</name>
      <value>org.apache.hadoop.hbase.codec.KeyValueCodecWithTags</value>
    </property>

    <property>
      <name>hbase.client.scanner.caching</name>
      <value>8</value>
    </property>

    <property>
      <name>hbase.client.scanner.max.result.size</name>
      <value>2097152</value>
    </property>

    <property>
      <name>hbase.client.scanner.timeout.period</name>
      <value>180000</value>
    </property>

    <property>
      <name>hbase.client.write.autoflush</name>
      <value>true</value>
    </property>

    <property>
      <name>hbase.client.write.buffer</name>
      <value>2097152</value>
    </property>

    <property>
      <name>hbase.cluster.distributed</name>
      <value>true</value>
    </property>

    <property>
      <name>hbase.coprocessor.abortonerror</name>
      <value>false</value>
    </property>

    <property>
      <name>hbase.defaults.for.version.skip</name>
      <value>true</value>
    </property>

    <property>
      <name>hbase.dfs.client.read.shortcircuit.buffer.size</name>
      <value>16384</value>
    </property>

    <property>
      <name>hbase.hlog.split.skip.errors</name>
      <value>false</value>
    </property>

    <property>
      <name>hbase.hregion.majorcompaction</name>
      <value>604800000</value>
    </property>

    <property>
      <name>hbase.hregion.max.filesize</name>
      <value>8589934592</value>
    </property>

    <property>
      <name>hbase.hregion.memstore.block.multiplier</name>
      <value>2</value>
    </property>

    <property>
      <name>hbase.hregion.memstore.chunkpool.initialsize</name>
      <value>1</value>
    </property>

    <property>
      <name>hbase.hregion.memstore.flush.size</name>
      <value>268435456</value>
    </property>

    <property>
      <name>hbase.hregion.memstore.mslab.enabled</name>
      <value>true</value>
    </property>

    <property>
      <name>hbase.hstore.block.storage.policy</name>
      <value>ALL_SSD</value>
    </property>

    <property>
      <name>hbase.hstore.blockingStoreFiles</name>
      <value>20</value>
    </property>

    <property>
      <name>hbase.hstore.blockingWaitTime</name>
      <value>90000</value>
    </property>

    <property>
      <name>hbase.hstore.compaction.kv.max</name>
      <value>128</value>
    </property>

    <property>
      <name>hbase.hstore.compaction.max</name>
      <value>10</value>
    </property>

    <property>
      <name>hbase.hstore.compaction.min</name>
      <value>4</value>
    </property>

    <property>
      <name>hbase.hstore.compaction.throughput.higher.bound</name>
      <value>167772160</value>
    </property>

    <property>
      <name>hbase.hstore.compaction.throughput.lower.bound</name>
      <value>167772160</value>
    </property>

    <property>
      <name>hbase.hstore.compaction.throughput.offpeak</name>
      <value>167772160</value>
    </property>

    <property>
      <name>hbase.hstore.compactionThreshold</name>
      <value>3</value>
    </property>

    <property>
      <name>hbase.hstore.flush.retries.number</name>
      <value>120</value>
    </property>

    <property>
      <name>hbase.hstore.flusher.count</name>
      <value>2</value>
    </property>

    <property>
      <name>hbase.hstore.open.and.close.threads.max</name>
      <value>10</value>
    </property>

    <property>
      <name>hbase.ipc.client.connection.maxidletime</name>
      <value>60000</value>
    </property>

    <property>
      <name>hbase.ipc.server.callqueue.handler.factor</name>
      <value>0.1</value>
    </property>

    <property>
      <name>hbase.ipc.server.callqueue.read.share</name>
      <value>0</value>
    </property>

    <property>
      <name>hbase.lease.recovery.dfs.timeout</name>
      <value>23000</value>
    </property>

    <property>
      <name>hbase.local.dir</name>
      <value>${hbase.tmp.dir}/local</value>
    </property>

    <property>
      <name>hbase.master.assignment.timeoutmonitor.timeout</name>
      <value>900000</value>
    </property>

    <property>
      <name>hbase.master.distributed.log.replay</name>
      <value>false</value>
    </property>

    <property>
      <name>hbase.master.executor.serverops.threads</name>
      <value>30</value>
    </property>

    <property>
      <name>hbase.master.info.bindAddress</name>
      <value>0.0.0.0</value>
    </property>

    <property>
      <name>hbase.master.info.port</name>
      <value>60010</value>
    </property>

    <property>
      <name>hbase.master.loadbalancer.class</name>
      <value>org.apache.hadoop.hbase.master.balancer.SimpleLoadBalancer</value>
    </property>

    <property>
      <name>hbase.master.port</name>
      <value>60100</value>
    </property>

    <property>
      <name>hbase.master.ui.readonly</name>
      <value>true</value>
    </property>

    <property>
      <name>hbase.offheapcache.minblocksize</name>
      <value>16384</value>
    </property>

    <property>
      <name>hbase.regions.slop</name>
      <value>0.1</value>
    </property>

    <property>
      <name>hbase.regionserver.compaction.private.readers</name>
      <value>true</value>
    </property>

    <property>
      <name>hbase.regionserver.executor.closeregion.threads</name>
      <value>10</value>
    </property>

    <property>
      <name>hbase.regionserver.executor.openregion.threads</name>
      <value>10</value>
    </property>

    <property>
      <name>hbase.regionserver.global.memstore.lowerLimit</name>
      <value>0.35</value>
    </property>

    <property>
      <name>hbase.regionserver.global.memstore.size</name>
      <value>0.5</value>
    </property>

    <property>
      <name>hbase.regionserver.global.memstore.size.lower.limit</name>
      <value>0.83</value>
    </property>

    <property>
      <name>hbase.regionserver.global.memstore.upperLimit</name>
      <value>0.4</value>
    </property>

    <property>
      <name>hbase.regionserver.hlog.blocksize</name>
      <value>268435456</value>
    </property>

    <property>
      <name>hbase.regionserver.hlog.splitlog.writer.threads</name>
      <value>8</value>
    </property>

    <property>
      <name>hbase.regionserver.hlog.tolerable.lowreplication</name>
      <value>2</value>
    </property>

    <property>
      <name>hbase.regionserver.info.port</name>
      <value>60030</value>
    </property>

    <property>
      <name>hbase.regionserver.info.port.auto</name>
      <value>true</value>
    </property>

    <property>
      <name>hbase.regionserver.logroll.errors.tolerated</name>
      <value>3</value>
    </property>

    <property>
      <name>hbase.regionserver.maxlogs</name>
      <value>32</value>
    </property>

    <property>
      <name>hbase.regionserver.metahandler.count</name>
      <value>90</value>
    </property>

    <property>
      <name>hbase.regionserver.msginterval</name>
      <value>10000</value>
    </property>

    <property>
      <name>hbase.regionserver.optionalcacheflushinterval</name>
      <value>3600000</value>
    </property>

    <property>
      <name>hbase.regionserver.thread.compaction.large</name>
      <value>2</value>
    </property>

    <property>
      <name>hbase.regionserver.thread.compaction.small</name>
      <value>2</value>
    </property>

    <property>
      <name>hbase.regionserver.thread.compaction.throttle</name>
      <value>1073741824</value>
    </property>

    <property>
      <name>hbase.regionserver.thread.hfilecleaner.throttle</name>
      <value>20971520</value>
    </property>

    <property>
      <name>hbase.regionserver.throughput.controller</name>
      
<value>org.apache.hadoop.hbase.regionserver.compactions.PressureAwareCompactionThroughputController</value>
    </property>

    <property>
      <name>hbase.regionserver.wal.enablecompression</name>
      <value>true</value>
    </property>

    <property>
      <name>hbase.replication</name>
      <value>false</value>
    </property>

    <property>
      <name>hbase.rpc.client.impl</name>
      <value>org.apache.hadoop.hbase.ipc.RpcClientImpl</value>
    </property>

    <property>
      <name>hbase.rpc.protection</name>
      <value>authentication</value>
    </property>

    <property>
      <name>hbase.rpc.timeout</name>
      <value>180000</value>
    </property>

    <property>
      <name>hbase.security.authentication</name>
      <value>simple</value>
    </property>

    <property>
      <name>hbase.security.authorization</name>
      <value>false</value>
    </property>

    <property>
      <name>hbase.superuser</name>
      <value>hadoop</value>
    </property>

    <property>
      <name>hbase.tmp.dir</name>
      <value>/home/${user.name}/hbase-data</value>
    </property>

    <property>
      <name>hbase.wal.grouping.strategy</name>
      <value>boundedRegionGroup</value>
    </property>

    <property>
      <name>hbase.wal.provider</name>
      <value>multiwal</value>
    </property>

    <property>
      <name>hbase.wal.regiongrouping.numgroups</name>
      <value>1</value>
    </property>

    <property>
      <name>hbase.wal.storage.policy</name>
      <value>ALL_SSD</value>
    </property>

    <property>
      <name>hbase.zookeeper.property.clientPort</name>
      <value>2181</value>
    </property>

    <property>
      <name>hbase.zookeeper.property.maxClientCnxns</name>
      <value>10240</value>
    </property>

    <property>
      <name>hbase.zookeeper.useMulti</name>
      <value>true</value>
    </property>

    <property>
      <name>hfile.block.bloom.cacheonwrite</name>
      <value>true</value>
    </property>

    <property>
      <name>hfile.block.cache.size</name>
      <value>0.08</value>
    </property>

    <property>
      <name>hfile.block.index.cacheonwrite</name>
      <value>true</value>
    </property>

    <property>
      <name>hfile.format.version</name>
      <value>3</value>
    </property>

    <property>
      <name>ipc.client.connect.max.retries.on.timeouts</name>
      <value>3</value>
    </property>

    <property>
      <name>ipc.client.connect.timeout</name>
      <value>3000</value>
    </property>

    <property>
      <name>zookeeper.session.timeout</name>
      <value>180000</value>
    </property>

   <property>
      <name>hbase.regionserver.regionSplitLimit</name>
      <value>0</value>
    </property>
    
   <property>
      <name>hbase.regionserver.hlog.syncer.count</name>
      <value>5</value>
    </property>

    <property>
      <name>hbase.hstore.flusher.count</name>
      <value>2</value>
    </property>

   <property>
      <name>hbase.ipc.server.callqueue.type</name>
      <value>fifo</value>
    </property>

    <property>
      <name>hbase.regionserver.handler.count</name>
      <value>192</value>
    </property>
   
    <property>
       <name>hbase.hregion.compacting.memstore.type</name>
       <value>NONE</value>
    </property>


> Performance regression in RegionServerRpcQuotaManager to get ugi 
> -----------------------------------------------------------------
>
>                 Key: HBASE-19338
>                 URL: https://issues.apache.org/jira/browse/HBASE-19338
>             Project: HBase
>          Issue Type: Improvement
>    Affects Versions: 3.0.0, 2.0.0-beta-2
>            Reporter: binlijin
>            Assignee: binlijin
>            Priority: Critical
>         Attachments: 19338.master.003.patch, HBASE-19338.master.001.patch, 
> HBASE-19338.master.002.patch
>
>
> we find hbase-2.0.0-beta-1.SNAPSHOT have performance regression with yscb put 
>  and have some finding.  
> {code}
> "RpcServer.default.FPBQ.Fifo.handler=131,queue=17,port=16020" #245 daemon 
> prio=5 os_prio=0 tid=0x00007fc82b22e000 nid=0x3a5db waiting for monitor entry 
> [0x00007fc50fafa000]
>    java.lang.Thread.State: BLOCKED (on object monitor)
>         at 
> org.apache.hadoop.security.UserGroupInformation.getCurrentUser(UserGroupInformation.java:647)
>         - waiting to lock <0x00007fcaedc20830> (a java.lang.Class for 
> org.apache.hadoop.security.UserGroupInformation)
>         at 
> org.apache.hadoop.hbase.security.User$SecureHadoopUser.<init>(User.java:264)
>         at org.apache.hadoop.hbase.security.User.getCurrent(User.java:162)
>         at 
> org.apache.hadoop.hbase.quotas.RegionServerRpcQuotaManager.checkQuota(RegionServerRpcQuotaManager.java:179)
>         at 
> org.apache.hadoop.hbase.quotas.RegionServerRpcQuotaManager.checkQuota(RegionServerRpcQuotaManager.java:162)
>         at 
> org.apache.hadoop.hbase.regionserver.RSRpcServices.multi(RSRpcServices.java:2521)
>         at 
> org.apache.hadoop.hbase.shaded.protobuf.generated.ClientProtos$ClientService$2.callBlockingMethod(ClientProtos.java:41560)
>         at org.apache.hadoop.hbase.ipc.RpcServer.call(RpcServer.java:406)
>         at org.apache.hadoop.hbase.ipc.CallRunner.run(CallRunner.java:130)
>         at 
> org.apache.hadoop.hbase.ipc.RpcExecutor$Handler.run(RpcExecutor.java:325)
>         at 
> org.apache.hadoop.hbase.ipc.RpcExecutor$Handler.run(RpcExecutor.java:305)
> {code}



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

Reply via email to