Use experience and performance data of offheap from Alibaba online cluster

2016-11-18 Thread Yu Li
Dear all,

We have backported read path offheap (HBASE-11425) to our customized
hbase-1.1.2 (thanks @Anoop for the help/support) and run it online for more
than a month, and would like to share our experience, for what it's worth
(smile).

Generally speaking, we gained a better and more stable
throughput/performance with offheap, and below are some details:

1. QPS become more stable with offheap

Performance w/o offheap:

Performance w/ offheap:

These data come from our online A/B test cluster (with 450 physical
machines, and each with 256G memory + 64 core) with real world workloads,
it shows using offheap we could gain a more stable throughput as well as
better performance

Not showing fully online data here because for online we published the
version with both offheap and NettyRpcServer together, so no standalone
comparison data for offheap

2. Full GC frequency and cost

Average Full GC STW time reduce from 11s to 7s with offheap.

3. Young GC frequency and cost

No performance degradation observed with offheap.

4. Peak throughput of one single RS

On Singles Day (11/11), peak throughput of one single RS reached 100K,
among which 90K from Get. Plus internet in/out data we could know the
average result size of get request is ~1KB

Offheap are used on all online machines (more than 1600 nodes) instead of
LruCache, so the above QPS is gained from offheap bucketcache, along with
NettyRpcServer(HBASE-15756).
Just let us know if any comments. Thanks.

Best Regards,
Yu


RE: Use experience and performance data of offheap from Alibaba online cluster

2016-11-18 Thread Du, Jingcheng
Thanks Yu for the sharing, great achievements.
It seems the images cannot be displayed? Maybe just me?

Regards,
Jingcheng

From: Yu Li [mailto:car...@gmail.com]
Sent: Friday, November 18, 2016 4:11 PM
To: user@hbase.apache.org; d...@hbase.apache.org
Subject: Use experience and performance data of offheap from Alibaba online 
cluster

Dear all,

We have backported read path offheap (HBASE-11425) to our customized 
hbase-1.1.2 (thanks @Anoop for the help/support) and run it online for more 
than a month, and would like to share our experience, for what it's worth 
(smile).

Generally speaking, we gained a better and more stable throughput/performance 
with offheap, and below are some details:

1. QPS become more stable with offheap

Performance w/o offheap:

[cid:part1.582d4b6424f071c]

Performance w/ offheap:

[cid:part2.582d4b6424f071c]

These data come from our online A/B test cluster (with 450 physical machines, 
and each with 256G memory + 64 core) with real world workloads, it shows using 
offheap we could gain a more stable throughput as well as better performance

Not showing fully online data here because for online we published the version 
with both offheap and NettyRpcServer together, so no standalone comparison data 
for offheap

2. Full GC frequency and cost

Average Full GC STW time reduce from 11s to 7s with offheap.

3. Young GC frequency and cost

No performance degradation observed with offheap.

4. Peak throughput of one single RS

On Singles Day (11/11), peak throughput of one single RS reached 100K, among 
which 90K from Get. Plus internet in/out data we could know the average result 
size of get request is ~1KB

[cid:part3.582d4b6424f071c]

Offheap are used on all online machines (more than 1600 nodes) instead of 
LruCache, so the above QPS is gained from offheap bucketcache, along with 
NettyRpcServer(HBASE-15756).
Just let us know if any comments. Thanks.

Best Regards,
Yu








Re: Use experience and performance data of offheap from Alibaba online cluster

2016-11-18 Thread Loïc Chanel
Nope, same here !

Loïc CHANEL
System Big Data engineer
MS&T - WASABI - Worldline (Villeurbanne, France)

2016-11-18 9:54 GMT+01:00 Du, Jingcheng :

> Thanks Yu for the sharing, great achievements.
> It seems the images cannot be displayed? Maybe just me?
>
> Regards,
> Jingcheng
>
> From: Yu Li [mailto:car...@gmail.com]
> Sent: Friday, November 18, 2016 4:11 PM
> To: user@hbase.apache.org; d...@hbase.apache.org
> Subject: Use experience and performance data of offheap from Alibaba
> online cluster
>
> Dear all,
>
> We have backported read path offheap (HBASE-11425) to our customized
> hbase-1.1.2 (thanks @Anoop for the help/support) and run it online for more
> than a month, and would like to share our experience, for what it's worth
> (smile).
>
> Generally speaking, we gained a better and more stable
> throughput/performance with offheap, and below are some details:
>
> 1. QPS become more stable with offheap
>
> Performance w/o offheap:
>
> [cid:part1.582d4b6424f071c]
>
> Performance w/ offheap:
>
> [cid:part2.582d4b6424f071c]
>
> These data come from our online A/B test cluster (with 450 physical
> machines, and each with 256G memory + 64 core) with real world workloads,
> it shows using offheap we could gain a more stable throughput as well as
> better performance
>
> Not showing fully online data here because for online we published the
> version with both offheap and NettyRpcServer together, so no standalone
> comparison data for offheap
>
> 2. Full GC frequency and cost
>
> Average Full GC STW time reduce from 11s to 7s with offheap.
>
> 3. Young GC frequency and cost
>
> No performance degradation observed with offheap.
>
> 4. Peak throughput of one single RS
>
> On Singles Day (11/11), peak throughput of one single RS reached 100K,
> among which 90K from Get. Plus internet in/out data we could know the
> average result size of get request is ~1KB
>
> [cid:part3.582d4b6424f071c]
>
> Offheap are used on all online machines (more than 1600 nodes) instead of
> LruCache, so the above QPS is gained from offheap bucketcache, along with
> NettyRpcServer(HBASE-15756).
> Just let us know if any comments. Thanks.
>
> Best Regards,
> Yu
>
>
>
>
>
>
>


Re: Use experience and performance data of offheap from Alibaba online cluster

2016-11-18 Thread 张铎
I can not see the images either...

Du, Jingcheng 于2016年11月18日 周五16:57写道:

> Thanks Yu for the sharing, great achievements.
> It seems the images cannot be displayed? Maybe just me?
>
> Regards,
> Jingcheng
>
> From: Yu Li [mailto:car...@gmail.com]
> Sent: Friday, November 18, 2016 4:11 PM
> To: user@hbase.apache.org; d...@hbase.apache.org
> Subject: Use experience and performance data of offheap from Alibaba
> online cluster
>
> Dear all,
>
> We have backported read path offheap (HBASE-11425) to our customized
> hbase-1.1.2 (thanks @Anoop for the help/support) and run it online for more
> than a month, and would like to share our experience, for what it's worth
> (smile).
>
> Generally speaking, we gained a better and more stable
> throughput/performance with offheap, and below are some details:
>
> 1. QPS become more stable with offheap
>
> Performance w/o offheap:
>
> [cid:part1.582d4b6424f071c]
>
> Performance w/ offheap:
>
> [cid:part2.582d4b6424f071c]
>
> These data come from our online A/B test cluster (with 450 physical
> machines, and each with 256G memory + 64 core) with real world workloads,
> it shows using offheap we could gain a more stable throughput as well as
> better performance
>
> Not showing fully online data here because for online we published the
> version with both offheap and NettyRpcServer together, so no standalone
> comparison data for offheap
>
> 2. Full GC frequency and cost
>
> Average Full GC STW time reduce from 11s to 7s with offheap.
>
> 3. Young GC frequency and cost
>
> No performance degradation observed with offheap.
>
> 4. Peak throughput of one single RS
>
> On Singles Day (11/11), peak throughput of one single RS reached 100K,
> among which 90K from Get. Plus internet in/out data we could know the
> average result size of get request is ~1KB
>
> [cid:part3.582d4b6424f071c]
>
> Offheap are used on all online machines (more than 1600 nodes) instead of
> LruCache, so the above QPS is gained from offheap bucketcache, along with
> NettyRpcServer(HBASE-15756).
> Just let us know if any comments. Thanks.
>
> Best Regards,
> Yu
>
>
>
>
>
>
>


Re: Use experience and performance data of offheap from Alibaba online cluster

2016-11-18 Thread Ted Yu
Yu:
With positive results, more hbase users would be asking for the backport of 
offheap read path patches. 

Do you think you or your coworker has the bandwidth to publish backport for 
branch-1 ?

Thanks 

> On Nov 18, 2016, at 12:11 AM, Yu Li  wrote:
> 
> Dear all,
> 
> We have backported read path offheap (HBASE-11425) to our customized 
> hbase-1.1.2 (thanks @Anoop for the help/support) and run it online for more 
> than a month, and would like to share our experience, for what it's worth 
> (smile).
> 
> Generally speaking, we gained a better and more stable throughput/performance 
> with offheap, and below are some details:
> 1. QPS become more stable with offheap
> 
> Performance w/o offheap:
> 
> 
> 
> Performance w/ offheap:
> 
> 
> 
> These data come from our online A/B test cluster (with 450 physical machines, 
> and each with 256G memory + 64 core) with real world workloads, it shows 
> using offheap we could gain a more stable throughput as well as better 
> performance
> 
> Not showing fully online data here because for online we published the 
> version with both offheap and NettyRpcServer together, so no standalone 
> comparison data for offheap
> 
> 2. Full GC frequency and cost
> 
> Average Full GC STW time reduce from 11s to 7s with offheap.
> 
> 3. Young GC frequency and cost
> 
> No performance degradation observed with offheap.
> 
> 4. Peak throughput of one single RS
> 
> On Singles Day (11/11), peak throughput of one single RS reached 100K, among 
> which 90K from Get. Plus internet in/out data we could know the average 
> result size of get request is ~1KB
> 
> 
> 
> Offheap are used on all online machines (more than 1600 nodes) instead of 
> LruCache, so the above QPS is gained from offheap bucketcache, along with 
> NettyRpcServer(HBASE-15756).
> 
> Just let us know if any comments. Thanks.
> 
> Best Regards,
> Yu
> 
> 
> 
> 
> 
> 
> 


hbase/spark - Delegation Token can be issued only with kerberos or web authentication

2016-11-18 Thread Abel Fernández
Hello,

We are having problems with the delegation of the token in a secure
cluster: Delegation Token can be issued only with kerberos or web
authentication

We have a spark process which is generating the hfiles to be loaded into
hbase. To generate these hfiles, (we are using a back-ported version of the
latest hbase/spark code), we are using this method
HBaseRDDFunctions.hbaseBulkLoadThinRows.

I think the problem is in the below piece of code. This function is
executed in every partition of the rdd, when the executors are trying to
execute the code, the executors do not have a valid kerberos credential and
cannot execute anything.

private def hbaseForeachPartition[T](configBroadcast:

 Broadcast[SerializableWritable[Configuration]],
it: Iterator[T],
f: (Iterator[T], Connection) =>
Unit) = {

val config = getConf(configBroadcast)

applyCreds
// specify that this is a proxy user
val smartConn = HBaseConnectionCache.getConnection(config)
f(it, smartConn.connection)
smartConn.close()
  }

I have attached the spark-submit and the complete error log trace. Has
anyone faced this problem before?

Thanks in advance.

Regards,
Abel.
-- 
Un saludo - Best Regards.
Abel
Script for submitting the spark action
#!/bin/bash

SPARK_CONF_DIR=conf-hbase spark-submit --master yarn-cluster \
  --executor-memory 6G \
  --num-executors 10 \
  --queue cards \
  --executor-cores 4 \
  --driver-java-options "-Dlog4j.configuration=file:log4j.properties" \
  --driver-class-path "$2" \
  --jars file:/opt/orange/lib/rocksdbjni-4.5.1.jar \
  --conf 
"spark.driver.extraClassPath=/var/cloudera/parcels/CDH/lib/hbase/lib/htrace-core-3.2.0-incubating.jar:/var/cloudera/parcels/CDH/jars/hbase-server-1.0.0-cdh5.5.4.jar:/var/cloudera/parcels/CDH/jars/hbase-common-1.0.0-cdh5.5.4.jar:/var/cloudera/parcels/CDH/lib/hbase/lib/hbase-client-1.0.0-cdh5.5.4.jar:/var/cloudera/parcels/CDH/lib/hbase/lib/hbase-protocol-1.0.0-cdh5.5.4.jar:/opt/orange/lib/rocksdbjni-4.5.1.jar:/var/cloudera/parcels/CLABS_PHOENIX-4.5.2-1.clabs_phoenix1.2.0.p0.774/lib/phoenix/lib/phoenix-core-1.2.0.jar:/var/cloudera/parcels/CDH/jars/hadoop-mapreduce-client-core-2.6.0-cdh5.5.4.jar"
 \
  --conf 
"spark.executor.extraClassPath=/var/cloudera/parcels/CDH/lib/hbase/lib/htrace-core-3.2.0-incubating.jar:/var/cloudera/parcels/CDH/jars/hbase-server-1.0.0-cdh5.5.4.jar:/var/cloudera/parcels/CDH/jars/hbase-common-1.0.0-cdh5.5.4.jar:/var/cloudera/parcels/CDH/lib/hbase/lib/hbase-client-1.0.0-cdh5.5.4.jar:/var/cloudera/parcels/CDH/lib/hbase/lib/hbase-protocol-1.0.0-cdh5.5.4.jar:/opt/orange/lib/rocksdbjni-4.5.1.jar:/var/cloudera/parcels/CLABS_PHOENIX-4.5.2-1.clabs_phoenix1.2.0.p0.774/lib/phoenix/lib/phoenix-core-1.2.0.jar:/var/cloudera/parcels/CDH/jars/hadoop-mapreduce-client-core-2.6.0-cdh5.5.4.jar"\
  --principal hb...@company.corp \
  --keytab /opt/company/conf/hbase.keytab \
  --files 
"owl.properties,conf-hbase/log4j.properties,conf-hbase/hbase-site.xml,conf-hbase/core-site.xml,$2"
 \
  --class $1 \
  cards-batch-$3-jar-with-dependencies.jar $2Complete log
16/11/18 10:42:34 DEBUG [Driver] ClosureCleaner: +++ Cleaning closure 
 
(org.apache.hadoop.hbase.spark.HBaseContext$$anonfun$bulkLoadThinRows$1) +++
16/11/18 10:42:34 DEBUG [Driver] ClosureCleaner:  + declared fields: 2
16/11/18 10:42:34 DEBUG [Driver] ClosureCleaner:  public static final long 
org.apache.hadoop.hbase.spark.HBaseContext$$anonfun$bulkLoadThinRows$1.serialVersionUID
16/11/18 10:42:34 DEBUG [Driver] ClosureCleaner:  private final 
scala.Function1 
org.apache.hadoop.hbase.spark.HBaseContext$$anonfun$bulkLoadThinRows$1.mapFunction$1
16/11/18 10:42:34 DEBUG [Driver] ClosureCleaner:  + declared methods: 2
16/11/18 10:42:34 DEBUG [Driver] ClosureCleaner:  public final 
java.lang.Object 
org.apache.hadoop.hbase.spark.HBaseContext$$anonfun$bulkLoadThinRows$1.apply(java.lang.Object)
16/11/18 10:42:34 DEBUG [Driver] ClosureCleaner:  public final scala.Tuple2 
org.apache.hadoop.hbase.spark.HBaseContext$$anonfun$bulkLoadThinRows$1.apply(java.lang.Object)
16/11/18 10:42:34 DEBUG [Driver] ClosureCleaner:  + inner classes: 0
16/11/18 10:42:34 DEBUG [Driver] ClosureCleaner:  + outer classes: 0
16/11/18 10:42:34 DEBUG [Driver] ClosureCleaner:  + outer objects: 0
16/11/18 10:42:34 DEBUG [Driver] ClosureCleaner:  + populating accessed fields 
because this is the starting closure
16/11/18 10:42:34 DEBUG [Driver] ClosureCleaner:  + fields accessed by starting 
closure: 0
16/11/18 10:42:34 DEBUG [Driver] ClosureCleaner:  + there are no enclosing 
objects!
16/11/18 10:42:34 DEBUG [Driver] ClosureCleaner:  +++ closure  
(org.apache.hadoop.hbase.spark.HBaseContext$$anonfun$bulkLoadThinRows$1) is now 
cleaned +++
16/11/18 10:42:34 DEBUG [Driver] ClosureCleaner: +++ Cleaning closure 
 
(org.apache.hadoop.hbase.spark.HBaseContext$$anonfun$foreachPartition$1) +++
16/11/18 10:42:34 DEBUG [Driver] ClosureCleaner:  + declared fields: 

Re: Hbase full scan table data will be fast?

2016-11-18 Thread Jean-Marc Spaggiari
Hi,

I will all depends of the number of columns, the performance of your
servers, if you are doing the scan in parallel across all the regions at
the same time, or not, the processing you will do, etc. So it is not
possible to give you any estimate. You will have to test it to figure it.

JMS

2016-11-17 22:29 GMT-05:00 446463...@qq.com <446463...@qq.com>:

> Hi:
> I have a question in HBase
> How long will it take? scan a Hbase table which include millions of data
>
>
>
> 446463...@qq.com
>


Re: hbase/spark - Delegation Token can be issued only with kerberos or web authentication

2016-11-18 Thread Nkechi Achara
Can you use the principal and keytab options in Spark submit? These should
circumvent this issue.

On 18 Nov 2016 1:01 p.m., "Abel Fernández"  wrote:

> Hello,
>
> We are having problems with the delegation of the token in a secure
> cluster: Delegation Token can be issued only with kerberos or web
> authentication
>
> We have a spark process which is generating the hfiles to be loaded into
> hbase. To generate these hfiles, (we are using a back-ported version of the
> latest hbase/spark code), we are using this method HBaseRDDFunctions.
> hbaseBulkLoadThinRows.
>
> I think the problem is in the below piece of code. This function is
> executed in every partition of the rdd, when the executors are trying to
> execute the code, the executors do not have a valid kerberos credential and
> cannot execute anything.
>
> private def hbaseForeachPartition[T](configBroadcast:
>Broadcast[SerializableWritable[
> Configuration]],
> it: Iterator[T],
> f: (Iterator[T], Connection) =>
> Unit) = {
>
> val config = getConf(configBroadcast)
>
> applyCreds
> // specify that this is a proxy user
> val smartConn = HBaseConnectionCache.getConnection(config)
> f(it, smartConn.connection)
> smartConn.close()
>   }
>
> I have attached the spark-submit and the complete error log trace. Has
> anyone faced this problem before?
>
> Thanks in advance.
>
> Regards,
> Abel.
> --
> Un saludo - Best Regards.
> Abel
>


Re: hbase/spark - Delegation Token can be issued only with kerberos or web authentication

2016-11-18 Thread Abel Fernández
Hi Nkechi,

Thank for your early response.

I am currently specifying the principal and the keytab in the spark-submit,
the keytab is in the same location in every node manager.

SPARK_CONF_DIR=conf-hbase spark-submit --master yarn-cluster \
  --executor-memory 6G \
  --num-executors 10 \
  --queue cards \
  --executor-cores 4 \
  --driver-java-options "-Dlog4j.configuration=file:log4j.properties" \
  --driver-class-path "$2" \
  --jars file:/opt/orange/lib/rocksdbjni-4.5.1.jar \
  --conf
"spark.driver.extraClassPath=/var/cloudera/parcels/CDH/lib/hbase/lib/htrace-core-3.2.0-incubating.jar:/var/cloudera/parcels/CDH/jars/hbase-server-1.0.0-cdh5.5.4.jar:/var/cloudera/parcels/CDH/jars/hbase-common-1.0.0-cdh5.5.4.jar:/var/cloudera/parcels/CDH/lib/hbase/lib/hbase-client-1.0.0-cdh5.5.4.jar:/var/cloudera/parcels/CDH/lib/hbase/lib/hbase-protocol-1.0.0-cdh5.5.4.jar:/opt/orange/lib/rocksdbjni-4.5.1.jar:/var/cloudera/parcels/CLABS_PHOENIX-4.5.2-1.clabs_phoenix1.2.0.p0.774/lib/phoenix/lib/phoenix-core-1.2.0.jar:/var/cloudera/parcels/CDH/jars/hadoop-mapreduce-client-core-2.6.0-cdh5.5.4.jar"
\
  --conf
"spark.executor.extraClassPath=/var/cloudera/parcels/CDH/lib/hbase/lib/htrace-core-3.2.0-incubating.jar:/var/cloudera/parcels/CDH/jars/hbase-server-1.0.0-cdh5.5.4.jar:/var/cloudera/parcels/CDH/jars/hbase-common-1.0.0-cdh5.5.4.jar:/var/cloudera/parcels/CDH/lib/hbase/lib/hbase-client-1.0.0-cdh5.5.4.jar:/var/cloudera/parcels/CDH/lib/hbase/lib/hbase-protocol-1.0.0-cdh5.5.4.jar:/opt/orange/lib/rocksdbjni-4.5.1.jar:/var/cloudera/parcels/CLABS_PHOENIX-4.5.2-1.clabs_phoenix1.2.0.p0.774/lib/phoenix/lib/phoenix-core-1.2.0.jar:/var/cloudera/parcels/CDH/jars/hadoop-mapreduce-client-core-2.6.0-cdh5.5.4.jar"\
  --principal hb...@company.corp \
  --keytab /opt/company/conf/hbase.keytab \
  --files
"owl.properties,conf-hbase/log4j.properties,conf-hbase/hbase-site.xml,conf-hbase/core-site.xml,$2"
\
  --class $1 \
  cards-batch-$3-jar-with-dependencies.jar $2

On Fri, 18 Nov 2016 at 14:01 Nkechi Achara  wrote:

> Can you use the principal and keytab options in Spark submit? These should
> circumvent this issue.
>
> On 18 Nov 2016 1:01 p.m., "Abel Fernández"  wrote:
>
> > Hello,
> >
> > We are having problems with the delegation of the token in a secure
> > cluster: Delegation Token can be issued only with kerberos or web
> > authentication
> >
> > We have a spark process which is generating the hfiles to be loaded into
> > hbase. To generate these hfiles, (we are using a back-ported version of
> the
> > latest hbase/spark code), we are using this method HBaseRDDFunctions.
> > hbaseBulkLoadThinRows.
> >
> > I think the problem is in the below piece of code. This function is
> > executed in every partition of the rdd, when the executors are trying to
> > execute the code, the executors do not have a valid kerberos credential
> and
> > cannot execute anything.
> >
> > private def hbaseForeachPartition[T](configBroadcast:
> >Broadcast[SerializableWritable[
> > Configuration]],
> > it: Iterator[T],
> > f: (Iterator[T], Connection) =>
> > Unit) = {
> >
> > val config = getConf(configBroadcast)
> >
> > applyCreds
> > // specify that this is a proxy user
> > val smartConn = HBaseConnectionCache.getConnection(config)
> > f(it, smartConn.connection)
> > smartConn.close()
> >   }
> >
> > I have attached the spark-submit and the complete error log trace. Has
> > anyone faced this problem before?
> >
> > Thanks in advance.
> >
> > Regards,
> > Abel.
> > --
> > Un saludo - Best Regards.
> > Abel
> >
>
-- 
Un saludo - Best Regards.
Abel


Re: hbase/spark - Delegation Token can be issued only with kerberos or web authentication

2016-11-18 Thread Nkechi Achara
Sorry just realised you had the submit command in the attached docs.

Can I ask if the keytab is also on the driver in the same location?

The spark option normally requires the keytab to be on the driver so it can
pick it up and pass it to yarn etc to perform the kerberos operations.

On 18 Nov 2016 3:10 p.m., "Abel Fernández"  wrote:

> Hi Nkechi,
>
> Thank for your early response.
>
> I am currently specifying the principal and the keytab in the spark-submit,
> the keytab is in the same location in every node manager.
>
> SPARK_CONF_DIR=conf-hbase spark-submit --master yarn-cluster \
>   --executor-memory 6G \
>   --num-executors 10 \
>   --queue cards \
>   --executor-cores 4 \
>   --driver-java-options "-Dlog4j.configuration=file:log4j.properties" \
>   --driver-class-path "$2" \
>   --jars file:/opt/orange/lib/rocksdbjni-4.5.1.jar \
>   --conf
> "spark.driver.extraClassPath=/var/cloudera/parcels/CDH/lib/
> hbase/lib/htrace-core-3.2.0-incubating.jar:/var/cloudera/
> parcels/CDH/jars/hbase-server-1.0.0-cdh5.5.4.jar:/var/
> cloudera/parcels/CDH/jars/hbase-common-1.0.0-cdh5.5.4.
> jar:/var/cloudera/parcels/CDH/lib/hbase/lib/hbase-client-1.
> 0.0-cdh5.5.4.jar:/var/cloudera/parcels/CDH/lib/
> hbase/lib/hbase-protocol-1.0.0-cdh5.5.4.jar:/opt/orange/
> lib/rocksdbjni-4.5.1.jar:/var/cloudera/parcels/CLABS_
> PHOENIX-4.5.2-1.clabs_phoenix1.2.0.p0.774/lib/
> phoenix/lib/phoenix-core-1.2.0.jar:/var/cloudera/parcels/
> CDH/jars/hadoop-mapreduce-client-core-2.6.0-cdh5.5.4.jar"
> \
>   --conf
> "spark.executor.extraClassPath=/var/cloudera/parcels/CDH/lib/hbase/lib/
> htrace-core-3.2.0-incubating.jar:/var/cloudera/parcels/CDH/
> jars/hbase-server-1.0.0-cdh5.5.4.jar:/var/cloudera/parcels/
> CDH/jars/hbase-common-1.0.0-cdh5.5.4.jar:/var/cloudera/
> parcels/CDH/lib/hbase/lib/hbase-client-1.0.0-cdh5.5.4.
> jar:/var/cloudera/parcels/CDH/lib/hbase/lib/hbase-protocol-
> 1.0.0-cdh5.5.4.jar:/opt/orange/lib/rocksdbjni-4.5.1.
> jar:/var/cloudera/parcels/CLABS_PHOENIX-4.5.2-1.clabs_
> phoenix1.2.0.p0.774/lib/phoenix/lib/phoenix-core-1.2.
> 0.jar:/var/cloudera/parcels/CDH/jars/hadoop-mapreduce-
> client-core-2.6.0-cdh5.5.4.jar"\
>   --principal hb...@company.corp \
>   --keytab /opt/company/conf/hbase.keytab \
>   --files
> "owl.properties,conf-hbase/log4j.properties,conf-hbase/
> hbase-site.xml,conf-hbase/core-site.xml,$2"
> \
>   --class $1 \
>   cards-batch-$3-jar-with-dependencies.jar $2
>
> On Fri, 18 Nov 2016 at 14:01 Nkechi Achara 
> wrote:
>
> > Can you use the principal and keytab options in Spark submit? These
> should
> > circumvent this issue.
> >
> > On 18 Nov 2016 1:01 p.m., "Abel Fernández"  wrote:
> >
> > > Hello,
> > >
> > > We are having problems with the delegation of the token in a secure
> > > cluster: Delegation Token can be issued only with kerberos or web
> > > authentication
> > >
> > > We have a spark process which is generating the hfiles to be loaded
> into
> > > hbase. To generate these hfiles, (we are using a back-ported version of
> > the
> > > latest hbase/spark code), we are using this method HBaseRDDFunctions.
> > > hbaseBulkLoadThinRows.
> > >
> > > I think the problem is in the below piece of code. This function is
> > > executed in every partition of the rdd, when the executors are trying
> to
> > > execute the code, the executors do not have a valid kerberos credential
> > and
> > > cannot execute anything.
> > >
> > > private def hbaseForeachPartition[T](configBroadcast:
> > >Broadcast[SerializableWritable[
> > > Configuration]],
> > > it: Iterator[T],
> > > f: (Iterator[T], Connection) =>
> > > Unit) = {
> > >
> > > val config = getConf(configBroadcast)
> > >
> > > applyCreds
> > > // specify that this is a proxy user
> > > val smartConn = HBaseConnectionCache.getConnection(config)
> > > f(it, smartConn.connection)
> > > smartConn.close()
> > >   }
> > >
> > > I have attached the spark-submit and the complete error log trace. Has
> > > anyone faced this problem before?
> > >
> > > Thanks in advance.
> > >
> > > Regards,
> > > Abel.
> > > --
> > > Un saludo - Best Regards.
> > > Abel
> > >
> >
> --
> Un saludo - Best Regards.
> Abel
>


Re: hbase/spark - Delegation Token can be issued only with kerberos or web authentication

2016-11-18 Thread Abel Fernández
Yep, the keytab is also in the driver into the same location.

-rw-r--r-- 1 hbase root  370 Nov 16 17:13 hbase.keytab

Do you know what are the permissions that the keytab should have?



On Fri, 18 Nov 2016 at 14:19 Nkechi Achara  wrote:

> Sorry just realised you had the submit command in the attached docs.
>
> Can I ask if the keytab is also on the driver in the same location?
>
> The spark option normally requires the keytab to be on the driver so it can
> pick it up and pass it to yarn etc to perform the kerberos operations.
>
> On 18 Nov 2016 3:10 p.m., "Abel Fernández"  wrote:
>
> > Hi Nkechi,
> >
> > Thank for your early response.
> >
> > I am currently specifying the principal and the keytab in the
> spark-submit,
> > the keytab is in the same location in every node manager.
> >
> > SPARK_CONF_DIR=conf-hbase spark-submit --master yarn-cluster \
> >   --executor-memory 6G \
> >   --num-executors 10 \
> >   --queue cards \
> >   --executor-cores 4 \
> >   --driver-java-options "-Dlog4j.configuration=file:log4j.properties" \
> >   --driver-class-path "$2" \
> >   --jars file:/opt/orange/lib/rocksdbjni-4.5.1.jar \
> >   --conf
> > "spark.driver.extraClassPath=/var/cloudera/parcels/CDH/lib/
> > hbase/lib/htrace-core-3.2.0-incubating.jar:/var/cloudera/
> > parcels/CDH/jars/hbase-server-1.0.0-cdh5.5.4.jar:/var/
> > cloudera/parcels/CDH/jars/hbase-common-1.0.0-cdh5.5.4.
> > jar:/var/cloudera/parcels/CDH/lib/hbase/lib/hbase-client-1.
> > 0.0-cdh5.5.4.jar:/var/cloudera/parcels/CDH/lib/
> > hbase/lib/hbase-protocol-1.0.0-cdh5.5.4.jar:/opt/orange/
> > lib/rocksdbjni-4.5.1.jar:/var/cloudera/parcels/CLABS_
> > PHOENIX-4.5.2-1.clabs_phoenix1.2.0.p0.774/lib/
> > phoenix/lib/phoenix-core-1.2.0.jar:/var/cloudera/parcels/
> > CDH/jars/hadoop-mapreduce-client-core-2.6.0-cdh5.5.4.jar"
> > \
> >   --conf
> > "spark.executor.extraClassPath=/var/cloudera/parcels/CDH/lib/hbase/lib/
> > htrace-core-3.2.0-incubating.jar:/var/cloudera/parcels/CDH/
> > jars/hbase-server-1.0.0-cdh5.5.4.jar:/var/cloudera/parcels/
> > CDH/jars/hbase-common-1.0.0-cdh5.5.4.jar:/var/cloudera/
> > parcels/CDH/lib/hbase/lib/hbase-client-1.0.0-cdh5.5.4.
> > jar:/var/cloudera/parcels/CDH/lib/hbase/lib/hbase-protocol-
> > 1.0.0-cdh5.5.4.jar:/opt/orange/lib/rocksdbjni-4.5.1.
> > jar:/var/cloudera/parcels/CLABS_PHOENIX-4.5.2-1.clabs_
> > phoenix1.2.0.p0.774/lib/phoenix/lib/phoenix-core-1.2.
> > 0.jar:/var/cloudera/parcels/CDH/jars/hadoop-mapreduce-
> > client-core-2.6.0-cdh5.5.4.jar"\
> >   --principal hb...@company.corp \
> >   --keytab /opt/company/conf/hbase.keytab \
> >   --files
> > "owl.properties,conf-hbase/log4j.properties,conf-hbase/
> > hbase-site.xml,conf-hbase/core-site.xml,$2"
> > \
> >   --class $1 \
> >   cards-batch-$3-jar-with-dependencies.jar $2
> >
> > On Fri, 18 Nov 2016 at 14:01 Nkechi Achara 
> > wrote:
> >
> > > Can you use the principal and keytab options in Spark submit? These
> > should
> > > circumvent this issue.
> > >
> > > On 18 Nov 2016 1:01 p.m., "Abel Fernández" 
> wrote:
> > >
> > > > Hello,
> > > >
> > > > We are having problems with the delegation of the token in a secure
> > > > cluster: Delegation Token can be issued only with kerberos or web
> > > > authentication
> > > >
> > > > We have a spark process which is generating the hfiles to be loaded
> > into
> > > > hbase. To generate these hfiles, (we are using a back-ported version
> of
> > > the
> > > > latest hbase/spark code), we are using this method HBaseRDDFunctions.
> > > > hbaseBulkLoadThinRows.
> > > >
> > > > I think the problem is in the below piece of code. This function is
> > > > executed in every partition of the rdd, when the executors are trying
> > to
> > > > execute the code, the executors do not have a valid kerberos
> credential
> > > and
> > > > cannot execute anything.
> > > >
> > > > private def hbaseForeachPartition[T](configBroadcast:
> > > >
> Broadcast[SerializableWritable[
> > > > Configuration]],
> > > > it: Iterator[T],
> > > > f: (Iterator[T], Connection)
> =>
> > > > Unit) = {
> > > >
> > > > val config = getConf(configBroadcast)
> > > >
> > > > applyCreds
> > > > // specify that this is a proxy user
> > > > val smartConn = HBaseConnectionCache.getConnection(config)
> > > > f(it, smartConn.connection)
> > > > smartConn.close()
> > > >   }
> > > >
> > > > I have attached the spark-submit and the complete error log trace.
> Has
> > > > anyone faced this problem before?
> > > >
> > > > Thanks in advance.
> > > >
> > > > Regards,
> > > > Abel.
> > > > --
> > > > Un saludo - Best Regards.
> > > > Abel
> > > >
> > >
> > --
> > Un saludo - Best Regards.
> > Abel
> >
>
-- 
Un saludo - Best Regards.
Abel


write performance regression in CDH 5.9?

2016-11-18 Thread Graham Baecher
Hi all,

We're looking to update the HBase version we're running on our servers, but
noticed while comparing performance test results a the previous version
that Puts now seem to be slower.
Specifically, comparing HBase 1.2.0-cdh5.9.0 to 1.2.0-cdh5.8.0 using YCSB,
it looks like mean read latency is improved in cdh5.9 but mean write
latency is worse, and the net effect in a 50% read/50% write workload is a
performance degradation. FWIW, we've been testing with
`hbase.ipc.server.callqueue.type` = "deadline" on both versions.

Has anyone else using CDH 5.9.0 HBase run into this, or know what might be
the cause?

~Graham


Re: hbase/spark - Delegation Token can be issued only with kerberos or web authentication

2016-11-18 Thread Nkechi Achara
Sorry on my way to a flight.

Read is required for a keytab to be permissioned properly. So that looks
fine in your case.

I do not have my PC with me, but have you tried to use Hbase without using
Hbase context.

Also which version of Spark are you using?

On 18 Nov 2016 16:01, "Abel Fernández"  wrote:

> Yep, the keytab is also in the driver into the same location.
>
> -rw-r--r-- 1 hbase root  370 Nov 16 17:13 hbase.keytab
>
> Do you know what are the permissions that the keytab should have?
>
>
>
> On Fri, 18 Nov 2016 at 14:19 Nkechi Achara 
> wrote:
>
> > Sorry just realised you had the submit command in the attached docs.
> >
> > Can I ask if the keytab is also on the driver in the same location?
> >
> > The spark option normally requires the keytab to be on the driver so it
> can
> > pick it up and pass it to yarn etc to perform the kerberos operations.
> >
> > On 18 Nov 2016 3:10 p.m., "Abel Fernández"  wrote:
> >
> > > Hi Nkechi,
> > >
> > > Thank for your early response.
> > >
> > > I am currently specifying the principal and the keytab in the
> > spark-submit,
> > > the keytab is in the same location in every node manager.
> > >
> > > SPARK_CONF_DIR=conf-hbase spark-submit --master yarn-cluster \
> > >   --executor-memory 6G \
> > >   --num-executors 10 \
> > >   --queue cards \
> > >   --executor-cores 4 \
> > >   --driver-java-options "-Dlog4j.configuration=file:log4j.properties"
> \
> > >   --driver-class-path "$2" \
> > >   --jars file:/opt/orange/lib/rocksdbjni-4.5.1.jar \
> > >   --conf
> > > "spark.driver.extraClassPath=/var/cloudera/parcels/CDH/lib/
> > > hbase/lib/htrace-core-3.2.0-incubating.jar:/var/cloudera/
> > > parcels/CDH/jars/hbase-server-1.0.0-cdh5.5.4.jar:/var/
> > > cloudera/parcels/CDH/jars/hbase-common-1.0.0-cdh5.5.4.
> > > jar:/var/cloudera/parcels/CDH/lib/hbase/lib/hbase-client-1.
> > > 0.0-cdh5.5.4.jar:/var/cloudera/parcels/CDH/lib/
> > > hbase/lib/hbase-protocol-1.0.0-cdh5.5.4.jar:/opt/orange/
> > > lib/rocksdbjni-4.5.1.jar:/var/cloudera/parcels/CLABS_
> > > PHOENIX-4.5.2-1.clabs_phoenix1.2.0.p0.774/lib/
> > > phoenix/lib/phoenix-core-1.2.0.jar:/var/cloudera/parcels/
> > > CDH/jars/hadoop-mapreduce-client-core-2.6.0-cdh5.5.4.jar"
> > > \
> > >   --conf
> > > "spark.executor.extraClassPath=/var/cloudera/
> parcels/CDH/lib/hbase/lib/
> > > htrace-core-3.2.0-incubating.jar:/var/cloudera/parcels/CDH/
> > > jars/hbase-server-1.0.0-cdh5.5.4.jar:/var/cloudera/parcels/
> > > CDH/jars/hbase-common-1.0.0-cdh5.5.4.jar:/var/cloudera/
> > > parcels/CDH/lib/hbase/lib/hbase-client-1.0.0-cdh5.5.4.
> > > jar:/var/cloudera/parcels/CDH/lib/hbase/lib/hbase-protocol-
> > > 1.0.0-cdh5.5.4.jar:/opt/orange/lib/rocksdbjni-4.5.1.
> > > jar:/var/cloudera/parcels/CLABS_PHOENIX-4.5.2-1.clabs_
> > > phoenix1.2.0.p0.774/lib/phoenix/lib/phoenix-core-1.2.
> > > 0.jar:/var/cloudera/parcels/CDH/jars/hadoop-mapreduce-
> > > client-core-2.6.0-cdh5.5.4.jar"\
> > >   --principal hb...@company.corp \
> > >   --keytab /opt/company/conf/hbase.keytab \
> > >   --files
> > > "owl.properties,conf-hbase/log4j.properties,conf-hbase/
> > > hbase-site.xml,conf-hbase/core-site.xml,$2"
> > > \
> > >   --class $1 \
> > >   cards-batch-$3-jar-with-dependencies.jar $2
> > >
> > > On Fri, 18 Nov 2016 at 14:01 Nkechi Achara 
> > > wrote:
> > >
> > > > Can you use the principal and keytab options in Spark submit? These
> > > should
> > > > circumvent this issue.
> > > >
> > > > On 18 Nov 2016 1:01 p.m., "Abel Fernández" 
> > wrote:
> > > >
> > > > > Hello,
> > > > >
> > > > > We are having problems with the delegation of the token in a secure
> > > > > cluster: Delegation Token can be issued only with kerberos or web
> > > > > authentication
> > > > >
> > > > > We have a spark process which is generating the hfiles to be loaded
> > > into
> > > > > hbase. To generate these hfiles, (we are using a back-ported
> version
> > of
> > > > the
> > > > > latest hbase/spark code), we are using this method
> HBaseRDDFunctions.
> > > > > hbaseBulkLoadThinRows.
> > > > >
> > > > > I think the problem is in the below piece of code. This function is
> > > > > executed in every partition of the rdd, when the executors are
> trying
> > > to
> > > > > execute the code, the executors do not have a valid kerberos
> > credential
> > > > and
> > > > > cannot execute anything.
> > > > >
> > > > > private def hbaseForeachPartition[T](configBroadcast:
> > > > >
> > Broadcast[SerializableWritable[
> > > > > Configuration]],
> > > > > it: Iterator[T],
> > > > > f: (Iterator[T],
> Connection)
> > =>
> > > > > Unit) = {
> > > > >
> > > > > val config = getConf(configBroadcast)
> > > > >
> > > > > applyCreds
> > > > > // specify that this is a proxy user
> > > > > val smartConn = HBaseConnectionCache.getConnection(config)
> > > > > f(it, smartConn.connection)
> > > > > smartConn.close()
> > > > >   }
> > > > >
> > > > > I have attached the spark-submit 

Re: write performance regression in CDH 5.9?

2016-11-18 Thread Jeremy Carroll
Use FIFO. Much better in our testing

On Fri, Nov 18, 2016 at 7:46 AM Graham Baecher  wrote:

> Hi all,
>
> We're looking to update the HBase version we're running on our servers, but
> noticed while comparing performance test results a the previous version
> that Puts now seem to be slower.
> Specifically, comparing HBase 1.2.0-cdh5.9.0 to 1.2.0-cdh5.8.0 using YCSB,
> it looks like mean read latency is improved in cdh5.9 but mean write
> latency is worse, and the net effect in a 50% read/50% write workload is a
> performance degradation. FWIW, we've been testing with
> `hbase.ipc.server.callqueue.type` = "deadline" on both versions.
>
> Has anyone else using CDH 5.9.0 HBase run into this, or know what might be
> the cause?
>
> ~Graham
>


Re: hbase/spark - Delegation Token can be issued only with kerberos or web authentication

2016-11-18 Thread Abel Fernández
No worries.

This is the spark version we are using:  1.5.0-cdh5.5.4

I have to use Hbase context, it is the first parameter for the method I am
using to generate the HFiles (HbaseRDDFunctions.hbaseBulkLoadThinRows)

On Fri, 18 Nov 2016 at 16:06 Nkechi Achara  wrote:

> Sorry on my way to a flight.
>
> Read is required for a keytab to be permissioned properly. So that looks
> fine in your case.
>
> I do not have my PC with me, but have you tried to use Hbase without using
> Hbase context.
>
> Also which version of Spark are you using?
>
> On 18 Nov 2016 16:01, "Abel Fernández"  wrote:
>
> > Yep, the keytab is also in the driver into the same location.
> >
> > -rw-r--r-- 1 hbase root  370 Nov 16 17:13 hbase.keytab
> >
> > Do you know what are the permissions that the keytab should have?
> >
> >
> >
> > On Fri, 18 Nov 2016 at 14:19 Nkechi Achara 
> > wrote:
> >
> > > Sorry just realised you had the submit command in the attached docs.
> > >
> > > Can I ask if the keytab is also on the driver in the same location?
> > >
> > > The spark option normally requires the keytab to be on the driver so it
> > can
> > > pick it up and pass it to yarn etc to perform the kerberos operations.
> > >
> > > On 18 Nov 2016 3:10 p.m., "Abel Fernández" 
> wrote:
> > >
> > > > Hi Nkechi,
> > > >
> > > > Thank for your early response.
> > > >
> > > > I am currently specifying the principal and the keytab in the
> > > spark-submit,
> > > > the keytab is in the same location in every node manager.
> > > >
> > > > SPARK_CONF_DIR=conf-hbase spark-submit --master yarn-cluster \
> > > >   --executor-memory 6G \
> > > >   --num-executors 10 \
> > > >   --queue cards \
> > > >   --executor-cores 4 \
> > > >   --driver-java-options "-Dlog4j.configuration=file:log4j.properties"
> > \
> > > >   --driver-class-path "$2" \
> > > >   --jars file:/opt/orange/lib/rocksdbjni-4.5.1.jar \
> > > >   --conf
> > > > "spark.driver.extraClassPath=/var/cloudera/parcels/CDH/lib/
> > > > hbase/lib/htrace-core-3.2.0-incubating.jar:/var/cloudera/
> > > > parcels/CDH/jars/hbase-server-1.0.0-cdh5.5.4.jar:/var/
> > > > cloudera/parcels/CDH/jars/hbase-common-1.0.0-cdh5.5.4.
> > > > jar:/var/cloudera/parcels/CDH/lib/hbase/lib/hbase-client-1.
> > > > 0.0-cdh5.5.4.jar:/var/cloudera/parcels/CDH/lib/
> > > > hbase/lib/hbase-protocol-1.0.0-cdh5.5.4.jar:/opt/orange/
> > > > lib/rocksdbjni-4.5.1.jar:/var/cloudera/parcels/CLABS_
> > > > PHOENIX-4.5.2-1.clabs_phoenix1.2.0.p0.774/lib/
> > > > phoenix/lib/phoenix-core-1.2.0.jar:/var/cloudera/parcels/
> > > > CDH/jars/hadoop-mapreduce-client-core-2.6.0-cdh5.5.4.jar"
> > > > \
> > > >   --conf
> > > > "spark.executor.extraClassPath=/var/cloudera/
> > parcels/CDH/lib/hbase/lib/
> > > > htrace-core-3.2.0-incubating.jar:/var/cloudera/parcels/CDH/
> > > > jars/hbase-server-1.0.0-cdh5.5.4.jar:/var/cloudera/parcels/
> > > > CDH/jars/hbase-common-1.0.0-cdh5.5.4.jar:/var/cloudera/
> > > > parcels/CDH/lib/hbase/lib/hbase-client-1.0.0-cdh5.5.4.
> > > > jar:/var/cloudera/parcels/CDH/lib/hbase/lib/hbase-protocol-
> > > > 1.0.0-cdh5.5.4.jar:/opt/orange/lib/rocksdbjni-4.5.1.
> > > > jar:/var/cloudera/parcels/CLABS_PHOENIX-4.5.2-1.clabs_
> > > > phoenix1.2.0.p0.774/lib/phoenix/lib/phoenix-core-1.2.
> > > > 0.jar:/var/cloudera/parcels/CDH/jars/hadoop-mapreduce-
> > > > client-core-2.6.0-cdh5.5.4.jar"\
> > > >   --principal hb...@company.corp \
> > > >   --keytab /opt/company/conf/hbase.keytab \
> > > >   --files
> > > > "owl.properties,conf-hbase/log4j.properties,conf-hbase/
> > > > hbase-site.xml,conf-hbase/core-site.xml,$2"
> > > > \
> > > >   --class $1 \
> > > >   cards-batch-$3-jar-with-dependencies.jar $2
> > > >
> > > > On Fri, 18 Nov 2016 at 14:01 Nkechi Achara 
> > > > wrote:
> > > >
> > > > > Can you use the principal and keytab options in Spark submit? These
> > > > should
> > > > > circumvent this issue.
> > > > >
> > > > > On 18 Nov 2016 1:01 p.m., "Abel Fernández" 
> > > wrote:
> > > > >
> > > > > > Hello,
> > > > > >
> > > > > > We are having problems with the delegation of the token in a
> secure
> > > > > > cluster: Delegation Token can be issued only with kerberos or web
> > > > > > authentication
> > > > > >
> > > > > > We have a spark process which is generating the hfiles to be
> loaded
> > > > into
> > > > > > hbase. To generate these hfiles, (we are using a back-ported
> > version
> > > of
> > > > > the
> > > > > > latest hbase/spark code), we are using this method
> > HBaseRDDFunctions.
> > > > > > hbaseBulkLoadThinRows.
> > > > > >
> > > > > > I think the problem is in the below piece of code. This function
> is
> > > > > > executed in every partition of the rdd, when the executors are
> > trying
> > > > to
> > > > > > execute the code, the executors do not have a valid kerberos
> > > credential
> > > > > and
> > > > > > cannot execute anything.
> > > > > >
> > > > > > private def hbaseForeachPartition[T](configBroadcast:
> > > > > >
> > > Broadcast[SerializableWritable[
> > > > > > Configuration]],
> > > > > >  

HBase Schema

2016-11-18 Thread Pradheep Shanmugam
Hi,

I have table in Hbase which stores multiple versions of data in different rows.
The key is something like  . The timestamp 
will differ for multiple versions of the same document.
Orgs are skewed say one org may have 1 billion docs while some orgs may have 
just 100K docs.
So I decided to do salting to spread the write across all region servers and to 
improve the writes..
Also one more factor for considering salting is these docs will not be 
referenced after say 6 months and only the new ones will be queried often.

Assuming a hybrid load, will this affect my read(to get the latest version of a 
document given the ) performance of large and small orgs 
when there are more than 10 billion rows in total?

Thanks,
Pradheep



Re: Use experience and performance data of offheap from Alibaba online cluster

2016-11-18 Thread Yu Li
Sorry guys, let me retry the inline images:

Performance w/o offheap:

​
Performance w/ offheap:

​
Peak Get QPS of one single RS during Singles' Day (11/11):

​

And attach the files in case inline still not working:
​​​
 Performance_without_offheap.png

​​
 Performance_with_offheap.png

​​
 Peak_Get_QPS_of_Single_RS.png

​


Best Regards,
Yu

On 18 November 2016 at 19:29, Ted Yu  wrote:

> Yu:
> With positive results, more hbase users would be asking for the backport
> of offheap read path patches.
>
> Do you think you or your coworker has the bandwidth to publish backport
> for branch-1 ?
>
> Thanks
>
> > On Nov 18, 2016, at 12:11 AM, Yu Li  wrote:
> >
> > Dear all,
> >
> > We have backported read path offheap (HBASE-11425) to our customized
> hbase-1.1.2 (thanks @Anoop for the help/support) and run it online for more
> than a month, and would like to share our experience, for what it's worth
> (smile).
> >
> > Generally speaking, we gained a better and more stable
> throughput/performance with offheap, and below are some details:
> > 1. QPS become more stable with offheap
> >
> > Performance w/o offheap:
> >
> >
> >
> > Performance w/ offheap:
> >
> >
> >
> > These data come from our online A/B test cluster (with 450 physical
> machines, and each with 256G memory + 64 core) with real world workloads,
> it shows using offheap we could gain a more stable throughput as well as
> better performance
> >
> > Not showing fully online data here because for online we published the
> version with both offheap and NettyRpcServer together, so no standalone
> comparison data for offheap
> >
> > 2. Full GC frequency and cost
> >
> > Average Full GC STW time reduce from 11s to 7s with offheap.
> >
> > 3. Young GC frequency and cost
> >
> > No performance degradation observed with offheap.
> >
> > 4. Peak throughput of one single RS
> >
> > On Singles Day (11/11), peak throughput of one single RS reached 100K,
> among which 90K from Get. Plus internet in/out data we could know the
> average result size of get request is ~1KB
> >
> >
> >
> > Offheap are used on all online machines (more than 1600 nodes) instead
> of LruCache, so the above QPS is gained from offheap bucketcache, along
> with NettyRpcServer(HBASE-15756).
> >
> > Just let us know if any comments. Thanks.
> >
> > Best Regards,
> > Yu
> >
> >
> >
> >
> >
> >
> >
>


Re: write performance regression in CDH 5.9?

2016-11-18 Thread Graham Baecher
Thanks Jeremy.
I ran the regressed workload this morning with FIFO enabled and though it
was faster, it looks like it was a flat ~1ms faster for both reads and
writes. This doesn't compensate for the write performance dropoff, so even
with FIFO our 5.9 writes are slower than 5.8.

On Fri, Nov 18, 2016 at 11:29 AM, Jeremy Carroll 
wrote:

> Use FIFO. Much better in our testing
>
> On Fri, Nov 18, 2016 at 7:46 AM Graham Baecher 
> wrote:
>
> > Hi all,
> >
> > We're looking to update the HBase version we're running on our servers,
> but
> > noticed while comparing performance test results a the previous version
> > that Puts now seem to be slower.
> > Specifically, comparing HBase 1.2.0-cdh5.9.0 to 1.2.0-cdh5.8.0 using
> YCSB,
> > it looks like mean read latency is improved in cdh5.9 but mean write
> > latency is worse, and the net effect in a 50% read/50% write workload is
> a
> > performance degradation. FWIW, we've been testing with
> > `hbase.ipc.server.callqueue.type` = "deadline" on both versions.
> >
> > Has anyone else using CDH 5.9.0 HBase run into this, or know what might
> be
> > the cause?
> >
> > ~Graham
> >
>


Re: Use experience and performance data of offheap from Alibaba online cluster

2016-11-18 Thread Anoop John
Hi Yu Li
   Good to see that the off heap work help you..  The perf
numbers looks great.  So this is a compare of on heap L1 cache vs off heap
L2 cache(HBASE-11425 enabled).   So for 2.0 we should make L2 off heap
cache ON by default I believe.  Will raise a jira for that we can discuss
under that.   Seems like L2 off heap cache for data blocks and L1 cache for
index blocks seems a right choice.

Thanks for the backport and the help in testing the feature..  You were
able to find some corner case bugs and helped community to fix them..
Thanks goes to ur whole team.

-Anoop-


On Fri, Nov 18, 2016 at 10:14 PM, Yu Li  wrote:

> Sorry guys, let me retry the inline images:
>
> Performance w/o offheap:
>
> ​
> Performance w/ offheap:
>
> ​
> Peak Get QPS of one single RS during Singles' Day (11/11):
>
> ​
>
> And attach the files in case inline still not working:
> ​​​
>  Performance_without_offheap.png
> 
> ​​
>  Performance_with_offheap.png
> 
> ​​
>  Peak_Get_QPS_of_Single_RS.png
> 
> ​
>
>
> Best Regards,
> Yu
>
> On 18 November 2016 at 19:29, Ted Yu  wrote:
>
>> Yu:
>> With positive results, more hbase users would be asking for the backport
>> of offheap read path patches.
>>
>> Do you think you or your coworker has the bandwidth to publish backport
>> for branch-1 ?
>>
>> Thanks
>>
>> > On Nov 18, 2016, at 12:11 AM, Yu Li  wrote:
>> >
>> > Dear all,
>> >
>> > We have backported read path offheap (HBASE-11425) to our customized
>> hbase-1.1.2 (thanks @Anoop for the help/support) and run it online for more
>> than a month, and would like to share our experience, for what it's worth
>> (smile).
>> >
>> > Generally speaking, we gained a better and more stable
>> throughput/performance with offheap, and below are some details:
>> > 1. QPS become more stable with offheap
>> >
>> > Performance w/o offheap:
>> >
>> >
>> >
>> > Performance w/ offheap:
>> >
>> >
>> >
>> > These data come from our online A/B test cluster (with 450 physical
>> machines, and each with 256G memory + 64 core) with real world workloads,
>> it shows using offheap we could gain a more stable throughput as well as
>> better performance
>> >
>> > Not showing fully online data here because for online we published the
>> version with both offheap and NettyRpcServer together, so no standalone
>> comparison data for offheap
>> >
>> > 2. Full GC frequency and cost
>> >
>> > Average Full GC STW time reduce from 11s to 7s with offheap.
>> >
>> > 3. Young GC frequency and cost
>> >
>> > No performance degradation observed with offheap.
>> >
>> > 4. Peak throughput of one single RS
>> >
>> > On Singles Day (11/11), peak throughput of one single RS reached 100K,
>> among which 90K from Get. Plus internet in/out data we could know the
>> average result size of get request is ~1KB
>> >
>> >
>> >
>> > Offheap are used on all online machines (more than 1600 nodes) instead
>> of LruCache, so the above QPS is gained from offheap bucketcache, along
>> with NettyRpcServer(HBASE-15756).
>> >
>> > Just let us know if any comments. Thanks.
>> >
>> > Best Regards,
>> > Yu
>> >
>> >
>> >
>> >
>> >
>> >
>> >
>>
>
>


Re: Use experience and performance data of offheap from Alibaba online cluster

2016-11-18 Thread Bryan Beaudreault
Is the backported patch available anywhere? Not seeing it on the referenced
JIRA. If it ends up not getting officially backported to branch-1 due to
2.0 around the corner, some of us who build our own deploy may want to
integrate into our builds. Thanks! These numbers look great

On Fri, Nov 18, 2016 at 12:20 PM Anoop John  wrote:

> Hi Yu Li
>Good to see that the off heap work help you..  The perf
> numbers looks great.  So this is a compare of on heap L1 cache vs off heap
> L2 cache(HBASE-11425 enabled).   So for 2.0 we should make L2 off heap
> cache ON by default I believe.  Will raise a jira for that we can discuss
> under that.   Seems like L2 off heap cache for data blocks and L1 cache for
> index blocks seems a right choice.
>
> Thanks for the backport and the help in testing the feature..  You were
> able to find some corner case bugs and helped community to fix them..
> Thanks goes to ur whole team.
>
> -Anoop-
>
>
> On Fri, Nov 18, 2016 at 10:14 PM, Yu Li  wrote:
>
> > Sorry guys, let me retry the inline images:
> >
> > Performance w/o offheap:
> >
> > ​
> > Performance w/ offheap:
> >
> > ​
> > Peak Get QPS of one single RS during Singles' Day (11/11):
> >
> > ​
> >
> > And attach the files in case inline still not working:
> > ​​​
> >  Performance_without_offheap.png
> > <
> https://drive.google.com/file/d/0B017Q40_F5uwbWEzUGktYVIya3JkcXVjRkFvVGNtM0VxWC1n/view?usp=drive_web
> >
> > ​​
> >  Performance_with_offheap.png
> > <
> https://drive.google.com/file/d/0B017Q40_F5uweGR2cnJEU0M1MWwtRFJ5YkxUeFVrcUdPc2ww/view?usp=drive_web
> >
> > ​​
> >  Peak_Get_QPS_of_Single_RS.png
> > <
> https://drive.google.com/file/d/0B017Q40_F5uwQ2FkR2k0ZmEtRVNGSFp5RUxHM3F6bHpNYnJz/view?usp=drive_web
> >
> > ​
> >
> >
> > Best Regards,
> > Yu
> >
> > On 18 November 2016 at 19:29, Ted Yu  wrote:
> >
> >> Yu:
> >> With positive results, more hbase users would be asking for the backport
> >> of offheap read path patches.
> >>
> >> Do you think you or your coworker has the bandwidth to publish backport
> >> for branch-1 ?
> >>
> >> Thanks
> >>
> >> > On Nov 18, 2016, at 12:11 AM, Yu Li  wrote:
> >> >
> >> > Dear all,
> >> >
> >> > We have backported read path offheap (HBASE-11425) to our customized
> >> hbase-1.1.2 (thanks @Anoop for the help/support) and run it online for
> more
> >> than a month, and would like to share our experience, for what it's
> worth
> >> (smile).
> >> >
> >> > Generally speaking, we gained a better and more stable
> >> throughput/performance with offheap, and below are some details:
> >> > 1. QPS become more stable with offheap
> >> >
> >> > Performance w/o offheap:
> >> >
> >> >
> >> >
> >> > Performance w/ offheap:
> >> >
> >> >
> >> >
> >> > These data come from our online A/B test cluster (with 450 physical
> >> machines, and each with 256G memory + 64 core) with real world
> workloads,
> >> it shows using offheap we could gain a more stable throughput as well as
> >> better performance
> >> >
> >> > Not showing fully online data here because for online we published the
> >> version with both offheap and NettyRpcServer together, so no standalone
> >> comparison data for offheap
> >> >
> >> > 2. Full GC frequency and cost
> >> >
> >> > Average Full GC STW time reduce from 11s to 7s with offheap.
> >> >
> >> > 3. Young GC frequency and cost
> >> >
> >> > No performance degradation observed with offheap.
> >> >
> >> > 4. Peak throughput of one single RS
> >> >
> >> > On Singles Day (11/11), peak throughput of one single RS reached 100K,
> >> among which 90K from Get. Plus internet in/out data we could know the
> >> average result size of get request is ~1KB
> >> >
> >> >
> >> >
> >> > Offheap are used on all online machines (more than 1600 nodes) instead
> >> of LruCache, so the above QPS is gained from offheap bucketcache, along
> >> with NettyRpcServer(HBASE-15756).
> >> >
> >> > Just let us know if any comments. Thanks.
> >> >
> >> > Best Regards,
> >> > Yu
> >> >
> >> >
> >> >
> >> >
> >> >
> >> >
> >> >
> >>
> >
> >
>


Re: Use experience and performance data of offheap from Alibaba online cluster

2016-11-18 Thread Andrew Purtell
Yes, please, the patches will be useful to the community even if we decide not 
to backport into an official 1.x release.


> On Nov 18, 2016, at 12:25 PM, Bryan Beaudreault  
> wrote:
> 
> Is the backported patch available anywhere? Not seeing it on the referenced
> JIRA. If it ends up not getting officially backported to branch-1 due to
> 2.0 around the corner, some of us who build our own deploy may want to
> integrate into our builds. Thanks! These numbers look great
> 
>> On Fri, Nov 18, 2016 at 12:20 PM Anoop John  wrote:
>> 
>> Hi Yu Li
>>   Good to see that the off heap work help you..  The perf
>> numbers looks great.  So this is a compare of on heap L1 cache vs off heap
>> L2 cache(HBASE-11425 enabled).   So for 2.0 we should make L2 off heap
>> cache ON by default I believe.  Will raise a jira for that we can discuss
>> under that.   Seems like L2 off heap cache for data blocks and L1 cache for
>> index blocks seems a right choice.
>> 
>> Thanks for the backport and the help in testing the feature..  You were
>> able to find some corner case bugs and helped community to fix them..
>> Thanks goes to ur whole team.
>> 
>> -Anoop-
>> 
>> 
>>> On Fri, Nov 18, 2016 at 10:14 PM, Yu Li  wrote:
>>> 
>>> Sorry guys, let me retry the inline images:
>>> 
>>> Performance w/o offheap:
>>> 
>>> ​
>>> Performance w/ offheap:
>>> 
>>> ​
>>> Peak Get QPS of one single RS during Singles' Day (11/11):
>>> 
>>> ​
>>> 
>>> And attach the files in case inline still not working:
>>> ​​​
>>> Performance_without_offheap.png
>>> <
>> https://drive.google.com/file/d/0B017Q40_F5uwbWEzUGktYVIya3JkcXVjRkFvVGNtM0VxWC1n/view?usp=drive_web
>>> 
>>> ​​
>>> Performance_with_offheap.png
>>> <
>> https://drive.google.com/file/d/0B017Q40_F5uweGR2cnJEU0M1MWwtRFJ5YkxUeFVrcUdPc2ww/view?usp=drive_web
>>> 
>>> ​​
>>> Peak_Get_QPS_of_Single_RS.png
>>> <
>> https://drive.google.com/file/d/0B017Q40_F5uwQ2FkR2k0ZmEtRVNGSFp5RUxHM3F6bHpNYnJz/view?usp=drive_web
>>> 
>>> ​
>>> 
>>> 
>>> Best Regards,
>>> Yu
>>> 
 On 18 November 2016 at 19:29, Ted Yu  wrote:
 
 Yu:
 With positive results, more hbase users would be asking for the backport
 of offheap read path patches.
 
 Do you think you or your coworker has the bandwidth to publish backport
 for branch-1 ?
 
 Thanks
 
> On Nov 18, 2016, at 12:11 AM, Yu Li  wrote:
> 
> Dear all,
> 
> We have backported read path offheap (HBASE-11425) to our customized
 hbase-1.1.2 (thanks @Anoop for the help/support) and run it online for
>> more
 than a month, and would like to share our experience, for what it's
>> worth
 (smile).
> 
> Generally speaking, we gained a better and more stable
 throughput/performance with offheap, and below are some details:
> 1. QPS become more stable with offheap
> 
> Performance w/o offheap:
> 
> 
> 
> Performance w/ offheap:
> 
> 
> 
> These data come from our online A/B test cluster (with 450 physical
 machines, and each with 256G memory + 64 core) with real world
>> workloads,
 it shows using offheap we could gain a more stable throughput as well as
 better performance
> 
> Not showing fully online data here because for online we published the
 version with both offheap and NettyRpcServer together, so no standalone
 comparison data for offheap
> 
> 2. Full GC frequency and cost
> 
> Average Full GC STW time reduce from 11s to 7s with offheap.
> 
> 3. Young GC frequency and cost
> 
> No performance degradation observed with offheap.
> 
> 4. Peak throughput of one single RS
> 
> On Singles Day (11/11), peak throughput of one single RS reached 100K,
 among which 90K from Get. Plus internet in/out data we could know the
 average result size of get request is ~1KB
> 
> 
> 
> Offheap are used on all online machines (more than 1600 nodes) instead
 of LruCache, so the above QPS is gained from offheap bucketcache, along
 with NettyRpcServer(HBASE-15756).
> 
> Just let us know if any comments. Thanks.
> 
> Best Regards,
> Yu
> 
> 
> 
> 
> 
> 
> 
 
>>> 
>>> 
>> 


Re: write performance regression in CDH 5.9?

2016-11-18 Thread Stack
On Fri, Nov 18, 2016 at 7:46 AM, Graham Baecher 
wrote:

> Hi all,
>
> We're looking to update the HBase version we're running on our servers, but
> noticed while comparing performance test results a the previous version
> that Puts now seem to be slower.
> Specifically, comparing HBase 1.2.0-cdh5.9.0 to 1.2.0-cdh5.8.0 using YCSB,
> it looks like mean read latency is improved in cdh5.9 but mean write
> latency is worse, and the net effect in a 50% read/50% write workload is a
> performance degradation. FWIW, we've been testing with
> `hbase.ipc.server.callqueue.type` = "deadline" on both versions.
>
> Has anyone else using CDH 5.9.0 HBase run into this, or know what might be
> the cause?
>
>
Hey Graham:

If you do pure YCSB write, do you see a difference? You updated HBase and
HDFS? Whats cluster size and commands you are running so can try replicate?

You are using a vendor's hbase so we'd suggest you go ping the vendor on
their mailing lists. That said, I'm interested if a regression in the 1.2
line (I'm not sure anyone has run compares of patch versions on 1.2 line).
I'm game for trying to figure which commit was responsible. I
opened HBASE-17128 as a working issue on which we can ping/pong
configs/test runs.

S



> ~Graham
>


Describe-Table Java API

2016-11-18 Thread Ganesh Viswanathan
Hello,

Is there a java API for HBase (using Admin or other libraries) to describe
a table and get its columnFamilies? I see the *describe 'tablename'* shell
command in HBase. But I don't see one in the Admin API docs:

https://hbase.apache.org/apidocs/org/apache/hadoop/hbase/client/Admin.html


Thanks,
Ganesh


Re: Use experience and performance data of offheap from Alibaba online cluster

2016-11-18 Thread 曾伟展
+ 1

 原始邮件
发件人: 张铎
收件人: d...@hbase.apache.org; 
user@hbase.apache.org
发送时间: 2016年11月18日(周五) 17:19
主题: Re: Use experience and performance data of offheap from Alibaba online 
cluster

正在载入邮件原文…


Re: Describe-Table Java API

2016-11-18 Thread Ted Yu
https://hbase.apache.org/apidocs/org/apache/hadoop/hbase/client/Admin.html#getTableDescriptor-org.apache.hadoop.hbase.TableName-

On Fri, Nov 18, 2016 at 10:44 AM, Ganesh Viswanathan 
wrote:

> Hello,
>
> Is there a java API for HBase (using Admin or other libraries) to describe
> a table and get its columnFamilies? I see the *describe 'tablename'* shell
> command in HBase. But I don't see one in the Admin API docs:
>
> https://hbase.apache.org/apidocs/org/apache/hadoop/hbase/client/Admin.html
>
>
> Thanks,
> Ganesh
>


Re: Use experience and performance data of offheap from Alibaba online cluster

2016-11-18 Thread Enis Söztutar
Thanks for sharing this. Great work.

I don't see any reason why we cannot backport to branch-1.

Enis

On Fri, Nov 18, 2016 at 9:37 AM, Andrew Purtell 
wrote:

> Yes, please, the patches will be useful to the community even if we decide
> not to backport into an official 1.x release.
>
>
> > On Nov 18, 2016, at 12:25 PM, Bryan Beaudreault <
> bbeaudrea...@hubspot.com> wrote:
> >
> > Is the backported patch available anywhere? Not seeing it on the
> referenced
> > JIRA. If it ends up not getting officially backported to branch-1 due to
> > 2.0 around the corner, some of us who build our own deploy may want to
> > integrate into our builds. Thanks! These numbers look great
> >
> >> On Fri, Nov 18, 2016 at 12:20 PM Anoop John 
> wrote:
> >>
> >> Hi Yu Li
> >>   Good to see that the off heap work help you..  The perf
> >> numbers looks great.  So this is a compare of on heap L1 cache vs off
> heap
> >> L2 cache(HBASE-11425 enabled).   So for 2.0 we should make L2 off heap
> >> cache ON by default I believe.  Will raise a jira for that we can
> discuss
> >> under that.   Seems like L2 off heap cache for data blocks and L1 cache
> for
> >> index blocks seems a right choice.
> >>
> >> Thanks for the backport and the help in testing the feature..  You were
> >> able to find some corner case bugs and helped community to fix them..
> >> Thanks goes to ur whole team.
> >>
> >> -Anoop-
> >>
> >>
> >>> On Fri, Nov 18, 2016 at 10:14 PM, Yu Li  wrote:
> >>>
> >>> Sorry guys, let me retry the inline images:
> >>>
> >>> Performance w/o offheap:
> >>>
> >>> ​
> >>> Performance w/ offheap:
> >>>
> >>> ​
> >>> Peak Get QPS of one single RS during Singles' Day (11/11):
> >>>
> >>> ​
> >>>
> >>> And attach the files in case inline still not working:
> >>> ​​​
> >>> Performance_without_offheap.png
> >>> <
> >> https://drive.google.com/file/d/0B017Q40_F5uwbWEzUGktYVIya3JkcXVjRkFvVG
> NtM0VxWC1n/view?usp=drive_web
> >>>
> >>> ​​
> >>> Performance_with_offheap.png
> >>> <
> >> https://drive.google.com/file/d/0B017Q40_F5uweGR2cnJEU0M1MWwtRFJ5YkxUeF
> VrcUdPc2ww/view?usp=drive_web
> >>>
> >>> ​​
> >>> Peak_Get_QPS_of_Single_RS.png
> >>> <
> >> https://drive.google.com/file/d/0B017Q40_F5uwQ2FkR2k0ZmEtRVNGSFp5RUxHM3
> F6bHpNYnJz/view?usp=drive_web
> >>>
> >>> ​
> >>>
> >>>
> >>> Best Regards,
> >>> Yu
> >>>
>  On 18 November 2016 at 19:29, Ted Yu  wrote:
> 
>  Yu:
>  With positive results, more hbase users would be asking for the
> backport
>  of offheap read path patches.
> 
>  Do you think you or your coworker has the bandwidth to publish
> backport
>  for branch-1 ?
> 
>  Thanks
> 
> > On Nov 18, 2016, at 12:11 AM, Yu Li  wrote:
> >
> > Dear all,
> >
> > We have backported read path offheap (HBASE-11425) to our customized
>  hbase-1.1.2 (thanks @Anoop for the help/support) and run it online for
> >> more
>  than a month, and would like to share our experience, for what it's
> >> worth
>  (smile).
> >
> > Generally speaking, we gained a better and more stable
>  throughput/performance with offheap, and below are some details:
> > 1. QPS become more stable with offheap
> >
> > Performance w/o offheap:
> >
> >
> >
> > Performance w/ offheap:
> >
> >
> >
> > These data come from our online A/B test cluster (with 450 physical
>  machines, and each with 256G memory + 64 core) with real world
> >> workloads,
>  it shows using offheap we could gain a more stable throughput as well
> as
>  better performance
> >
> > Not showing fully online data here because for online we published
> the
>  version with both offheap and NettyRpcServer together, so no
> standalone
>  comparison data for offheap
> >
> > 2. Full GC frequency and cost
> >
> > Average Full GC STW time reduce from 11s to 7s with offheap.
> >
> > 3. Young GC frequency and cost
> >
> > No performance degradation observed with offheap.
> >
> > 4. Peak throughput of one single RS
> >
> > On Singles Day (11/11), peak throughput of one single RS reached
> 100K,
>  among which 90K from Get. Plus internet in/out data we could know the
>  average result size of get request is ~1KB
> >
> >
> >
> > Offheap are used on all online machines (more than 1600 nodes)
> instead
>  of LruCache, so the above QPS is gained from offheap bucketcache,
> along
>  with NettyRpcServer(HBASE-15756).
> >
> > Just let us know if any comments. Thanks.
> >
> > Best Regards,
> > Yu
> >
> >
> >
> >
> >
> >
> >
> 
> >>>
> >>>
> >>
>


Re: Describe-Table Java API

2016-11-18 Thread Ganesh Viswanathan
Thanks Ted. Yes, that worked great.



On Fri, Nov 18, 2016 at 10:51 AM, Ted Yu  wrote:

> https://hbase.apache.org/apidocs/org/apache/hadoop/hbase/
> client/Admin.html#getTableDescriptor-org.apache.hadoop.hbase.TableName-
>
> On Fri, Nov 18, 2016 at 10:44 AM, Ganesh Viswanathan 
> wrote:
>
> > Hello,
> >
> > Is there a java API for HBase (using Admin or other libraries) to
> describe
> > a table and get its columnFamilies? I see the *describe 'tablename'*
> shell
> > command in HBase. But I don't see one in the Admin API docs:
> >
> > https://hbase.apache.org/apidocs/org/apache/hadoop/hbase/
> client/Admin.html
> >
> >
> > Thanks,
> > Ganesh
> >
>


Rolling upgrade with coprocessors

2016-11-18 Thread Albert Shau
Hi all,
I'm using coprocessors with my tables and am wondering how I would perform an 
HBase rolling upgrade, since it seems like there are no compatibility 
guarantees for the coprocessor APIs. I'm guessing I would have to disable the 
table, alter the table to use a coprocessor compatible with the next HBase 
version, perform the rolling upgrade, then enable the table? Is there a way to 
keep the table enabled for this?
Thanks,Albert

Re: Rolling upgrade with coprocessors

2016-11-18 Thread Ted Yu
Can you tell us the version of hbase you are using and the new version
which you plan to upgrade to ?

A bit more detail on your coprocessor would also help narrow the scope of
search.

Cheers

On Fri, Nov 18, 2016 at 4:28 PM, Albert Shau 
wrote:

> Hi all,
> I'm using coprocessors with my tables and am wondering how I would perform
> an HBase rolling upgrade, since it seems like there are no compatibility
> guarantees for the coprocessor APIs. I'm guessing I would have to disable
> the table, alter the table to use a coprocessor compatible with the next
> HBase version, perform the rolling upgrade, then enable the table? Is there
> a way to keep the table enabled for this?
> Thanks,Albert


Re: Use experience and performance data of offheap from Alibaba online cluster

2016-11-18 Thread Anoop John
Because of some compatibility issues, we decide that this will be done
in 2.0 only..  Ya as Andy said, it would be great to share the 1.x
backported patches.  Is it a mega patch at ur end?  Or issue by issue
patches?  Latter would be best.  Pls share patches in some place and a
list of issues backported. I can help with verifying the issues once
so as to make sure we dont miss any...

-Anoop-

On Sat, Nov 19, 2016 at 12:32 AM, Enis Söztutar  wrote:
> Thanks for sharing this. Great work.
>
> I don't see any reason why we cannot backport to branch-1.
>
> Enis
>
> On Fri, Nov 18, 2016 at 9:37 AM, Andrew Purtell 
> wrote:
>
>> Yes, please, the patches will be useful to the community even if we decide
>> not to backport into an official 1.x release.
>>
>>
>> > On Nov 18, 2016, at 12:25 PM, Bryan Beaudreault <
>> bbeaudrea...@hubspot.com> wrote:
>> >
>> > Is the backported patch available anywhere? Not seeing it on the
>> referenced
>> > JIRA. If it ends up not getting officially backported to branch-1 due to
>> > 2.0 around the corner, some of us who build our own deploy may want to
>> > integrate into our builds. Thanks! These numbers look great
>> >
>> >> On Fri, Nov 18, 2016 at 12:20 PM Anoop John 
>> wrote:
>> >>
>> >> Hi Yu Li
>> >>   Good to see that the off heap work help you..  The perf
>> >> numbers looks great.  So this is a compare of on heap L1 cache vs off
>> heap
>> >> L2 cache(HBASE-11425 enabled).   So for 2.0 we should make L2 off heap
>> >> cache ON by default I believe.  Will raise a jira for that we can
>> discuss
>> >> under that.   Seems like L2 off heap cache for data blocks and L1 cache
>> for
>> >> index blocks seems a right choice.
>> >>
>> >> Thanks for the backport and the help in testing the feature..  You were
>> >> able to find some corner case bugs and helped community to fix them..
>> >> Thanks goes to ur whole team.
>> >>
>> >> -Anoop-
>> >>
>> >>
>> >>> On Fri, Nov 18, 2016 at 10:14 PM, Yu Li  wrote:
>> >>>
>> >>> Sorry guys, let me retry the inline images:
>> >>>
>> >>> Performance w/o offheap:
>> >>>
>> >>>
>> >>> Performance w/ offheap:
>> >>>
>> >>>
>> >>> Peak Get QPS of one single RS during Singles' Day (11/11):
>> >>>
>> >>>
>> >>>
>> >>> And attach the files in case inline still not working:
>> >>>
>> >>> Performance_without_offheap.png
>> >>> <
>> >> https://drive.google.com/file/d/0B017Q40_F5uwbWEzUGktYVIya3JkcXVjRkFvVG
>> NtM0VxWC1n/view?usp=drive_web
>> >>>
>> >>>
>> >>> Performance_with_offheap.png
>> >>> <
>> >> https://drive.google.com/file/d/0B017Q40_F5uweGR2cnJEU0M1MWwtRFJ5YkxUeF
>> VrcUdPc2ww/view?usp=drive_web
>> >>>
>> >>>
>> >>> Peak_Get_QPS_of_Single_RS.png
>> >>> <
>> >> https://drive.google.com/file/d/0B017Q40_F5uwQ2FkR2k0ZmEtRVNGSFp5RUxHM3
>> F6bHpNYnJz/view?usp=drive_web
>> >>>
>> >>>
>> >>>
>> >>>
>> >>> Best Regards,
>> >>> Yu
>> >>>
>>  On 18 November 2016 at 19:29, Ted Yu  wrote:
>> 
>>  Yu:
>>  With positive results, more hbase users would be asking for the
>> backport
>>  of offheap read path patches.
>> 
>>  Do you think you or your coworker has the bandwidth to publish
>> backport
>>  for branch-1 ?
>> 
>>  Thanks
>> 
>> > On Nov 18, 2016, at 12:11 AM, Yu Li  wrote:
>> >
>> > Dear all,
>> >
>> > We have backported read path offheap (HBASE-11425) to our customized
>>  hbase-1.1.2 (thanks @Anoop for the help/support) and run it online for
>> >> more
>>  than a month, and would like to share our experience, for what it's
>> >> worth
>>  (smile).
>> >
>> > Generally speaking, we gained a better and more stable
>>  throughput/performance with offheap, and below are some details:
>> > 1. QPS become more stable with offheap
>> >
>> > Performance w/o offheap:
>> >
>> >
>> >
>> > Performance w/ offheap:
>> >
>> >
>> >
>> > These data come from our online A/B test cluster (with 450 physical
>>  machines, and each with 256G memory + 64 core) with real world
>> >> workloads,
>>  it shows using offheap we could gain a more stable throughput as well
>> as
>>  better performance
>> >
>> > Not showing fully online data here because for online we published
>> the
>>  version with both offheap and NettyRpcServer together, so no
>> standalone
>>  comparison data for offheap
>> >
>> > 2. Full GC frequency and cost
>> >
>> > Average Full GC STW time reduce from 11s to 7s with offheap.
>> >
>> > 3. Young GC frequency and cost
>> >
>> > No performance degradation observed with offheap.
>> >
>> > 4. Peak throughput of one single RS
>> >
>> > On Singles Day (11/11), peak throughput of one single RS reached
>> 100K,
>>  among which 90K from Get. Plus internet in/out data we could know the
>>  average result size of get request is ~1KB
>> >
>> >
>> >
>> > Offheap are used on all online machines (more than 1600 nodes)
>> instead
>>