You are viewing a plain text version of this content. The canonical link for it is here.
Posted to dev@hbase.apache.org by Mikael Sitruk <mi...@gmail.com> on 2012/01/08 16:25:02 UTC

Major Compaction Concerns

Hi



I have some concern regarding major compactions below...


   1. According to best practices from the mailing list and from the book,
   automatic major compaction should be disabled. This can be done by setting
   the property ‘hbase.hregion.majorcompaction’ to ‘0’. Neverhteless even
   after having doing this I STILL see “major compaction” messages in logs.
   therefore it is unclear how can I manage major compactions. (The system has
   heavy insert - uniformly on the cluster, and major compaction affect the
   performance of the system).
   If I'm not wrong it seems from the code that: even if not requested and
   even if the indicator is set to '0' (no automatic major compaction), major
   compaction can be triggered by the code in case all store files are
   candidate for a compaction (from Store.compact(final boolean forceMajor)).
   Shouldn't the code add a condition that automatic major compaction is
   disabled??

   2. I tried to check the parameter  ‘hbase.hregion.majorcompaction’  at
   runtime using several approaches - to validate that the server indeed
   loaded the parameter.

a. Using a connection created from local config

*conn = (HConnection) HConnectionManager.getConnection(m_hbConfig);*

*conn.getConfiguration().getString(“hbase.hregion.majorcompaction”)*

returns the parameter from local config and not from cluster. Is it a bug?
If I set the property via the configuration shouldn’t all the cluster be
aware of? (supposing that the connection indeed connected to the cluster)

b.  fetching the property from the table descriptor

*HTableDescriptor hTableDescriptor =
conn.getHTableDescriptor(Bytes.toBytes("my table"));*

*hTableDescriptor.getValue("hbase.hregion.majorcompaction")*

This will returns the default parameter value (1 day) not the parameter
from the configuration (on the cluster). It seems to be a bug, isn’t it?
(the parameter from the config, should be the default if not set at the
table level)

c. The only way I could set the parameter to 0 and really see it is via the
Admin API, updating the table descriptor or the column descriptor. Now I
could see the parameter on the web UI. So is it the only way to set
correctly the parameter? If setting the parameter via the configuration
file, shouldn’t the webUI show this on any table created?

d. I tried also to setup the parameter via hbase shell but setting such
properties is not supported. (do you plan to add such support via the
shell?)

e. Generally is it possible to get via API the configuration used by the
servers? (at cluster/server level)

    3.  I ran both major compaction  requests from the shell or from API
but since both are async there is no progress indication. Neither the JMX
nor the Web will help here since you don’t know if a compaction task is
running. Tailling the logs is not an efficient way to do this neither. The
point is that I would like to automate the process and avoid compaction
storm. So I want to do that region, region, but if I don’t know when a
compaction started/ended I can’t automate it.

4.       In case there is no compaction files in queue (but still you have
more than 1 storefile per store e.g. minor compaction just finished) then
invoking major_compact will indeed decrease the number of store files, but
the compaction queue will remain to 0 during the compaction task (shouldn’t
the compaction queue increase by the number of file to compact and be
reduced when the task ended?)


5.       I saw already HBASE-3965 for getting status of major compaction,
nevertheless it has be removed from 0.92, is it possible to put it back?
Even sooner than 0.92?

6.       In case a compaction (major) is running it seems there is no way
to stop-it. Do you plan to add such feature?

7.       Do you plan to add functionality via JMX (starting/stopping
compaction, splitting....)

8.       Finally there were some request for allowing custom compaction,
part of this was given via the RegionObserver in HBASE-2001, nevertheless
do you consider adding support for custom compaction (providing real
pluggable compaction stategy not just observer)?


Regards,
Mikael.S

Re: Major Compaction Concerns

Posted by Ted Yu <yu...@gmail.com>.
For #2 below, I suggest more validation against 0.90.5 - 0.90.1 is pretty
old.

Cheers

On Sun, Jan 8, 2012 at 3:05 PM, Mikael Sitruk <mi...@gmail.com>wrote:

> Ted hi
>
> 1. thanks for pointing on  HBASE-3051, Compaction at the granularity of a
> column-family, it seems promising
>
> 2. Regarding manual management of compaction - it is exactly what i tried
> to do and found all the finding. *In short there is no way to disable major
> compaction from running automatically* (point #1 in original email), should
> a JIRA be opened?
>
> 3. I have opened the following ones
> HBASE-5146  - Hbase Shell - allow setting config properties
> HBASE-5147 - Compaction/Major compaction operation from shell/API/JMX
> HBASE-5148 - Compaction property at the server level are not propagated at
> the table level
> HBASE-5149 - getConfiguration() implementation is misleading
>
> Regards,
> Mikael.S
>
> On Sun, Jan 8, 2012 at 11:07 PM, Ted Yu <yu...@gmail.com> wrote:
>
> > HBASE-3051, Compaction at the granularity of a column-family, is marked
> > implemented by HBASE-3796
> > <https://issues.apache.org/jira/browse/HBASE-3796>which is in 0.92
> > (0.92 RC3 is coming out soon)
> >
> > Please see http://hbase.apache.org/book/regions.arch.html, 8.7.5.5 which
> > refers to
> >
> >
> http://hbase.apache.org/book/important_configurations.html#managed.compactions
> >
> > Cheers
> >
> > On Sun, Jan 8, 2012 at 12:55 PM, Mikael Sitruk <mikael.sitruk@gmail.com
> > >wrote:
> >
> > > Well I'm very interested to dig further. I can also tell that the
> number
> > of
> > > log is getting very high very fast and of course a flush is triggered
> > > adding more store files. Very fast the high number of store files
> trigger
> > > compaction and delay the flushing (default delay is 90000 ms).  The
> files
> > > are small in size, major compaction is not needed but minor yes.
> > > Nevertheless the code ignore the disabled automatic compaction and
> > promotes
> > > files to major compaction.
> > > I think I need to play with both the log file size the compaction
> > threshold
> > > and the Max number of stores file. Do you have some recommendations?
> > > Btw the compaction take about 1min 40 sec for a store size of 900MB
> +/-.
> > Is
> > > it normal?
> > > One thing that does not help in this story is that I have 2 column
> > families
> > > and each RS manages 100 of regions each cf growth with differents
> speed.
> > > Is there a version of hbase handling better such case (not flushing
> both
> > cf
> > > if not needed to)?
> > >
> > > I will review the release note of the versions you suggested and open
> > > issues/enhancements we discuss.
> > >
> > > Thanks
> > > Cheers.
> > > On Jan 8, 2012 10:22 PM, "Ted Yu" <yu...@gmail.com> wrote:
> > >
> > > > Your request in first paragraph below deserves a JIRA.
> > > >
> > > > For 2.b I agree a bug should be filed.
> > > >
> > > > For major compaction, adding more logs on region server side should
> > help
> > > > you understand the situation better - assuming you have interest to
> dig
> > > > further.
> > > > Please upgrade to 0.90.5, or you can wait for 0.90.6 release which is
> > > > slated for Jan. 19th.
> > > >
> > > > After upgrade, the logs and code would be more pertinent to the tip
> of
> > > 0.90
> > > > branch.
> > > >
> > > > Thanks for summarizing your findings.
> > > >
> > > > On Sun, Jan 8, 2012 at 12:04 PM, Mikael Sitruk <
> > mikael.sitruk@gmail.com
> > > > >wrote:
> > > >
> > > > > In fact I think that for 2.a the current implementation is
> > misleading.
> > > > > Creating a connection and getting the configuration from the
> > connection
> > > > > should return the configuration of the cluster.
> > > > > Requesting the configuration used to build an object should return
> > the
> > > > > configuration set on the object
> > > > > Additionally it should be a new method like getConfigurations(), or
> > > > > getClusterConfigurations() returning a map of serverinfo and
> > > > > configuration.  Another option is to add on the HRegionServer and
> > > > HMaster a
> > > > > method getConfiguration() returning the configuration object used
> by
> > > the
> > > > > RegionServer or Master
> > > > >
> > > > > Regarding 2.b yes I tried but it did not return the setting from
> the
> > > > > cluster configuration (again server has non default configuration,
> > > table
> > > > > was not configured with specific values then cluster configuration
> > > should
> > > > > apply on the table object). So I see it as problematic.
> > > > >
> > > > > Mikael.s
> > > > >  On Jan 8, 2012 7:54 PM, <yu...@gmail.com> wrote:
> > > > >
> > > > > > About 2b, have you tried getting the major compaction setting
> from
> > > > column
> > > > > > descriptor ?
> > > > > >
> > > > > > For 2a, what you requested would result in new methods of
> > > > > > HBaseConfiguration class to be added. Currently the configuration
> > on
> > > > > client
> > > > > > class path would be used.
> > > > > >
> > > > > > Cheers
> > > > > >
> > > > > >
> > > > > >
> > > > > > On Jan 8, 2012, at 9:28 AM, Mikael Sitruk <
> mikael.sitruk@gmail.com
> > >
> > > > > wrote:
> > > > > >
> > > > > > > Ted hi
> > > > > > > First thanks for answering, regarding the JIRA i will fill them
> > > > > > > Second, it seems that i did not explain myself correctly
> > regarding
> > > > > 2.a. -
> > > > > > > As you i do not expect that a configuration set on my client
> will
> > > be
> > > > > > > propagated to the cluster, but i do expect that if i set a
> > > > > configuration
> > > > > > on
> > > > > > > a server then doing connection.getConfiguration() from a
> client i
> > > > will
> > > > > > get
> > > > > > > teh configuration from the cluster.
> > > > > > > Currently the configuration returned is from the client config.
> > > > > > > So the problem is that you have no way to check the
> configuration
> > > of
> > > > a
> > > > > > > cluster.
> > > > > > > I would expect to have some API to return the cluster config
> and
> > > even
> > > > > > > getting a map <serverInfo, config> so it can be easy to check
> > > cluster
> > > > > > > problem using code.
> > > > > > >
> > > > > > > 2.b. I know this code, and i tried to validate it. I set in the
> > > > server
> > > > > > > config the "hbase.hregion.majorcompaction" to "0", then start
> the
> > > > > server
> > > > > > > (cluster). Since from the UI or from JMX this parameter is not
> > > > visible
> > > > > at
> > > > > > > the cluster level, I try to get the value from the client (to
> see
> > > > that
> > > > > > the
> > > > > > > cluster is using it)
> > > > > > >
> > > > > > > *HTableDescriptor hTableDescriptor =
> > > > > > > conn.getHTableDescriptor(Bytes.toBytes("my table"));*
> > > > > > >
> > > > > > > *hTableDescriptor.getValue("hbase.hregion.majorcompaction")*
> > > > > > > but i still got 24h (and not the value set in the config)! that
> > was
> > > > my
> > > > > > > problem from the beginning! ==> Using the config (on the server
> > > side)
> > > > > > will
> > > > > > > not propagate into the table/column family
> > > > > > >
> > > > > > > Mikael.S
> > > > > > >
> > > > > > > On Sun, Jan 8, 2012 at 7:13 PM, Ted Yu <yu...@gmail.com>
> > > wrote:
> > > > > > >
> > > > > > >> I am not expert in major compaction feature.
> > > > > > >> Let me try to answer questions in #2.
> > > > > > >>
> > > > > > >> 2.a
> > > > > > >>> If I set the property via the configuration shouldn’t all the
> > > > cluster
> > > > > > be
> > > > > > >>> aware of?
> > > > > > >>
> > > > > > >> There're multiple clients connecting to one cluster. I
> wouldn't
> > > > expect
> > > > > > >> values in the configuration (m_hbConfig) to propagate onto the
> > > > > cluster.
> > > > > > >>
> > > > > > >> 2.b
> > > > > > >> Store.getNextMajorCompactTime() shows that
> > > > > > "hbase.hregion.majorcompaction"
> > > > > > >> can be specified per column family:
> > > > > > >>
> > > > > > >> long getNextMajorCompactTime() {
> > > > > > >>   // default = 24hrs
> > > > > > >>   long ret = conf.getLong(HConstants.MAJOR_COMPACTION_PERIOD,
> > > > > > >> 1000*60*60*24);
> > > > > > >>   if (family.getValue(HConstants.MAJOR_COMPACTION_PERIOD) !=
> > > null) {
> > > > > > >>
> > > > > > >> 2.d
> > > > > > >>> d. I tried also to setup the parameter via hbase shell but
> > > setting
> > > > > such
> > > > > > >>> properties is not supported. (do you plan to add such support
> > via
> > > > the
> > > > > > >>> shell?)
> > > > > > >>
> > > > > > >> This is a good idea. Please open a JIRA.
> > > > > > >>
> > > > > > >> For #5, HBASE-3965 is an improvement and doesn't have a patch
> > yet.
> > > > > > >>
> > > > > > >> Allow me to quote Alan Kay: 'The best way to predict the
> future
> > is
> > > > to
> > > > > > >> invent it.'
> > > > > > >>
> > > > > > >> Once we have a patch, we can always backport it to 0.92 after
> > some
> > > > > > people
> > > > > > >> have verified the improvement.
> > > > > > >>
> > > > > > >>> 6.       In case a compaction (major) is running it seems
> there
> > > is
> > > > no
> > > > > > way
> > > > > > >>> to stop-it. Do you plan to add such feature?
> > > > > > >>
> > > > > > >> Again, logging a JIRA would provide a good starting point for
> > > > > > discussion.
> > > > > > >>
> > > > > > >> Thanks for the verification work and suggestions, Mikael.
> > > > > > >>
> > > > > > >> On Sun, Jan 8, 2012 at 7:27 AM, Mikael Sitruk <
> > > > > mikael.sitruk@gmail.com
> > > > > > >>> wrote:
> > > > > > >>
> > > > > > >>> I forgot to mention, I'm using HBase 0.90.1
> > > > > > >>>
> > > > > > >>> Regards,
> > > > > > >>> Mikael.S
> > > > > > >>>
> > > > > > >>> On Sun, Jan 8, 2012 at 5:25 PM, Mikael Sitruk <
> > > > > mikael.sitruk@gmail.com
> > > > > > >>>> wrote:
> > > > > > >>>
> > > > > > >>>> Hi
> > > > > > >>>>
> > > > > > >>>>
> > > > > > >>>>
> > > > > > >>>> I have some concern regarding major compactions below...
> > > > > > >>>>
> > > > > > >>>>
> > > > > > >>>>   1. According to best practices from the mailing list and
> > from
> > > > the
> > > > > > >>>>   book, automatic major compaction should be disabled. This
> > can
> > > be
> > > > > > >> done
> > > > > > >>> by
> > > > > > >>>>   setting the property ‘hbase.hregion.majorcompaction’ to
> ‘0’.
> > > > > > >>> Neverhteless
> > > > > > >>>>   even after having doing this I STILL see “major
> compaction”
> > > > > messages
> > > > > > >>> in
> > > > > > >>>>   logs. therefore it is unclear how can I manage major
> > > > compactions.
> > > > > > >> (The
> > > > > > >>>>   system has heavy insert - uniformly on the cluster, and
> > major
> > > > > > >>> compaction
> > > > > > >>>>   affect the performance of the system).
> > > > > > >>>>   If I'm not wrong it seems from the code that: even if not
> > > > > requested
> > > > > > >>>>   and even if the indicator is set to '0' (no automatic
> major
> > > > > > >>> compaction),
> > > > > > >>>>   major compaction can be triggered by the code in case all
> > > store
> > > > > > >> files
> > > > > > >>> are
> > > > > > >>>>   candidate for a compaction (from Store.compact(final
> boolean
> > > > > > >>> forceMajor)).
> > > > > > >>>>   Shouldn't the code add a condition that automatic major
> > > > compaction
> > > > > > >> is
> > > > > > >>>>   disabled??
> > > > > > >>>>
> > > > > > >>>>   2. I tried to check the parameter
> > > >  ‘hbase.hregion.majorcompaction’
> > > > > > >> at
> > > > > > >>>>   runtime using several approaches - to validate that the
> > server
> > > > > > >> indeed
> > > > > > >>>>   loaded the parameter.
> > > > > > >>>>
> > > > > > >>>> a. Using a connection created from local config
> > > > > > >>>>
> > > > > > >>>> *conn = (HConnection)
> > > > HConnectionManager.getConnection(m_hbConfig);*
> > > > > > >>>>
> > > > > > >>>>
> > > > *conn.getConfiguration().getString(“hbase.hregion.majorcompaction”)*
> > > > > > >>>>
> > > > > > >>>> returns the parameter from local config and not from
> cluster.
> > Is
> > > > it
> > > > > a
> > > > > > >>> bug?
> > > > > > >>>> If I set the property via the configuration shouldn’t all
> the
> > > > > cluster
> > > > > > >> be
> > > > > > >>>> aware of? (supposing that the connection indeed connected to
> > the
> > > > > > >> cluster)
> > > > > > >>>>
> > > > > > >>>> b.  fetching the property from the table descriptor
> > > > > > >>>>
> > > > > > >>>> *HTableDescriptor hTableDescriptor =
> > > > > > >>>> conn.getHTableDescriptor(Bytes.toBytes("my table"));*
> > > > > > >>>>
> > > > > > >>>> *hTableDescriptor.getValue("hbase.hregion.majorcompaction")*
> > > > > > >>>>
> > > > > > >>>> This will returns the default parameter value (1 day) not
> the
> > > > > > parameter
> > > > > > >>>> from the configuration (on the cluster). It seems to be a
> bug,
> > > > isn’t
> > > > > > >> it?
> > > > > > >>>> (the parameter from the config, should be the default if not
> > set
> > > > at
> > > > > > the
> > > > > > >>>> table level)
> > > > > > >>>>
> > > > > > >>>> c. The only way I could set the parameter to 0 and really
> see
> > it
> > > > is
> > > > > > via
> > > > > > >>>> the Admin API, updating the table descriptor or the column
> > > > > descriptor.
> > > > > > >>> Now
> > > > > > >>>> I could see the parameter on the web UI. So is it the only
> way
> > > to
> > > > > set
> > > > > > >>>> correctly the parameter? If setting the parameter via the
> > > > > > configuration
> > > > > > >>>> file, shouldn’t the webUI show this on any table created?
> > > > > > >>>>
> > > > > > >>>> d. I tried also to setup the parameter via hbase shell but
> > > setting
> > > > > > such
> > > > > > >>>> properties is not supported. (do you plan to add such
> support
> > > via
> > > > > the
> > > > > > >>>> shell?)
> > > > > > >>>>
> > > > > > >>>> e. Generally is it possible to get via API the configuration
> > > used
> > > > by
> > > > > > >> the
> > > > > > >>>> servers? (at cluster/server level)
> > > > > > >>>>
> > > > > > >>>>    3.  I ran both major compaction  requests from the shell
> or
> > > > from
> > > > > > >> API
> > > > > > >>>> but since both are async there is no progress indication.
> > > Neither
> > > > > the
> > > > > > >> JMX
> > > > > > >>>> nor the Web will help here since you don’t know if a
> > compaction
> > > > task
> > > > > > is
> > > > > > >>>> running. Tailling the logs is not an efficient way to do
> this
> > > > > neither.
> > > > > > >>> The
> > > > > > >>>> point is that I would like to automate the process and avoid
> > > > > > compaction
> > > > > > >>>> storm. So I want to do that region, region, but if I don’t
> > know
> > > > > when a
> > > > > > >>>> compaction started/ended I can’t automate it.
> > > > > > >>>>
> > > > > > >>>> 4.       In case there is no compaction files in queue (but
> > > still
> > > > > you
> > > > > > >>> have
> > > > > > >>>> more than 1 storefile per store e.g. minor compaction just
> > > > finished)
> > > > > > >> then
> > > > > > >>>> invoking major_compact will indeed decrease the number of
> > store
> > > > > files,
> > > > > > >>> but
> > > > > > >>>> the compaction queue will remain to 0 during the compaction
> > task
> > > > > > >>> (shouldn’t
> > > > > > >>>> the compaction queue increase by the number of file to
> compact
> > > and
> > > > > be
> > > > > > >>>> reduced when the task ended?)
> > > > > > >>>>
> > > > > > >>>>
> > > > > > >>>> 5.       I saw already HBASE-3965 for getting status of
> major
> > > > > > >> compaction,
> > > > > > >>>> nevertheless it has be removed from 0.92, is it possible to
> > put
> > > it
> > > > > > >> back?
> > > > > > >>>> Even sooner than 0.92?
> > > > > > >>>>
> > > > > > >>>> 6.       In case a compaction (major) is running it seems
> > there
> > > is
> > > > > no
> > > > > > >> way
> > > > > > >>>> to stop-it. Do you plan to add such feature?
> > > > > > >>>>
> > > > > > >>>> 7.       Do you plan to add functionality via JMX
> > > > (starting/stopping
> > > > > > >>>> compaction, splitting....)
> > > > > > >>>>
> > > > > > >>>> 8.       Finally there were some request for allowing custom
> > > > > > >> compaction,
> > > > > > >>>> part of this was given via the RegionObserver in HBASE-2001,
> > > > > > >> nevertheless
> > > > > > >>>> do you consider adding support for custom compaction
> > (providing
> > > > real
> > > > > > >>>> pluggable compaction stategy not just observer)?
> > > > > > >>>>
> > > > > > >>>>
> > > > > > >>>> Regards,
> > > > > > >>>> Mikael.S
> > > > > > >>>>
> > > > > > >>>>
> > > > > > >>>
> > > > > > >>>
> > > > > > >>> --
> > > > > > >>> Mikael.S
> > > > > > >>>
> > > > > > >>
> > > > > > >
> > > > > > >
> > > > > > >
> > > > > > > --
> > > > > > > Mikael.S
> > > > > >
> > > > >
> > > >
> > >
> >
>

Re: Major Compaction Concerns

Posted by Mikael Sitruk <mi...@gmail.com>.
Ted hi

1. thanks for pointing on  HBASE-3051, Compaction at the granularity of a
column-family, it seems promising

2. Regarding manual management of compaction - it is exactly what i tried
to do and found all the finding. *In short there is no way to disable major
compaction from running automatically* (point #1 in original email), should
a JIRA be opened?

3. I have opened the following ones
HBASE-5146  - Hbase Shell - allow setting config properties
HBASE-5147 - Compaction/Major compaction operation from shell/API/JMX
HBASE-5148 - Compaction property at the server level are not propagated at
the table level
HBASE-5149 - getConfiguration() implementation is misleading

Regards,
Mikael.S

On Sun, Jan 8, 2012 at 11:07 PM, Ted Yu <yu...@gmail.com> wrote:

> HBASE-3051, Compaction at the granularity of a column-family, is marked
> implemented by HBASE-3796
> <https://issues.apache.org/jira/browse/HBASE-3796>which is in 0.92
> (0.92 RC3 is coming out soon)
>
> Please see http://hbase.apache.org/book/regions.arch.html, 8.7.5.5 which
> refers to
>
> http://hbase.apache.org/book/important_configurations.html#managed.compactions
>
> Cheers
>
> On Sun, Jan 8, 2012 at 12:55 PM, Mikael Sitruk <mikael.sitruk@gmail.com
> >wrote:
>
> > Well I'm very interested to dig further. I can also tell that the number
> of
> > log is getting very high very fast and of course a flush is triggered
> > adding more store files. Very fast the high number of store files trigger
> > compaction and delay the flushing (default delay is 90000 ms).  The files
> > are small in size, major compaction is not needed but minor yes.
> > Nevertheless the code ignore the disabled automatic compaction and
> promotes
> > files to major compaction.
> > I think I need to play with both the log file size the compaction
> threshold
> > and the Max number of stores file. Do you have some recommendations?
> > Btw the compaction take about 1min 40 sec for a store size of 900MB +/-.
> Is
> > it normal?
> > One thing that does not help in this story is that I have 2 column
> families
> > and each RS manages 100 of regions each cf growth with differents speed.
> > Is there a version of hbase handling better such case (not flushing both
> cf
> > if not needed to)?
> >
> > I will review the release note of the versions you suggested and open
> > issues/enhancements we discuss.
> >
> > Thanks
> > Cheers.
> > On Jan 8, 2012 10:22 PM, "Ted Yu" <yu...@gmail.com> wrote:
> >
> > > Your request in first paragraph below deserves a JIRA.
> > >
> > > For 2.b I agree a bug should be filed.
> > >
> > > For major compaction, adding more logs on region server side should
> help
> > > you understand the situation better - assuming you have interest to dig
> > > further.
> > > Please upgrade to 0.90.5, or you can wait for 0.90.6 release which is
> > > slated for Jan. 19th.
> > >
> > > After upgrade, the logs and code would be more pertinent to the tip of
> > 0.90
> > > branch.
> > >
> > > Thanks for summarizing your findings.
> > >
> > > On Sun, Jan 8, 2012 at 12:04 PM, Mikael Sitruk <
> mikael.sitruk@gmail.com
> > > >wrote:
> > >
> > > > In fact I think that for 2.a the current implementation is
> misleading.
> > > > Creating a connection and getting the configuration from the
> connection
> > > > should return the configuration of the cluster.
> > > > Requesting the configuration used to build an object should return
> the
> > > > configuration set on the object
> > > > Additionally it should be a new method like getConfigurations(), or
> > > > getClusterConfigurations() returning a map of serverinfo and
> > > > configuration.  Another option is to add on the HRegionServer and
> > > HMaster a
> > > > method getConfiguration() returning the configuration object used by
> > the
> > > > RegionServer or Master
> > > >
> > > > Regarding 2.b yes I tried but it did not return the setting from the
> > > > cluster configuration (again server has non default configuration,
> > table
> > > > was not configured with specific values then cluster configuration
> > should
> > > > apply on the table object). So I see it as problematic.
> > > >
> > > > Mikael.s
> > > >  On Jan 8, 2012 7:54 PM, <yu...@gmail.com> wrote:
> > > >
> > > > > About 2b, have you tried getting the major compaction setting from
> > > column
> > > > > descriptor ?
> > > > >
> > > > > For 2a, what you requested would result in new methods of
> > > > > HBaseConfiguration class to be added. Currently the configuration
> on
> > > > client
> > > > > class path would be used.
> > > > >
> > > > > Cheers
> > > > >
> > > > >
> > > > >
> > > > > On Jan 8, 2012, at 9:28 AM, Mikael Sitruk <mikael.sitruk@gmail.com
> >
> > > > wrote:
> > > > >
> > > > > > Ted hi
> > > > > > First thanks for answering, regarding the JIRA i will fill them
> > > > > > Second, it seems that i did not explain myself correctly
> regarding
> > > > 2.a. -
> > > > > > As you i do not expect that a configuration set on my client will
> > be
> > > > > > propagated to the cluster, but i do expect that if i set a
> > > > configuration
> > > > > on
> > > > > > a server then doing connection.getConfiguration() from a client i
> > > will
> > > > > get
> > > > > > teh configuration from the cluster.
> > > > > > Currently the configuration returned is from the client config.
> > > > > > So the problem is that you have no way to check the configuration
> > of
> > > a
> > > > > > cluster.
> > > > > > I would expect to have some API to return the cluster config and
> > even
> > > > > > getting a map <serverInfo, config> so it can be easy to check
> > cluster
> > > > > > problem using code.
> > > > > >
> > > > > > 2.b. I know this code, and i tried to validate it. I set in the
> > > server
> > > > > > config the "hbase.hregion.majorcompaction" to "0", then start the
> > > > server
> > > > > > (cluster). Since from the UI or from JMX this parameter is not
> > > visible
> > > > at
> > > > > > the cluster level, I try to get the value from the client (to see
> > > that
> > > > > the
> > > > > > cluster is using it)
> > > > > >
> > > > > > *HTableDescriptor hTableDescriptor =
> > > > > > conn.getHTableDescriptor(Bytes.toBytes("my table"));*
> > > > > >
> > > > > > *hTableDescriptor.getValue("hbase.hregion.majorcompaction")*
> > > > > > but i still got 24h (and not the value set in the config)! that
> was
> > > my
> > > > > > problem from the beginning! ==> Using the config (on the server
> > side)
> > > > > will
> > > > > > not propagate into the table/column family
> > > > > >
> > > > > > Mikael.S
> > > > > >
> > > > > > On Sun, Jan 8, 2012 at 7:13 PM, Ted Yu <yu...@gmail.com>
> > wrote:
> > > > > >
> > > > > >> I am not expert in major compaction feature.
> > > > > >> Let me try to answer questions in #2.
> > > > > >>
> > > > > >> 2.a
> > > > > >>> If I set the property via the configuration shouldn’t all the
> > > cluster
> > > > > be
> > > > > >>> aware of?
> > > > > >>
> > > > > >> There're multiple clients connecting to one cluster. I wouldn't
> > > expect
> > > > > >> values in the configuration (m_hbConfig) to propagate onto the
> > > > cluster.
> > > > > >>
> > > > > >> 2.b
> > > > > >> Store.getNextMajorCompactTime() shows that
> > > > > "hbase.hregion.majorcompaction"
> > > > > >> can be specified per column family:
> > > > > >>
> > > > > >> long getNextMajorCompactTime() {
> > > > > >>   // default = 24hrs
> > > > > >>   long ret = conf.getLong(HConstants.MAJOR_COMPACTION_PERIOD,
> > > > > >> 1000*60*60*24);
> > > > > >>   if (family.getValue(HConstants.MAJOR_COMPACTION_PERIOD) !=
> > null) {
> > > > > >>
> > > > > >> 2.d
> > > > > >>> d. I tried also to setup the parameter via hbase shell but
> > setting
> > > > such
> > > > > >>> properties is not supported. (do you plan to add such support
> via
> > > the
> > > > > >>> shell?)
> > > > > >>
> > > > > >> This is a good idea. Please open a JIRA.
> > > > > >>
> > > > > >> For #5, HBASE-3965 is an improvement and doesn't have a patch
> yet.
> > > > > >>
> > > > > >> Allow me to quote Alan Kay: 'The best way to predict the future
> is
> > > to
> > > > > >> invent it.'
> > > > > >>
> > > > > >> Once we have a patch, we can always backport it to 0.92 after
> some
> > > > > people
> > > > > >> have verified the improvement.
> > > > > >>
> > > > > >>> 6.       In case a compaction (major) is running it seems there
> > is
> > > no
> > > > > way
> > > > > >>> to stop-it. Do you plan to add such feature?
> > > > > >>
> > > > > >> Again, logging a JIRA would provide a good starting point for
> > > > > discussion.
> > > > > >>
> > > > > >> Thanks for the verification work and suggestions, Mikael.
> > > > > >>
> > > > > >> On Sun, Jan 8, 2012 at 7:27 AM, Mikael Sitruk <
> > > > mikael.sitruk@gmail.com
> > > > > >>> wrote:
> > > > > >>
> > > > > >>> I forgot to mention, I'm using HBase 0.90.1
> > > > > >>>
> > > > > >>> Regards,
> > > > > >>> Mikael.S
> > > > > >>>
> > > > > >>> On Sun, Jan 8, 2012 at 5:25 PM, Mikael Sitruk <
> > > > mikael.sitruk@gmail.com
> > > > > >>>> wrote:
> > > > > >>>
> > > > > >>>> Hi
> > > > > >>>>
> > > > > >>>>
> > > > > >>>>
> > > > > >>>> I have some concern regarding major compactions below...
> > > > > >>>>
> > > > > >>>>
> > > > > >>>>   1. According to best practices from the mailing list and
> from
> > > the
> > > > > >>>>   book, automatic major compaction should be disabled. This
> can
> > be
> > > > > >> done
> > > > > >>> by
> > > > > >>>>   setting the property ‘hbase.hregion.majorcompaction’ to ‘0’.
> > > > > >>> Neverhteless
> > > > > >>>>   even after having doing this I STILL see “major compaction”
> > > > messages
> > > > > >>> in
> > > > > >>>>   logs. therefore it is unclear how can I manage major
> > > compactions.
> > > > > >> (The
> > > > > >>>>   system has heavy insert - uniformly on the cluster, and
> major
> > > > > >>> compaction
> > > > > >>>>   affect the performance of the system).
> > > > > >>>>   If I'm not wrong it seems from the code that: even if not
> > > > requested
> > > > > >>>>   and even if the indicator is set to '0' (no automatic major
> > > > > >>> compaction),
> > > > > >>>>   major compaction can be triggered by the code in case all
> > store
> > > > > >> files
> > > > > >>> are
> > > > > >>>>   candidate for a compaction (from Store.compact(final boolean
> > > > > >>> forceMajor)).
> > > > > >>>>   Shouldn't the code add a condition that automatic major
> > > compaction
> > > > > >> is
> > > > > >>>>   disabled??
> > > > > >>>>
> > > > > >>>>   2. I tried to check the parameter
> > >  ‘hbase.hregion.majorcompaction’
> > > > > >> at
> > > > > >>>>   runtime using several approaches - to validate that the
> server
> > > > > >> indeed
> > > > > >>>>   loaded the parameter.
> > > > > >>>>
> > > > > >>>> a. Using a connection created from local config
> > > > > >>>>
> > > > > >>>> *conn = (HConnection)
> > > HConnectionManager.getConnection(m_hbConfig);*
> > > > > >>>>
> > > > > >>>>
> > > *conn.getConfiguration().getString(“hbase.hregion.majorcompaction”)*
> > > > > >>>>
> > > > > >>>> returns the parameter from local config and not from cluster.
> Is
> > > it
> > > > a
> > > > > >>> bug?
> > > > > >>>> If I set the property via the configuration shouldn’t all the
> > > > cluster
> > > > > >> be
> > > > > >>>> aware of? (supposing that the connection indeed connected to
> the
> > > > > >> cluster)
> > > > > >>>>
> > > > > >>>> b.  fetching the property from the table descriptor
> > > > > >>>>
> > > > > >>>> *HTableDescriptor hTableDescriptor =
> > > > > >>>> conn.getHTableDescriptor(Bytes.toBytes("my table"));*
> > > > > >>>>
> > > > > >>>> *hTableDescriptor.getValue("hbase.hregion.majorcompaction")*
> > > > > >>>>
> > > > > >>>> This will returns the default parameter value (1 day) not the
> > > > > parameter
> > > > > >>>> from the configuration (on the cluster). It seems to be a bug,
> > > isn’t
> > > > > >> it?
> > > > > >>>> (the parameter from the config, should be the default if not
> set
> > > at
> > > > > the
> > > > > >>>> table level)
> > > > > >>>>
> > > > > >>>> c. The only way I could set the parameter to 0 and really see
> it
> > > is
> > > > > via
> > > > > >>>> the Admin API, updating the table descriptor or the column
> > > > descriptor.
> > > > > >>> Now
> > > > > >>>> I could see the parameter on the web UI. So is it the only way
> > to
> > > > set
> > > > > >>>> correctly the parameter? If setting the parameter via the
> > > > > configuration
> > > > > >>>> file, shouldn’t the webUI show this on any table created?
> > > > > >>>>
> > > > > >>>> d. I tried also to setup the parameter via hbase shell but
> > setting
> > > > > such
> > > > > >>>> properties is not supported. (do you plan to add such support
> > via
> > > > the
> > > > > >>>> shell?)
> > > > > >>>>
> > > > > >>>> e. Generally is it possible to get via API the configuration
> > used
> > > by
> > > > > >> the
> > > > > >>>> servers? (at cluster/server level)
> > > > > >>>>
> > > > > >>>>    3.  I ran both major compaction  requests from the shell or
> > > from
> > > > > >> API
> > > > > >>>> but since both are async there is no progress indication.
> > Neither
> > > > the
> > > > > >> JMX
> > > > > >>>> nor the Web will help here since you don’t know if a
> compaction
> > > task
> > > > > is
> > > > > >>>> running. Tailling the logs is not an efficient way to do this
> > > > neither.
> > > > > >>> The
> > > > > >>>> point is that I would like to automate the process and avoid
> > > > > compaction
> > > > > >>>> storm. So I want to do that region, region, but if I don’t
> know
> > > > when a
> > > > > >>>> compaction started/ended I can’t automate it.
> > > > > >>>>
> > > > > >>>> 4.       In case there is no compaction files in queue (but
> > still
> > > > you
> > > > > >>> have
> > > > > >>>> more than 1 storefile per store e.g. minor compaction just
> > > finished)
> > > > > >> then
> > > > > >>>> invoking major_compact will indeed decrease the number of
> store
> > > > files,
> > > > > >>> but
> > > > > >>>> the compaction queue will remain to 0 during the compaction
> task
> > > > > >>> (shouldn’t
> > > > > >>>> the compaction queue increase by the number of file to compact
> > and
> > > > be
> > > > > >>>> reduced when the task ended?)
> > > > > >>>>
> > > > > >>>>
> > > > > >>>> 5.       I saw already HBASE-3965 for getting status of major
> > > > > >> compaction,
> > > > > >>>> nevertheless it has be removed from 0.92, is it possible to
> put
> > it
> > > > > >> back?
> > > > > >>>> Even sooner than 0.92?
> > > > > >>>>
> > > > > >>>> 6.       In case a compaction (major) is running it seems
> there
> > is
> > > > no
> > > > > >> way
> > > > > >>>> to stop-it. Do you plan to add such feature?
> > > > > >>>>
> > > > > >>>> 7.       Do you plan to add functionality via JMX
> > > (starting/stopping
> > > > > >>>> compaction, splitting....)
> > > > > >>>>
> > > > > >>>> 8.       Finally there were some request for allowing custom
> > > > > >> compaction,
> > > > > >>>> part of this was given via the RegionObserver in HBASE-2001,
> > > > > >> nevertheless
> > > > > >>>> do you consider adding support for custom compaction
> (providing
> > > real
> > > > > >>>> pluggable compaction stategy not just observer)?
> > > > > >>>>
> > > > > >>>>
> > > > > >>>> Regards,
> > > > > >>>> Mikael.S
> > > > > >>>>
> > > > > >>>>
> > > > > >>>
> > > > > >>>
> > > > > >>> --
> > > > > >>> Mikael.S
> > > > > >>>
> > > > > >>
> > > > > >
> > > > > >
> > > > > >
> > > > > > --
> > > > > > Mikael.S
> > > > >
> > > >
> > >
> >
>

Re: Major Compaction Concerns

Posted by Ted Yu <yu...@gmail.com>.
HBASE-3051, Compaction at the granularity of a column-family, is marked
implemented by HBASE-3796
<https://issues.apache.org/jira/browse/HBASE-3796>which is in 0.92
(0.92 RC3 is coming out soon)

Please see http://hbase.apache.org/book/regions.arch.html, 8.7.5.5 which
refers to
http://hbase.apache.org/book/important_configurations.html#managed.compactions

Cheers

On Sun, Jan 8, 2012 at 12:55 PM, Mikael Sitruk <mi...@gmail.com>wrote:

> Well I'm very interested to dig further. I can also tell that the number of
> log is getting very high very fast and of course a flush is triggered
> adding more store files. Very fast the high number of store files trigger
> compaction and delay the flushing (default delay is 90000 ms).  The files
> are small in size, major compaction is not needed but minor yes.
> Nevertheless the code ignore the disabled automatic compaction and promotes
> files to major compaction.
> I think I need to play with both the log file size the compaction threshold
> and the Max number of stores file. Do you have some recommendations?
> Btw the compaction take about 1min 40 sec for a store size of 900MB +/-. Is
> it normal?
> One thing that does not help in this story is that I have 2 column families
> and each RS manages 100 of regions each cf growth with differents speed.
> Is there a version of hbase handling better such case (not flushing both cf
> if not needed to)?
>
> I will review the release note of the versions you suggested and open
> issues/enhancements we discuss.
>
> Thanks
> Cheers.
> On Jan 8, 2012 10:22 PM, "Ted Yu" <yu...@gmail.com> wrote:
>
> > Your request in first paragraph below deserves a JIRA.
> >
> > For 2.b I agree a bug should be filed.
> >
> > For major compaction, adding more logs on region server side should help
> > you understand the situation better - assuming you have interest to dig
> > further.
> > Please upgrade to 0.90.5, or you can wait for 0.90.6 release which is
> > slated for Jan. 19th.
> >
> > After upgrade, the logs and code would be more pertinent to the tip of
> 0.90
> > branch.
> >
> > Thanks for summarizing your findings.
> >
> > On Sun, Jan 8, 2012 at 12:04 PM, Mikael Sitruk <mikael.sitruk@gmail.com
> > >wrote:
> >
> > > In fact I think that for 2.a the current implementation is misleading.
> > > Creating a connection and getting the configuration from the connection
> > > should return the configuration of the cluster.
> > > Requesting the configuration used to build an object should return the
> > > configuration set on the object
> > > Additionally it should be a new method like getConfigurations(), or
> > > getClusterConfigurations() returning a map of serverinfo and
> > > configuration.  Another option is to add on the HRegionServer and
> > HMaster a
> > > method getConfiguration() returning the configuration object used by
> the
> > > RegionServer or Master
> > >
> > > Regarding 2.b yes I tried but it did not return the setting from the
> > > cluster configuration (again server has non default configuration,
> table
> > > was not configured with specific values then cluster configuration
> should
> > > apply on the table object). So I see it as problematic.
> > >
> > > Mikael.s
> > >  On Jan 8, 2012 7:54 PM, <yu...@gmail.com> wrote:
> > >
> > > > About 2b, have you tried getting the major compaction setting from
> > column
> > > > descriptor ?
> > > >
> > > > For 2a, what you requested would result in new methods of
> > > > HBaseConfiguration class to be added. Currently the configuration on
> > > client
> > > > class path would be used.
> > > >
> > > > Cheers
> > > >
> > > >
> > > >
> > > > On Jan 8, 2012, at 9:28 AM, Mikael Sitruk <mi...@gmail.com>
> > > wrote:
> > > >
> > > > > Ted hi
> > > > > First thanks for answering, regarding the JIRA i will fill them
> > > > > Second, it seems that i did not explain myself correctly regarding
> > > 2.a. -
> > > > > As you i do not expect that a configuration set on my client will
> be
> > > > > propagated to the cluster, but i do expect that if i set a
> > > configuration
> > > > on
> > > > > a server then doing connection.getConfiguration() from a client i
> > will
> > > > get
> > > > > teh configuration from the cluster.
> > > > > Currently the configuration returned is from the client config.
> > > > > So the problem is that you have no way to check the configuration
> of
> > a
> > > > > cluster.
> > > > > I would expect to have some API to return the cluster config and
> even
> > > > > getting a map <serverInfo, config> so it can be easy to check
> cluster
> > > > > problem using code.
> > > > >
> > > > > 2.b. I know this code, and i tried to validate it. I set in the
> > server
> > > > > config the "hbase.hregion.majorcompaction" to "0", then start the
> > > server
> > > > > (cluster). Since from the UI or from JMX this parameter is not
> > visible
> > > at
> > > > > the cluster level, I try to get the value from the client (to see
> > that
> > > > the
> > > > > cluster is using it)
> > > > >
> > > > > *HTableDescriptor hTableDescriptor =
> > > > > conn.getHTableDescriptor(Bytes.toBytes("my table"));*
> > > > >
> > > > > *hTableDescriptor.getValue("hbase.hregion.majorcompaction")*
> > > > > but i still got 24h (and not the value set in the config)! that was
> > my
> > > > > problem from the beginning! ==> Using the config (on the server
> side)
> > > > will
> > > > > not propagate into the table/column family
> > > > >
> > > > > Mikael.S
> > > > >
> > > > > On Sun, Jan 8, 2012 at 7:13 PM, Ted Yu <yu...@gmail.com>
> wrote:
> > > > >
> > > > >> I am not expert in major compaction feature.
> > > > >> Let me try to answer questions in #2.
> > > > >>
> > > > >> 2.a
> > > > >>> If I set the property via the configuration shouldn’t all the
> > cluster
> > > > be
> > > > >>> aware of?
> > > > >>
> > > > >> There're multiple clients connecting to one cluster. I wouldn't
> > expect
> > > > >> values in the configuration (m_hbConfig) to propagate onto the
> > > cluster.
> > > > >>
> > > > >> 2.b
> > > > >> Store.getNextMajorCompactTime() shows that
> > > > "hbase.hregion.majorcompaction"
> > > > >> can be specified per column family:
> > > > >>
> > > > >> long getNextMajorCompactTime() {
> > > > >>   // default = 24hrs
> > > > >>   long ret = conf.getLong(HConstants.MAJOR_COMPACTION_PERIOD,
> > > > >> 1000*60*60*24);
> > > > >>   if (family.getValue(HConstants.MAJOR_COMPACTION_PERIOD) !=
> null) {
> > > > >>
> > > > >> 2.d
> > > > >>> d. I tried also to setup the parameter via hbase shell but
> setting
> > > such
> > > > >>> properties is not supported. (do you plan to add such support via
> > the
> > > > >>> shell?)
> > > > >>
> > > > >> This is a good idea. Please open a JIRA.
> > > > >>
> > > > >> For #5, HBASE-3965 is an improvement and doesn't have a patch yet.
> > > > >>
> > > > >> Allow me to quote Alan Kay: 'The best way to predict the future is
> > to
> > > > >> invent it.'
> > > > >>
> > > > >> Once we have a patch, we can always backport it to 0.92 after some
> > > > people
> > > > >> have verified the improvement.
> > > > >>
> > > > >>> 6.       In case a compaction (major) is running it seems there
> is
> > no
> > > > way
> > > > >>> to stop-it. Do you plan to add such feature?
> > > > >>
> > > > >> Again, logging a JIRA would provide a good starting point for
> > > > discussion.
> > > > >>
> > > > >> Thanks for the verification work and suggestions, Mikael.
> > > > >>
> > > > >> On Sun, Jan 8, 2012 at 7:27 AM, Mikael Sitruk <
> > > mikael.sitruk@gmail.com
> > > > >>> wrote:
> > > > >>
> > > > >>> I forgot to mention, I'm using HBase 0.90.1
> > > > >>>
> > > > >>> Regards,
> > > > >>> Mikael.S
> > > > >>>
> > > > >>> On Sun, Jan 8, 2012 at 5:25 PM, Mikael Sitruk <
> > > mikael.sitruk@gmail.com
> > > > >>>> wrote:
> > > > >>>
> > > > >>>> Hi
> > > > >>>>
> > > > >>>>
> > > > >>>>
> > > > >>>> I have some concern regarding major compactions below...
> > > > >>>>
> > > > >>>>
> > > > >>>>   1. According to best practices from the mailing list and from
> > the
> > > > >>>>   book, automatic major compaction should be disabled. This can
> be
> > > > >> done
> > > > >>> by
> > > > >>>>   setting the property ‘hbase.hregion.majorcompaction’ to ‘0’.
> > > > >>> Neverhteless
> > > > >>>>   even after having doing this I STILL see “major compaction”
> > > messages
> > > > >>> in
> > > > >>>>   logs. therefore it is unclear how can I manage major
> > compactions.
> > > > >> (The
> > > > >>>>   system has heavy insert - uniformly on the cluster, and major
> > > > >>> compaction
> > > > >>>>   affect the performance of the system).
> > > > >>>>   If I'm not wrong it seems from the code that: even if not
> > > requested
> > > > >>>>   and even if the indicator is set to '0' (no automatic major
> > > > >>> compaction),
> > > > >>>>   major compaction can be triggered by the code in case all
> store
> > > > >> files
> > > > >>> are
> > > > >>>>   candidate for a compaction (from Store.compact(final boolean
> > > > >>> forceMajor)).
> > > > >>>>   Shouldn't the code add a condition that automatic major
> > compaction
> > > > >> is
> > > > >>>>   disabled??
> > > > >>>>
> > > > >>>>   2. I tried to check the parameter
> >  ‘hbase.hregion.majorcompaction’
> > > > >> at
> > > > >>>>   runtime using several approaches - to validate that the server
> > > > >> indeed
> > > > >>>>   loaded the parameter.
> > > > >>>>
> > > > >>>> a. Using a connection created from local config
> > > > >>>>
> > > > >>>> *conn = (HConnection)
> > HConnectionManager.getConnection(m_hbConfig);*
> > > > >>>>
> > > > >>>>
> > *conn.getConfiguration().getString(“hbase.hregion.majorcompaction”)*
> > > > >>>>
> > > > >>>> returns the parameter from local config and not from cluster. Is
> > it
> > > a
> > > > >>> bug?
> > > > >>>> If I set the property via the configuration shouldn’t all the
> > > cluster
> > > > >> be
> > > > >>>> aware of? (supposing that the connection indeed connected to the
> > > > >> cluster)
> > > > >>>>
> > > > >>>> b.  fetching the property from the table descriptor
> > > > >>>>
> > > > >>>> *HTableDescriptor hTableDescriptor =
> > > > >>>> conn.getHTableDescriptor(Bytes.toBytes("my table"));*
> > > > >>>>
> > > > >>>> *hTableDescriptor.getValue("hbase.hregion.majorcompaction")*
> > > > >>>>
> > > > >>>> This will returns the default parameter value (1 day) not the
> > > > parameter
> > > > >>>> from the configuration (on the cluster). It seems to be a bug,
> > isn’t
> > > > >> it?
> > > > >>>> (the parameter from the config, should be the default if not set
> > at
> > > > the
> > > > >>>> table level)
> > > > >>>>
> > > > >>>> c. The only way I could set the parameter to 0 and really see it
> > is
> > > > via
> > > > >>>> the Admin API, updating the table descriptor or the column
> > > descriptor.
> > > > >>> Now
> > > > >>>> I could see the parameter on the web UI. So is it the only way
> to
> > > set
> > > > >>>> correctly the parameter? If setting the parameter via the
> > > > configuration
> > > > >>>> file, shouldn’t the webUI show this on any table created?
> > > > >>>>
> > > > >>>> d. I tried also to setup the parameter via hbase shell but
> setting
> > > > such
> > > > >>>> properties is not supported. (do you plan to add such support
> via
> > > the
> > > > >>>> shell?)
> > > > >>>>
> > > > >>>> e. Generally is it possible to get via API the configuration
> used
> > by
> > > > >> the
> > > > >>>> servers? (at cluster/server level)
> > > > >>>>
> > > > >>>>    3.  I ran both major compaction  requests from the shell or
> > from
> > > > >> API
> > > > >>>> but since both are async there is no progress indication.
> Neither
> > > the
> > > > >> JMX
> > > > >>>> nor the Web will help here since you don’t know if a compaction
> > task
> > > > is
> > > > >>>> running. Tailling the logs is not an efficient way to do this
> > > neither.
> > > > >>> The
> > > > >>>> point is that I would like to automate the process and avoid
> > > > compaction
> > > > >>>> storm. So I want to do that region, region, but if I don’t know
> > > when a
> > > > >>>> compaction started/ended I can’t automate it.
> > > > >>>>
> > > > >>>> 4.       In case there is no compaction files in queue (but
> still
> > > you
> > > > >>> have
> > > > >>>> more than 1 storefile per store e.g. minor compaction just
> > finished)
> > > > >> then
> > > > >>>> invoking major_compact will indeed decrease the number of store
> > > files,
> > > > >>> but
> > > > >>>> the compaction queue will remain to 0 during the compaction task
> > > > >>> (shouldn’t
> > > > >>>> the compaction queue increase by the number of file to compact
> and
> > > be
> > > > >>>> reduced when the task ended?)
> > > > >>>>
> > > > >>>>
> > > > >>>> 5.       I saw already HBASE-3965 for getting status of major
> > > > >> compaction,
> > > > >>>> nevertheless it has be removed from 0.92, is it possible to put
> it
> > > > >> back?
> > > > >>>> Even sooner than 0.92?
> > > > >>>>
> > > > >>>> 6.       In case a compaction (major) is running it seems there
> is
> > > no
> > > > >> way
> > > > >>>> to stop-it. Do you plan to add such feature?
> > > > >>>>
> > > > >>>> 7.       Do you plan to add functionality via JMX
> > (starting/stopping
> > > > >>>> compaction, splitting....)
> > > > >>>>
> > > > >>>> 8.       Finally there were some request for allowing custom
> > > > >> compaction,
> > > > >>>> part of this was given via the RegionObserver in HBASE-2001,
> > > > >> nevertheless
> > > > >>>> do you consider adding support for custom compaction (providing
> > real
> > > > >>>> pluggable compaction stategy not just observer)?
> > > > >>>>
> > > > >>>>
> > > > >>>> Regards,
> > > > >>>> Mikael.S
> > > > >>>>
> > > > >>>>
> > > > >>>
> > > > >>>
> > > > >>> --
> > > > >>> Mikael.S
> > > > >>>
> > > > >>
> > > > >
> > > > >
> > > > >
> > > > > --
> > > > > Mikael.S
> > > >
> > >
> >
>

Re: Major Compaction Concerns

Posted by Mikael Sitruk <mi...@gmail.com>.
Well I'm very interested to dig further. I can also tell that the number of
log is getting very high very fast and of course a flush is triggered
adding more store files. Very fast the high number of store files trigger
compaction and delay the flushing (default delay is 90000 ms).  The files
are small in size, major compaction is not needed but minor yes.
Nevertheless the code ignore the disabled automatic compaction and promotes
files to major compaction.
I think I need to play with both the log file size the compaction threshold
and the Max number of stores file. Do you have some recommendations?
Btw the compaction take about 1min 40 sec for a store size of 900MB +/-. Is
it normal?
One thing that does not help in this story is that I have 2 column families
and each RS manages 100 of regions each cf growth with differents speed.
Is there a version of hbase handling better such case (not flushing both cf
if not needed to)?

I will review the release note of the versions you suggested and open
issues/enhancements we discuss.

Thanks
Cheers.
On Jan 8, 2012 10:22 PM, "Ted Yu" <yu...@gmail.com> wrote:

> Your request in first paragraph below deserves a JIRA.
>
> For 2.b I agree a bug should be filed.
>
> For major compaction, adding more logs on region server side should help
> you understand the situation better - assuming you have interest to dig
> further.
> Please upgrade to 0.90.5, or you can wait for 0.90.6 release which is
> slated for Jan. 19th.
>
> After upgrade, the logs and code would be more pertinent to the tip of 0.90
> branch.
>
> Thanks for summarizing your findings.
>
> On Sun, Jan 8, 2012 at 12:04 PM, Mikael Sitruk <mikael.sitruk@gmail.com
> >wrote:
>
> > In fact I think that for 2.a the current implementation is misleading.
> > Creating a connection and getting the configuration from the connection
> > should return the configuration of the cluster.
> > Requesting the configuration used to build an object should return the
> > configuration set on the object
> > Additionally it should be a new method like getConfigurations(), or
> > getClusterConfigurations() returning a map of serverinfo and
> > configuration.  Another option is to add on the HRegionServer and
> HMaster a
> > method getConfiguration() returning the configuration object used by the
> > RegionServer or Master
> >
> > Regarding 2.b yes I tried but it did not return the setting from the
> > cluster configuration (again server has non default configuration, table
> > was not configured with specific values then cluster configuration should
> > apply on the table object). So I see it as problematic.
> >
> > Mikael.s
> >  On Jan 8, 2012 7:54 PM, <yu...@gmail.com> wrote:
> >
> > > About 2b, have you tried getting the major compaction setting from
> column
> > > descriptor ?
> > >
> > > For 2a, what you requested would result in new methods of
> > > HBaseConfiguration class to be added. Currently the configuration on
> > client
> > > class path would be used.
> > >
> > > Cheers
> > >
> > >
> > >
> > > On Jan 8, 2012, at 9:28 AM, Mikael Sitruk <mi...@gmail.com>
> > wrote:
> > >
> > > > Ted hi
> > > > First thanks for answering, regarding the JIRA i will fill them
> > > > Second, it seems that i did not explain myself correctly regarding
> > 2.a. -
> > > > As you i do not expect that a configuration set on my client will be
> > > > propagated to the cluster, but i do expect that if i set a
> > configuration
> > > on
> > > > a server then doing connection.getConfiguration() from a client i
> will
> > > get
> > > > teh configuration from the cluster.
> > > > Currently the configuration returned is from the client config.
> > > > So the problem is that you have no way to check the configuration of
> a
> > > > cluster.
> > > > I would expect to have some API to return the cluster config and even
> > > > getting a map <serverInfo, config> so it can be easy to check cluster
> > > > problem using code.
> > > >
> > > > 2.b. I know this code, and i tried to validate it. I set in the
> server
> > > > config the "hbase.hregion.majorcompaction" to "0", then start the
> > server
> > > > (cluster). Since from the UI or from JMX this parameter is not
> visible
> > at
> > > > the cluster level, I try to get the value from the client (to see
> that
> > > the
> > > > cluster is using it)
> > > >
> > > > *HTableDescriptor hTableDescriptor =
> > > > conn.getHTableDescriptor(Bytes.toBytes("my table"));*
> > > >
> > > > *hTableDescriptor.getValue("hbase.hregion.majorcompaction")*
> > > > but i still got 24h (and not the value set in the config)! that was
> my
> > > > problem from the beginning! ==> Using the config (on the server side)
> > > will
> > > > not propagate into the table/column family
> > > >
> > > > Mikael.S
> > > >
> > > > On Sun, Jan 8, 2012 at 7:13 PM, Ted Yu <yu...@gmail.com> wrote:
> > > >
> > > >> I am not expert in major compaction feature.
> > > >> Let me try to answer questions in #2.
> > > >>
> > > >> 2.a
> > > >>> If I set the property via the configuration shouldn’t all the
> cluster
> > > be
> > > >>> aware of?
> > > >>
> > > >> There're multiple clients connecting to one cluster. I wouldn't
> expect
> > > >> values in the configuration (m_hbConfig) to propagate onto the
> > cluster.
> > > >>
> > > >> 2.b
> > > >> Store.getNextMajorCompactTime() shows that
> > > "hbase.hregion.majorcompaction"
> > > >> can be specified per column family:
> > > >>
> > > >> long getNextMajorCompactTime() {
> > > >>   // default = 24hrs
> > > >>   long ret = conf.getLong(HConstants.MAJOR_COMPACTION_PERIOD,
> > > >> 1000*60*60*24);
> > > >>   if (family.getValue(HConstants.MAJOR_COMPACTION_PERIOD) != null) {
> > > >>
> > > >> 2.d
> > > >>> d. I tried also to setup the parameter via hbase shell but setting
> > such
> > > >>> properties is not supported. (do you plan to add such support via
> the
> > > >>> shell?)
> > > >>
> > > >> This is a good idea. Please open a JIRA.
> > > >>
> > > >> For #5, HBASE-3965 is an improvement and doesn't have a patch yet.
> > > >>
> > > >> Allow me to quote Alan Kay: 'The best way to predict the future is
> to
> > > >> invent it.'
> > > >>
> > > >> Once we have a patch, we can always backport it to 0.92 after some
> > > people
> > > >> have verified the improvement.
> > > >>
> > > >>> 6.       In case a compaction (major) is running it seems there is
> no
> > > way
> > > >>> to stop-it. Do you plan to add such feature?
> > > >>
> > > >> Again, logging a JIRA would provide a good starting point for
> > > discussion.
> > > >>
> > > >> Thanks for the verification work and suggestions, Mikael.
> > > >>
> > > >> On Sun, Jan 8, 2012 at 7:27 AM, Mikael Sitruk <
> > mikael.sitruk@gmail.com
> > > >>> wrote:
> > > >>
> > > >>> I forgot to mention, I'm using HBase 0.90.1
> > > >>>
> > > >>> Regards,
> > > >>> Mikael.S
> > > >>>
> > > >>> On Sun, Jan 8, 2012 at 5:25 PM, Mikael Sitruk <
> > mikael.sitruk@gmail.com
> > > >>>> wrote:
> > > >>>
> > > >>>> Hi
> > > >>>>
> > > >>>>
> > > >>>>
> > > >>>> I have some concern regarding major compactions below...
> > > >>>>
> > > >>>>
> > > >>>>   1. According to best practices from the mailing list and from
> the
> > > >>>>   book, automatic major compaction should be disabled. This can be
> > > >> done
> > > >>> by
> > > >>>>   setting the property ‘hbase.hregion.majorcompaction’ to ‘0’.
> > > >>> Neverhteless
> > > >>>>   even after having doing this I STILL see “major compaction”
> > messages
> > > >>> in
> > > >>>>   logs. therefore it is unclear how can I manage major
> compactions.
> > > >> (The
> > > >>>>   system has heavy insert - uniformly on the cluster, and major
> > > >>> compaction
> > > >>>>   affect the performance of the system).
> > > >>>>   If I'm not wrong it seems from the code that: even if not
> > requested
> > > >>>>   and even if the indicator is set to '0' (no automatic major
> > > >>> compaction),
> > > >>>>   major compaction can be triggered by the code in case all store
> > > >> files
> > > >>> are
> > > >>>>   candidate for a compaction (from Store.compact(final boolean
> > > >>> forceMajor)).
> > > >>>>   Shouldn't the code add a condition that automatic major
> compaction
> > > >> is
> > > >>>>   disabled??
> > > >>>>
> > > >>>>   2. I tried to check the parameter
>  ‘hbase.hregion.majorcompaction’
> > > >> at
> > > >>>>   runtime using several approaches - to validate that the server
> > > >> indeed
> > > >>>>   loaded the parameter.
> > > >>>>
> > > >>>> a. Using a connection created from local config
> > > >>>>
> > > >>>> *conn = (HConnection)
> HConnectionManager.getConnection(m_hbConfig);*
> > > >>>>
> > > >>>>
> *conn.getConfiguration().getString(“hbase.hregion.majorcompaction”)*
> > > >>>>
> > > >>>> returns the parameter from local config and not from cluster. Is
> it
> > a
> > > >>> bug?
> > > >>>> If I set the property via the configuration shouldn’t all the
> > cluster
> > > >> be
> > > >>>> aware of? (supposing that the connection indeed connected to the
> > > >> cluster)
> > > >>>>
> > > >>>> b.  fetching the property from the table descriptor
> > > >>>>
> > > >>>> *HTableDescriptor hTableDescriptor =
> > > >>>> conn.getHTableDescriptor(Bytes.toBytes("my table"));*
> > > >>>>
> > > >>>> *hTableDescriptor.getValue("hbase.hregion.majorcompaction")*
> > > >>>>
> > > >>>> This will returns the default parameter value (1 day) not the
> > > parameter
> > > >>>> from the configuration (on the cluster). It seems to be a bug,
> isn’t
> > > >> it?
> > > >>>> (the parameter from the config, should be the default if not set
> at
> > > the
> > > >>>> table level)
> > > >>>>
> > > >>>> c. The only way I could set the parameter to 0 and really see it
> is
> > > via
> > > >>>> the Admin API, updating the table descriptor or the column
> > descriptor.
> > > >>> Now
> > > >>>> I could see the parameter on the web UI. So is it the only way to
> > set
> > > >>>> correctly the parameter? If setting the parameter via the
> > > configuration
> > > >>>> file, shouldn’t the webUI show this on any table created?
> > > >>>>
> > > >>>> d. I tried also to setup the parameter via hbase shell but setting
> > > such
> > > >>>> properties is not supported. (do you plan to add such support via
> > the
> > > >>>> shell?)
> > > >>>>
> > > >>>> e. Generally is it possible to get via API the configuration used
> by
> > > >> the
> > > >>>> servers? (at cluster/server level)
> > > >>>>
> > > >>>>    3.  I ran both major compaction  requests from the shell or
> from
> > > >> API
> > > >>>> but since both are async there is no progress indication. Neither
> > the
> > > >> JMX
> > > >>>> nor the Web will help here since you don’t know if a compaction
> task
> > > is
> > > >>>> running. Tailling the logs is not an efficient way to do this
> > neither.
> > > >>> The
> > > >>>> point is that I would like to automate the process and avoid
> > > compaction
> > > >>>> storm. So I want to do that region, region, but if I don’t know
> > when a
> > > >>>> compaction started/ended I can’t automate it.
> > > >>>>
> > > >>>> 4.       In case there is no compaction files in queue (but still
> > you
> > > >>> have
> > > >>>> more than 1 storefile per store e.g. minor compaction just
> finished)
> > > >> then
> > > >>>> invoking major_compact will indeed decrease the number of store
> > files,
> > > >>> but
> > > >>>> the compaction queue will remain to 0 during the compaction task
> > > >>> (shouldn’t
> > > >>>> the compaction queue increase by the number of file to compact and
> > be
> > > >>>> reduced when the task ended?)
> > > >>>>
> > > >>>>
> > > >>>> 5.       I saw already HBASE-3965 for getting status of major
> > > >> compaction,
> > > >>>> nevertheless it has be removed from 0.92, is it possible to put it
> > > >> back?
> > > >>>> Even sooner than 0.92?
> > > >>>>
> > > >>>> 6.       In case a compaction (major) is running it seems there is
> > no
> > > >> way
> > > >>>> to stop-it. Do you plan to add such feature?
> > > >>>>
> > > >>>> 7.       Do you plan to add functionality via JMX
> (starting/stopping
> > > >>>> compaction, splitting....)
> > > >>>>
> > > >>>> 8.       Finally there were some request for allowing custom
> > > >> compaction,
> > > >>>> part of this was given via the RegionObserver in HBASE-2001,
> > > >> nevertheless
> > > >>>> do you consider adding support for custom compaction (providing
> real
> > > >>>> pluggable compaction stategy not just observer)?
> > > >>>>
> > > >>>>
> > > >>>> Regards,
> > > >>>> Mikael.S
> > > >>>>
> > > >>>>
> > > >>>
> > > >>>
> > > >>> --
> > > >>> Mikael.S
> > > >>>
> > > >>
> > > >
> > > >
> > > >
> > > > --
> > > > Mikael.S
> > >
> >
>

Re: Major Compaction Concerns

Posted by Ted Yu <yu...@gmail.com>.
Your request in first paragraph below deserves a JIRA.

For 2.b I agree a bug should be filed.

For major compaction, adding more logs on region server side should help
you understand the situation better - assuming you have interest to dig
further.
Please upgrade to 0.90.5, or you can wait for 0.90.6 release which is
slated for Jan. 19th.

After upgrade, the logs and code would be more pertinent to the tip of 0.90
branch.

Thanks for summarizing your findings.

On Sun, Jan 8, 2012 at 12:04 PM, Mikael Sitruk <mi...@gmail.com>wrote:

> In fact I think that for 2.a the current implementation is misleading.
> Creating a connection and getting the configuration from the connection
> should return the configuration of the cluster.
> Requesting the configuration used to build an object should return the
> configuration set on the object
> Additionally it should be a new method like getConfigurations(), or
> getClusterConfigurations() returning a map of serverinfo and
> configuration.  Another option is to add on the HRegionServer and HMaster a
> method getConfiguration() returning the configuration object used by the
> RegionServer or Master
>
> Regarding 2.b yes I tried but it did not return the setting from the
> cluster configuration (again server has non default configuration, table
> was not configured with specific values then cluster configuration should
> apply on the table object). So I see it as problematic.
>
> Mikael.s
>  On Jan 8, 2012 7:54 PM, <yu...@gmail.com> wrote:
>
> > About 2b, have you tried getting the major compaction setting from column
> > descriptor ?
> >
> > For 2a, what you requested would result in new methods of
> > HBaseConfiguration class to be added. Currently the configuration on
> client
> > class path would be used.
> >
> > Cheers
> >
> >
> >
> > On Jan 8, 2012, at 9:28 AM, Mikael Sitruk <mi...@gmail.com>
> wrote:
> >
> > > Ted hi
> > > First thanks for answering, regarding the JIRA i will fill them
> > > Second, it seems that i did not explain myself correctly regarding
> 2.a. -
> > > As you i do not expect that a configuration set on my client will be
> > > propagated to the cluster, but i do expect that if i set a
> configuration
> > on
> > > a server then doing connection.getConfiguration() from a client i will
> > get
> > > teh configuration from the cluster.
> > > Currently the configuration returned is from the client config.
> > > So the problem is that you have no way to check the configuration of a
> > > cluster.
> > > I would expect to have some API to return the cluster config and even
> > > getting a map <serverInfo, config> so it can be easy to check cluster
> > > problem using code.
> > >
> > > 2.b. I know this code, and i tried to validate it. I set in the server
> > > config the "hbase.hregion.majorcompaction" to "0", then start the
> server
> > > (cluster). Since from the UI or from JMX this parameter is not visible
> at
> > > the cluster level, I try to get the value from the client (to see that
> > the
> > > cluster is using it)
> > >
> > > *HTableDescriptor hTableDescriptor =
> > > conn.getHTableDescriptor(Bytes.toBytes("my table"));*
> > >
> > > *hTableDescriptor.getValue("hbase.hregion.majorcompaction")*
> > > but i still got 24h (and not the value set in the config)! that was my
> > > problem from the beginning! ==> Using the config (on the server side)
> > will
> > > not propagate into the table/column family
> > >
> > > Mikael.S
> > >
> > > On Sun, Jan 8, 2012 at 7:13 PM, Ted Yu <yu...@gmail.com> wrote:
> > >
> > >> I am not expert in major compaction feature.
> > >> Let me try to answer questions in #2.
> > >>
> > >> 2.a
> > >>> If I set the property via the configuration shouldn’t all the cluster
> > be
> > >>> aware of?
> > >>
> > >> There're multiple clients connecting to one cluster. I wouldn't expect
> > >> values in the configuration (m_hbConfig) to propagate onto the
> cluster.
> > >>
> > >> 2.b
> > >> Store.getNextMajorCompactTime() shows that
> > "hbase.hregion.majorcompaction"
> > >> can be specified per column family:
> > >>
> > >> long getNextMajorCompactTime() {
> > >>   // default = 24hrs
> > >>   long ret = conf.getLong(HConstants.MAJOR_COMPACTION_PERIOD,
> > >> 1000*60*60*24);
> > >>   if (family.getValue(HConstants.MAJOR_COMPACTION_PERIOD) != null) {
> > >>
> > >> 2.d
> > >>> d. I tried also to setup the parameter via hbase shell but setting
> such
> > >>> properties is not supported. (do you plan to add such support via the
> > >>> shell?)
> > >>
> > >> This is a good idea. Please open a JIRA.
> > >>
> > >> For #5, HBASE-3965 is an improvement and doesn't have a patch yet.
> > >>
> > >> Allow me to quote Alan Kay: 'The best way to predict the future is to
> > >> invent it.'
> > >>
> > >> Once we have a patch, we can always backport it to 0.92 after some
> > people
> > >> have verified the improvement.
> > >>
> > >>> 6.       In case a compaction (major) is running it seems there is no
> > way
> > >>> to stop-it. Do you plan to add such feature?
> > >>
> > >> Again, logging a JIRA would provide a good starting point for
> > discussion.
> > >>
> > >> Thanks for the verification work and suggestions, Mikael.
> > >>
> > >> On Sun, Jan 8, 2012 at 7:27 AM, Mikael Sitruk <
> mikael.sitruk@gmail.com
> > >>> wrote:
> > >>
> > >>> I forgot to mention, I'm using HBase 0.90.1
> > >>>
> > >>> Regards,
> > >>> Mikael.S
> > >>>
> > >>> On Sun, Jan 8, 2012 at 5:25 PM, Mikael Sitruk <
> mikael.sitruk@gmail.com
> > >>>> wrote:
> > >>>
> > >>>> Hi
> > >>>>
> > >>>>
> > >>>>
> > >>>> I have some concern regarding major compactions below...
> > >>>>
> > >>>>
> > >>>>   1. According to best practices from the mailing list and from the
> > >>>>   book, automatic major compaction should be disabled. This can be
> > >> done
> > >>> by
> > >>>>   setting the property ‘hbase.hregion.majorcompaction’ to ‘0’.
> > >>> Neverhteless
> > >>>>   even after having doing this I STILL see “major compaction”
> messages
> > >>> in
> > >>>>   logs. therefore it is unclear how can I manage major compactions.
> > >> (The
> > >>>>   system has heavy insert - uniformly on the cluster, and major
> > >>> compaction
> > >>>>   affect the performance of the system).
> > >>>>   If I'm not wrong it seems from the code that: even if not
> requested
> > >>>>   and even if the indicator is set to '0' (no automatic major
> > >>> compaction),
> > >>>>   major compaction can be triggered by the code in case all store
> > >> files
> > >>> are
> > >>>>   candidate for a compaction (from Store.compact(final boolean
> > >>> forceMajor)).
> > >>>>   Shouldn't the code add a condition that automatic major compaction
> > >> is
> > >>>>   disabled??
> > >>>>
> > >>>>   2. I tried to check the parameter  ‘hbase.hregion.majorcompaction’
> > >> at
> > >>>>   runtime using several approaches - to validate that the server
> > >> indeed
> > >>>>   loaded the parameter.
> > >>>>
> > >>>> a. Using a connection created from local config
> > >>>>
> > >>>> *conn = (HConnection) HConnectionManager.getConnection(m_hbConfig);*
> > >>>>
> > >>>> *conn.getConfiguration().getString(“hbase.hregion.majorcompaction”)*
> > >>>>
> > >>>> returns the parameter from local config and not from cluster. Is it
> a
> > >>> bug?
> > >>>> If I set the property via the configuration shouldn’t all the
> cluster
> > >> be
> > >>>> aware of? (supposing that the connection indeed connected to the
> > >> cluster)
> > >>>>
> > >>>> b.  fetching the property from the table descriptor
> > >>>>
> > >>>> *HTableDescriptor hTableDescriptor =
> > >>>> conn.getHTableDescriptor(Bytes.toBytes("my table"));*
> > >>>>
> > >>>> *hTableDescriptor.getValue("hbase.hregion.majorcompaction")*
> > >>>>
> > >>>> This will returns the default parameter value (1 day) not the
> > parameter
> > >>>> from the configuration (on the cluster). It seems to be a bug, isn’t
> > >> it?
> > >>>> (the parameter from the config, should be the default if not set at
> > the
> > >>>> table level)
> > >>>>
> > >>>> c. The only way I could set the parameter to 0 and really see it is
> > via
> > >>>> the Admin API, updating the table descriptor or the column
> descriptor.
> > >>> Now
> > >>>> I could see the parameter on the web UI. So is it the only way to
> set
> > >>>> correctly the parameter? If setting the parameter via the
> > configuration
> > >>>> file, shouldn’t the webUI show this on any table created?
> > >>>>
> > >>>> d. I tried also to setup the parameter via hbase shell but setting
> > such
> > >>>> properties is not supported. (do you plan to add such support via
> the
> > >>>> shell?)
> > >>>>
> > >>>> e. Generally is it possible to get via API the configuration used by
> > >> the
> > >>>> servers? (at cluster/server level)
> > >>>>
> > >>>>    3.  I ran both major compaction  requests from the shell or from
> > >> API
> > >>>> but since both are async there is no progress indication. Neither
> the
> > >> JMX
> > >>>> nor the Web will help here since you don’t know if a compaction task
> > is
> > >>>> running. Tailling the logs is not an efficient way to do this
> neither.
> > >>> The
> > >>>> point is that I would like to automate the process and avoid
> > compaction
> > >>>> storm. So I want to do that region, region, but if I don’t know
> when a
> > >>>> compaction started/ended I can’t automate it.
> > >>>>
> > >>>> 4.       In case there is no compaction files in queue (but still
> you
> > >>> have
> > >>>> more than 1 storefile per store e.g. minor compaction just finished)
> > >> then
> > >>>> invoking major_compact will indeed decrease the number of store
> files,
> > >>> but
> > >>>> the compaction queue will remain to 0 during the compaction task
> > >>> (shouldn’t
> > >>>> the compaction queue increase by the number of file to compact and
> be
> > >>>> reduced when the task ended?)
> > >>>>
> > >>>>
> > >>>> 5.       I saw already HBASE-3965 for getting status of major
> > >> compaction,
> > >>>> nevertheless it has be removed from 0.92, is it possible to put it
> > >> back?
> > >>>> Even sooner than 0.92?
> > >>>>
> > >>>> 6.       In case a compaction (major) is running it seems there is
> no
> > >> way
> > >>>> to stop-it. Do you plan to add such feature?
> > >>>>
> > >>>> 7.       Do you plan to add functionality via JMX (starting/stopping
> > >>>> compaction, splitting....)
> > >>>>
> > >>>> 8.       Finally there were some request for allowing custom
> > >> compaction,
> > >>>> part of this was given via the RegionObserver in HBASE-2001,
> > >> nevertheless
> > >>>> do you consider adding support for custom compaction (providing real
> > >>>> pluggable compaction stategy not just observer)?
> > >>>>
> > >>>>
> > >>>> Regards,
> > >>>> Mikael.S
> > >>>>
> > >>>>
> > >>>
> > >>>
> > >>> --
> > >>> Mikael.S
> > >>>
> > >>
> > >
> > >
> > >
> > > --
> > > Mikael.S
> >
>

Re: Major Compaction Concerns

Posted by Mikael Sitruk <mi...@gmail.com>.
In fact I think that for 2.a the current implementation is misleading.
Creating a connection and getting the configuration from the connection
should return the configuration of the cluster.
Requesting the configuration used to build an object should return the
configuration set on the object
Additionally it should be a new method like getConfigurations(), or
getClusterConfigurations() returning a map of serverinfo and
configuration.  Another option is to add on the HRegionServer and HMaster a
method getConfiguration() returning the configuration object used by the
RegionServer or Master

Regarding 2.b yes I tried but it did not return the setting from the
cluster configuration (again server has non default configuration, table
was not configured with specific values then cluster configuration should
apply on the table object). So I see it as problematic.

Mikael.s
 On Jan 8, 2012 7:54 PM, <yu...@gmail.com> wrote:

> About 2b, have you tried getting the major compaction setting from column
> descriptor ?
>
> For 2a, what you requested would result in new methods of
> HBaseConfiguration class to be added. Currently the configuration on client
> class path would be used.
>
> Cheers
>
>
>
> On Jan 8, 2012, at 9:28 AM, Mikael Sitruk <mi...@gmail.com> wrote:
>
> > Ted hi
> > First thanks for answering, regarding the JIRA i will fill them
> > Second, it seems that i did not explain myself correctly regarding 2.a. -
> > As you i do not expect that a configuration set on my client will be
> > propagated to the cluster, but i do expect that if i set a configuration
> on
> > a server then doing connection.getConfiguration() from a client i will
> get
> > teh configuration from the cluster.
> > Currently the configuration returned is from the client config.
> > So the problem is that you have no way to check the configuration of a
> > cluster.
> > I would expect to have some API to return the cluster config and even
> > getting a map <serverInfo, config> so it can be easy to check cluster
> > problem using code.
> >
> > 2.b. I know this code, and i tried to validate it. I set in the server
> > config the "hbase.hregion.majorcompaction" to "0", then start the server
> > (cluster). Since from the UI or from JMX this parameter is not visible at
> > the cluster level, I try to get the value from the client (to see that
> the
> > cluster is using it)
> >
> > *HTableDescriptor hTableDescriptor =
> > conn.getHTableDescriptor(Bytes.toBytes("my table"));*
> >
> > *hTableDescriptor.getValue("hbase.hregion.majorcompaction")*
> > but i still got 24h (and not the value set in the config)! that was my
> > problem from the beginning! ==> Using the config (on the server side)
> will
> > not propagate into the table/column family
> >
> > Mikael.S
> >
> > On Sun, Jan 8, 2012 at 7:13 PM, Ted Yu <yu...@gmail.com> wrote:
> >
> >> I am not expert in major compaction feature.
> >> Let me try to answer questions in #2.
> >>
> >> 2.a
> >>> If I set the property via the configuration shouldn’t all the cluster
> be
> >>> aware of?
> >>
> >> There're multiple clients connecting to one cluster. I wouldn't expect
> >> values in the configuration (m_hbConfig) to propagate onto the cluster.
> >>
> >> 2.b
> >> Store.getNextMajorCompactTime() shows that
> "hbase.hregion.majorcompaction"
> >> can be specified per column family:
> >>
> >> long getNextMajorCompactTime() {
> >>   // default = 24hrs
> >>   long ret = conf.getLong(HConstants.MAJOR_COMPACTION_PERIOD,
> >> 1000*60*60*24);
> >>   if (family.getValue(HConstants.MAJOR_COMPACTION_PERIOD) != null) {
> >>
> >> 2.d
> >>> d. I tried also to setup the parameter via hbase shell but setting such
> >>> properties is not supported. (do you plan to add such support via the
> >>> shell?)
> >>
> >> This is a good idea. Please open a JIRA.
> >>
> >> For #5, HBASE-3965 is an improvement and doesn't have a patch yet.
> >>
> >> Allow me to quote Alan Kay: 'The best way to predict the future is to
> >> invent it.'
> >>
> >> Once we have a patch, we can always backport it to 0.92 after some
> people
> >> have verified the improvement.
> >>
> >>> 6.       In case a compaction (major) is running it seems there is no
> way
> >>> to stop-it. Do you plan to add such feature?
> >>
> >> Again, logging a JIRA would provide a good starting point for
> discussion.
> >>
> >> Thanks for the verification work and suggestions, Mikael.
> >>
> >> On Sun, Jan 8, 2012 at 7:27 AM, Mikael Sitruk <mikael.sitruk@gmail.com
> >>> wrote:
> >>
> >>> I forgot to mention, I'm using HBase 0.90.1
> >>>
> >>> Regards,
> >>> Mikael.S
> >>>
> >>> On Sun, Jan 8, 2012 at 5:25 PM, Mikael Sitruk <mikael.sitruk@gmail.com
> >>>> wrote:
> >>>
> >>>> Hi
> >>>>
> >>>>
> >>>>
> >>>> I have some concern regarding major compactions below...
> >>>>
> >>>>
> >>>>   1. According to best practices from the mailing list and from the
> >>>>   book, automatic major compaction should be disabled. This can be
> >> done
> >>> by
> >>>>   setting the property ‘hbase.hregion.majorcompaction’ to ‘0’.
> >>> Neverhteless
> >>>>   even after having doing this I STILL see “major compaction” messages
> >>> in
> >>>>   logs. therefore it is unclear how can I manage major compactions.
> >> (The
> >>>>   system has heavy insert - uniformly on the cluster, and major
> >>> compaction
> >>>>   affect the performance of the system).
> >>>>   If I'm not wrong it seems from the code that: even if not requested
> >>>>   and even if the indicator is set to '0' (no automatic major
> >>> compaction),
> >>>>   major compaction can be triggered by the code in case all store
> >> files
> >>> are
> >>>>   candidate for a compaction (from Store.compact(final boolean
> >>> forceMajor)).
> >>>>   Shouldn't the code add a condition that automatic major compaction
> >> is
> >>>>   disabled??
> >>>>
> >>>>   2. I tried to check the parameter  ‘hbase.hregion.majorcompaction’
> >> at
> >>>>   runtime using several approaches - to validate that the server
> >> indeed
> >>>>   loaded the parameter.
> >>>>
> >>>> a. Using a connection created from local config
> >>>>
> >>>> *conn = (HConnection) HConnectionManager.getConnection(m_hbConfig);*
> >>>>
> >>>> *conn.getConfiguration().getString(“hbase.hregion.majorcompaction”)*
> >>>>
> >>>> returns the parameter from local config and not from cluster. Is it a
> >>> bug?
> >>>> If I set the property via the configuration shouldn’t all the cluster
> >> be
> >>>> aware of? (supposing that the connection indeed connected to the
> >> cluster)
> >>>>
> >>>> b.  fetching the property from the table descriptor
> >>>>
> >>>> *HTableDescriptor hTableDescriptor =
> >>>> conn.getHTableDescriptor(Bytes.toBytes("my table"));*
> >>>>
> >>>> *hTableDescriptor.getValue("hbase.hregion.majorcompaction")*
> >>>>
> >>>> This will returns the default parameter value (1 day) not the
> parameter
> >>>> from the configuration (on the cluster). It seems to be a bug, isn’t
> >> it?
> >>>> (the parameter from the config, should be the default if not set at
> the
> >>>> table level)
> >>>>
> >>>> c. The only way I could set the parameter to 0 and really see it is
> via
> >>>> the Admin API, updating the table descriptor or the column descriptor.
> >>> Now
> >>>> I could see the parameter on the web UI. So is it the only way to set
> >>>> correctly the parameter? If setting the parameter via the
> configuration
> >>>> file, shouldn’t the webUI show this on any table created?
> >>>>
> >>>> d. I tried also to setup the parameter via hbase shell but setting
> such
> >>>> properties is not supported. (do you plan to add such support via the
> >>>> shell?)
> >>>>
> >>>> e. Generally is it possible to get via API the configuration used by
> >> the
> >>>> servers? (at cluster/server level)
> >>>>
> >>>>    3.  I ran both major compaction  requests from the shell or from
> >> API
> >>>> but since both are async there is no progress indication. Neither the
> >> JMX
> >>>> nor the Web will help here since you don’t know if a compaction task
> is
> >>>> running. Tailling the logs is not an efficient way to do this neither.
> >>> The
> >>>> point is that I would like to automate the process and avoid
> compaction
> >>>> storm. So I want to do that region, region, but if I don’t know when a
> >>>> compaction started/ended I can’t automate it.
> >>>>
> >>>> 4.       In case there is no compaction files in queue (but still you
> >>> have
> >>>> more than 1 storefile per store e.g. minor compaction just finished)
> >> then
> >>>> invoking major_compact will indeed decrease the number of store files,
> >>> but
> >>>> the compaction queue will remain to 0 during the compaction task
> >>> (shouldn’t
> >>>> the compaction queue increase by the number of file to compact and be
> >>>> reduced when the task ended?)
> >>>>
> >>>>
> >>>> 5.       I saw already HBASE-3965 for getting status of major
> >> compaction,
> >>>> nevertheless it has be removed from 0.92, is it possible to put it
> >> back?
> >>>> Even sooner than 0.92?
> >>>>
> >>>> 6.       In case a compaction (major) is running it seems there is no
> >> way
> >>>> to stop-it. Do you plan to add such feature?
> >>>>
> >>>> 7.       Do you plan to add functionality via JMX (starting/stopping
> >>>> compaction, splitting....)
> >>>>
> >>>> 8.       Finally there were some request for allowing custom
> >> compaction,
> >>>> part of this was given via the RegionObserver in HBASE-2001,
> >> nevertheless
> >>>> do you consider adding support for custom compaction (providing real
> >>>> pluggable compaction stategy not just observer)?
> >>>>
> >>>>
> >>>> Regards,
> >>>> Mikael.S
> >>>>
> >>>>
> >>>
> >>>
> >>> --
> >>> Mikael.S
> >>>
> >>
> >
> >
> >
> > --
> > Mikael.S
>

Re: Major Compaction Concerns

Posted by yu...@gmail.com.
About 2b, have you tried getting the major compaction setting from column descriptor ?

For 2a, what you requested would result in new methods of HBaseConfiguration class to be added. Currently the configuration on client class path would be used. 

Cheers



On Jan 8, 2012, at 9:28 AM, Mikael Sitruk <mi...@gmail.com> wrote:

> Ted hi
> First thanks for answering, regarding the JIRA i will fill them
> Second, it seems that i did not explain myself correctly regarding 2.a. -
> As you i do not expect that a configuration set on my client will be
> propagated to the cluster, but i do expect that if i set a configuration on
> a server then doing connection.getConfiguration() from a client i will get
> teh configuration from the cluster.
> Currently the configuration returned is from the client config.
> So the problem is that you have no way to check the configuration of a
> cluster.
> I would expect to have some API to return the cluster config and even
> getting a map <serverInfo, config> so it can be easy to check cluster
> problem using code.
> 
> 2.b. I know this code, and i tried to validate it. I set in the server
> config the "hbase.hregion.majorcompaction" to "0", then start the server
> (cluster). Since from the UI or from JMX this parameter is not visible at
> the cluster level, I try to get the value from the client (to see that the
> cluster is using it)
> 
> *HTableDescriptor hTableDescriptor =
> conn.getHTableDescriptor(Bytes.toBytes("my table"));*
> 
> *hTableDescriptor.getValue("hbase.hregion.majorcompaction")*
> but i still got 24h (and not the value set in the config)! that was my
> problem from the beginning! ==> Using the config (on the server side) will
> not propagate into the table/column family
> 
> Mikael.S
> 
> On Sun, Jan 8, 2012 at 7:13 PM, Ted Yu <yu...@gmail.com> wrote:
> 
>> I am not expert in major compaction feature.
>> Let me try to answer questions in #2.
>> 
>> 2.a
>>> If I set the property via the configuration shouldn’t all the cluster be
>>> aware of?
>> 
>> There're multiple clients connecting to one cluster. I wouldn't expect
>> values in the configuration (m_hbConfig) to propagate onto the cluster.
>> 
>> 2.b
>> Store.getNextMajorCompactTime() shows that "hbase.hregion.majorcompaction"
>> can be specified per column family:
>> 
>> long getNextMajorCompactTime() {
>>   // default = 24hrs
>>   long ret = conf.getLong(HConstants.MAJOR_COMPACTION_PERIOD,
>> 1000*60*60*24);
>>   if (family.getValue(HConstants.MAJOR_COMPACTION_PERIOD) != null) {
>> 
>> 2.d
>>> d. I tried also to setup the parameter via hbase shell but setting such
>>> properties is not supported. (do you plan to add such support via the
>>> shell?)
>> 
>> This is a good idea. Please open a JIRA.
>> 
>> For #5, HBASE-3965 is an improvement and doesn't have a patch yet.
>> 
>> Allow me to quote Alan Kay: 'The best way to predict the future is to
>> invent it.'
>> 
>> Once we have a patch, we can always backport it to 0.92 after some people
>> have verified the improvement.
>> 
>>> 6.       In case a compaction (major) is running it seems there is no way
>>> to stop-it. Do you plan to add such feature?
>> 
>> Again, logging a JIRA would provide a good starting point for discussion.
>> 
>> Thanks for the verification work and suggestions, Mikael.
>> 
>> On Sun, Jan 8, 2012 at 7:27 AM, Mikael Sitruk <mikael.sitruk@gmail.com
>>> wrote:
>> 
>>> I forgot to mention, I'm using HBase 0.90.1
>>> 
>>> Regards,
>>> Mikael.S
>>> 
>>> On Sun, Jan 8, 2012 at 5:25 PM, Mikael Sitruk <mikael.sitruk@gmail.com
>>>> wrote:
>>> 
>>>> Hi
>>>> 
>>>> 
>>>> 
>>>> I have some concern regarding major compactions below...
>>>> 
>>>> 
>>>>   1. According to best practices from the mailing list and from the
>>>>   book, automatic major compaction should be disabled. This can be
>> done
>>> by
>>>>   setting the property ‘hbase.hregion.majorcompaction’ to ‘0’.
>>> Neverhteless
>>>>   even after having doing this I STILL see “major compaction” messages
>>> in
>>>>   logs. therefore it is unclear how can I manage major compactions.
>> (The
>>>>   system has heavy insert - uniformly on the cluster, and major
>>> compaction
>>>>   affect the performance of the system).
>>>>   If I'm not wrong it seems from the code that: even if not requested
>>>>   and even if the indicator is set to '0' (no automatic major
>>> compaction),
>>>>   major compaction can be triggered by the code in case all store
>> files
>>> are
>>>>   candidate for a compaction (from Store.compact(final boolean
>>> forceMajor)).
>>>>   Shouldn't the code add a condition that automatic major compaction
>> is
>>>>   disabled??
>>>> 
>>>>   2. I tried to check the parameter  ‘hbase.hregion.majorcompaction’
>> at
>>>>   runtime using several approaches - to validate that the server
>> indeed
>>>>   loaded the parameter.
>>>> 
>>>> a. Using a connection created from local config
>>>> 
>>>> *conn = (HConnection) HConnectionManager.getConnection(m_hbConfig);*
>>>> 
>>>> *conn.getConfiguration().getString(“hbase.hregion.majorcompaction”)*
>>>> 
>>>> returns the parameter from local config and not from cluster. Is it a
>>> bug?
>>>> If I set the property via the configuration shouldn’t all the cluster
>> be
>>>> aware of? (supposing that the connection indeed connected to the
>> cluster)
>>>> 
>>>> b.  fetching the property from the table descriptor
>>>> 
>>>> *HTableDescriptor hTableDescriptor =
>>>> conn.getHTableDescriptor(Bytes.toBytes("my table"));*
>>>> 
>>>> *hTableDescriptor.getValue("hbase.hregion.majorcompaction")*
>>>> 
>>>> This will returns the default parameter value (1 day) not the parameter
>>>> from the configuration (on the cluster). It seems to be a bug, isn’t
>> it?
>>>> (the parameter from the config, should be the default if not set at the
>>>> table level)
>>>> 
>>>> c. The only way I could set the parameter to 0 and really see it is via
>>>> the Admin API, updating the table descriptor or the column descriptor.
>>> Now
>>>> I could see the parameter on the web UI. So is it the only way to set
>>>> correctly the parameter? If setting the parameter via the configuration
>>>> file, shouldn’t the webUI show this on any table created?
>>>> 
>>>> d. I tried also to setup the parameter via hbase shell but setting such
>>>> properties is not supported. (do you plan to add such support via the
>>>> shell?)
>>>> 
>>>> e. Generally is it possible to get via API the configuration used by
>> the
>>>> servers? (at cluster/server level)
>>>> 
>>>>    3.  I ran both major compaction  requests from the shell or from
>> API
>>>> but since both are async there is no progress indication. Neither the
>> JMX
>>>> nor the Web will help here since you don’t know if a compaction task is
>>>> running. Tailling the logs is not an efficient way to do this neither.
>>> The
>>>> point is that I would like to automate the process and avoid compaction
>>>> storm. So I want to do that region, region, but if I don’t know when a
>>>> compaction started/ended I can’t automate it.
>>>> 
>>>> 4.       In case there is no compaction files in queue (but still you
>>> have
>>>> more than 1 storefile per store e.g. minor compaction just finished)
>> then
>>>> invoking major_compact will indeed decrease the number of store files,
>>> but
>>>> the compaction queue will remain to 0 during the compaction task
>>> (shouldn’t
>>>> the compaction queue increase by the number of file to compact and be
>>>> reduced when the task ended?)
>>>> 
>>>> 
>>>> 5.       I saw already HBASE-3965 for getting status of major
>> compaction,
>>>> nevertheless it has be removed from 0.92, is it possible to put it
>> back?
>>>> Even sooner than 0.92?
>>>> 
>>>> 6.       In case a compaction (major) is running it seems there is no
>> way
>>>> to stop-it. Do you plan to add such feature?
>>>> 
>>>> 7.       Do you plan to add functionality via JMX (starting/stopping
>>>> compaction, splitting....)
>>>> 
>>>> 8.       Finally there were some request for allowing custom
>> compaction,
>>>> part of this was given via the RegionObserver in HBASE-2001,
>> nevertheless
>>>> do you consider adding support for custom compaction (providing real
>>>> pluggable compaction stategy not just observer)?
>>>> 
>>>> 
>>>> Regards,
>>>> Mikael.S
>>>> 
>>>> 
>>> 
>>> 
>>> --
>>> Mikael.S
>>> 
>> 
> 
> 
> 
> -- 
> Mikael.S

Re: Major Compaction Concerns

Posted by Mikael Sitruk <mi...@gmail.com>.
Ted hi
First thanks for answering, regarding the JIRA i will fill them
Second, it seems that i did not explain myself correctly regarding 2.a. -
As you i do not expect that a configuration set on my client will be
propagated to the cluster, but i do expect that if i set a configuration on
a server then doing connection.getConfiguration() from a client i will get
teh configuration from the cluster.
Currently the configuration returned is from the client config.
So the problem is that you have no way to check the configuration of a
cluster.
I would expect to have some API to return the cluster config and even
getting a map <serverInfo, config> so it can be easy to check cluster
problem using code.

2.b. I know this code, and i tried to validate it. I set in the server
config the "hbase.hregion.majorcompaction" to "0", then start the server
(cluster). Since from the UI or from JMX this parameter is not visible at
the cluster level, I try to get the value from the client (to see that the
cluster is using it)

*HTableDescriptor hTableDescriptor =
conn.getHTableDescriptor(Bytes.toBytes("my table"));*

*hTableDescriptor.getValue("hbase.hregion.majorcompaction")*
but i still got 24h (and not the value set in the config)! that was my
problem from the beginning! ==> Using the config (on the server side) will
not propagate into the table/column family

Mikael.S

On Sun, Jan 8, 2012 at 7:13 PM, Ted Yu <yu...@gmail.com> wrote:

> I am not expert in major compaction feature.
> Let me try to answer questions in #2.
>
> 2.a
> > If I set the property via the configuration shouldn’t all the cluster be
> > aware of?
>
> There're multiple clients connecting to one cluster. I wouldn't expect
> values in the configuration (m_hbConfig) to propagate onto the cluster.
>
> 2.b
> Store.getNextMajorCompactTime() shows that "hbase.hregion.majorcompaction"
> can be specified per column family:
>
>  long getNextMajorCompactTime() {
>    // default = 24hrs
>    long ret = conf.getLong(HConstants.MAJOR_COMPACTION_PERIOD,
> 1000*60*60*24);
>    if (family.getValue(HConstants.MAJOR_COMPACTION_PERIOD) != null) {
>
> 2.d
> > d. I tried also to setup the parameter via hbase shell but setting such
> > properties is not supported. (do you plan to add such support via the
> > shell?)
>
> This is a good idea. Please open a JIRA.
>
> For #5, HBASE-3965 is an improvement and doesn't have a patch yet.
>
> Allow me to quote Alan Kay: 'The best way to predict the future is to
> invent it.'
>
> Once we have a patch, we can always backport it to 0.92 after some people
> have verified the improvement.
>
> > 6.       In case a compaction (major) is running it seems there is no way
> > to stop-it. Do you plan to add such feature?
>
> Again, logging a JIRA would provide a good starting point for discussion.
>
> Thanks for the verification work and suggestions, Mikael.
>
> On Sun, Jan 8, 2012 at 7:27 AM, Mikael Sitruk <mikael.sitruk@gmail.com
> >wrote:
>
> > I forgot to mention, I'm using HBase 0.90.1
> >
> > Regards,
> > Mikael.S
> >
> > On Sun, Jan 8, 2012 at 5:25 PM, Mikael Sitruk <mikael.sitruk@gmail.com
> > >wrote:
> >
> > > Hi
> > >
> > >
> > >
> > > I have some concern regarding major compactions below...
> > >
> > >
> > >    1. According to best practices from the mailing list and from the
> > >    book, automatic major compaction should be disabled. This can be
> done
> > by
> > >    setting the property ‘hbase.hregion.majorcompaction’ to ‘0’.
> > Neverhteless
> > >    even after having doing this I STILL see “major compaction” messages
> > in
> > >    logs. therefore it is unclear how can I manage major compactions.
> (The
> > >    system has heavy insert - uniformly on the cluster, and major
> > compaction
> > >    affect the performance of the system).
> > >    If I'm not wrong it seems from the code that: even if not requested
> > >    and even if the indicator is set to '0' (no automatic major
> > compaction),
> > >    major compaction can be triggered by the code in case all store
> files
> > are
> > >    candidate for a compaction (from Store.compact(final boolean
> > forceMajor)).
> > >    Shouldn't the code add a condition that automatic major compaction
> is
> > >    disabled??
> > >
> > >    2. I tried to check the parameter  ‘hbase.hregion.majorcompaction’
>  at
> > >    runtime using several approaches - to validate that the server
> indeed
> > >    loaded the parameter.
> > >
> > > a. Using a connection created from local config
> > >
> > > *conn = (HConnection) HConnectionManager.getConnection(m_hbConfig);*
> > >
> > > *conn.getConfiguration().getString(“hbase.hregion.majorcompaction”)*
> > >
> > > returns the parameter from local config and not from cluster. Is it a
> > bug?
> > > If I set the property via the configuration shouldn’t all the cluster
> be
> > > aware of? (supposing that the connection indeed connected to the
> cluster)
> > >
> > > b.  fetching the property from the table descriptor
> > >
> > > *HTableDescriptor hTableDescriptor =
> > > conn.getHTableDescriptor(Bytes.toBytes("my table"));*
> > >
> > > *hTableDescriptor.getValue("hbase.hregion.majorcompaction")*
> > >
> > > This will returns the default parameter value (1 day) not the parameter
> > > from the configuration (on the cluster). It seems to be a bug, isn’t
> it?
> > > (the parameter from the config, should be the default if not set at the
> > > table level)
> > >
> > > c. The only way I could set the parameter to 0 and really see it is via
> > > the Admin API, updating the table descriptor or the column descriptor.
> > Now
> > > I could see the parameter on the web UI. So is it the only way to set
> > > correctly the parameter? If setting the parameter via the configuration
> > > file, shouldn’t the webUI show this on any table created?
> > >
> > > d. I tried also to setup the parameter via hbase shell but setting such
> > > properties is not supported. (do you plan to add such support via the
> > > shell?)
> > >
> > > e. Generally is it possible to get via API the configuration used by
> the
> > > servers? (at cluster/server level)
> > >
> > >     3.  I ran both major compaction  requests from the shell or from
> API
> > > but since both are async there is no progress indication. Neither the
> JMX
> > > nor the Web will help here since you don’t know if a compaction task is
> > > running. Tailling the logs is not an efficient way to do this neither.
> > The
> > > point is that I would like to automate the process and avoid compaction
> > > storm. So I want to do that region, region, but if I don’t know when a
> > > compaction started/ended I can’t automate it.
> > >
> > > 4.       In case there is no compaction files in queue (but still you
> > have
> > > more than 1 storefile per store e.g. minor compaction just finished)
> then
> > > invoking major_compact will indeed decrease the number of store files,
> > but
> > > the compaction queue will remain to 0 during the compaction task
> > (shouldn’t
> > > the compaction queue increase by the number of file to compact and be
> > > reduced when the task ended?)
> > >
> > >
> > > 5.       I saw already HBASE-3965 for getting status of major
> compaction,
> > > nevertheless it has be removed from 0.92, is it possible to put it
> back?
> > > Even sooner than 0.92?
> > >
> > > 6.       In case a compaction (major) is running it seems there is no
> way
> > > to stop-it. Do you plan to add such feature?
> > >
> > > 7.       Do you plan to add functionality via JMX (starting/stopping
> > > compaction, splitting....)
> > >
> > > 8.       Finally there were some request for allowing custom
> compaction,
> > > part of this was given via the RegionObserver in HBASE-2001,
> nevertheless
> > > do you consider adding support for custom compaction (providing real
> > > pluggable compaction stategy not just observer)?
> > >
> > >
> > > Regards,
> > > Mikael.S
> > >
> > >
> >
> >
> > --
> > Mikael.S
> >
>



-- 
Mikael.S

Re: Major Compaction Concerns

Posted by Ted Yu <yu...@gmail.com>.
I am not expert in major compaction feature.
Let me try to answer questions in #2.

2.a
> If I set the property via the configuration shouldn’t all the cluster be
> aware of?

There're multiple clients connecting to one cluster. I wouldn't expect
values in the configuration (m_hbConfig) to propagate onto the cluster.

2.b
Store.getNextMajorCompactTime() shows that "hbase.hregion.majorcompaction"
can be specified per column family:

  long getNextMajorCompactTime() {
    // default = 24hrs
    long ret = conf.getLong(HConstants.MAJOR_COMPACTION_PERIOD,
1000*60*60*24);
    if (family.getValue(HConstants.MAJOR_COMPACTION_PERIOD) != null) {

2.d
> d. I tried also to setup the parameter via hbase shell but setting such
> properties is not supported. (do you plan to add such support via the
> shell?)

This is a good idea. Please open a JIRA.

For #5, HBASE-3965 is an improvement and doesn't have a patch yet.

Allow me to quote Alan Kay: 'The best way to predict the future is to
invent it.'

Once we have a patch, we can always backport it to 0.92 after some people
have verified the improvement.

> 6.       In case a compaction (major) is running it seems there is no way
> to stop-it. Do you plan to add such feature?

Again, logging a JIRA would provide a good starting point for discussion.

Thanks for the verification work and suggestions, Mikael.

On Sun, Jan 8, 2012 at 7:27 AM, Mikael Sitruk <mi...@gmail.com>wrote:

> I forgot to mention, I'm using HBase 0.90.1
>
> Regards,
> Mikael.S
>
> On Sun, Jan 8, 2012 at 5:25 PM, Mikael Sitruk <mikael.sitruk@gmail.com
> >wrote:
>
> > Hi
> >
> >
> >
> > I have some concern regarding major compactions below...
> >
> >
> >    1. According to best practices from the mailing list and from the
> >    book, automatic major compaction should be disabled. This can be done
> by
> >    setting the property ‘hbase.hregion.majorcompaction’ to ‘0’.
> Neverhteless
> >    even after having doing this I STILL see “major compaction” messages
> in
> >    logs. therefore it is unclear how can I manage major compactions. (The
> >    system has heavy insert - uniformly on the cluster, and major
> compaction
> >    affect the performance of the system).
> >    If I'm not wrong it seems from the code that: even if not requested
> >    and even if the indicator is set to '0' (no automatic major
> compaction),
> >    major compaction can be triggered by the code in case all store files
> are
> >    candidate for a compaction (from Store.compact(final boolean
> forceMajor)).
> >    Shouldn't the code add a condition that automatic major compaction is
> >    disabled??
> >
> >    2. I tried to check the parameter  ‘hbase.hregion.majorcompaction’  at
> >    runtime using several approaches - to validate that the server indeed
> >    loaded the parameter.
> >
> > a. Using a connection created from local config
> >
> > *conn = (HConnection) HConnectionManager.getConnection(m_hbConfig);*
> >
> > *conn.getConfiguration().getString(“hbase.hregion.majorcompaction”)*
> >
> > returns the parameter from local config and not from cluster. Is it a
> bug?
> > If I set the property via the configuration shouldn’t all the cluster be
> > aware of? (supposing that the connection indeed connected to the cluster)
> >
> > b.  fetching the property from the table descriptor
> >
> > *HTableDescriptor hTableDescriptor =
> > conn.getHTableDescriptor(Bytes.toBytes("my table"));*
> >
> > *hTableDescriptor.getValue("hbase.hregion.majorcompaction")*
> >
> > This will returns the default parameter value (1 day) not the parameter
> > from the configuration (on the cluster). It seems to be a bug, isn’t it?
> > (the parameter from the config, should be the default if not set at the
> > table level)
> >
> > c. The only way I could set the parameter to 0 and really see it is via
> > the Admin API, updating the table descriptor or the column descriptor.
> Now
> > I could see the parameter on the web UI. So is it the only way to set
> > correctly the parameter? If setting the parameter via the configuration
> > file, shouldn’t the webUI show this on any table created?
> >
> > d. I tried also to setup the parameter via hbase shell but setting such
> > properties is not supported. (do you plan to add such support via the
> > shell?)
> >
> > e. Generally is it possible to get via API the configuration used by the
> > servers? (at cluster/server level)
> >
> >     3.  I ran both major compaction  requests from the shell or from API
> > but since both are async there is no progress indication. Neither the JMX
> > nor the Web will help here since you don’t know if a compaction task is
> > running. Tailling the logs is not an efficient way to do this neither.
> The
> > point is that I would like to automate the process and avoid compaction
> > storm. So I want to do that region, region, but if I don’t know when a
> > compaction started/ended I can’t automate it.
> >
> > 4.       In case there is no compaction files in queue (but still you
> have
> > more than 1 storefile per store e.g. minor compaction just finished) then
> > invoking major_compact will indeed decrease the number of store files,
> but
> > the compaction queue will remain to 0 during the compaction task
> (shouldn’t
> > the compaction queue increase by the number of file to compact and be
> > reduced when the task ended?)
> >
> >
> > 5.       I saw already HBASE-3965 for getting status of major compaction,
> > nevertheless it has be removed from 0.92, is it possible to put it back?
> > Even sooner than 0.92?
> >
> > 6.       In case a compaction (major) is running it seems there is no way
> > to stop-it. Do you plan to add such feature?
> >
> > 7.       Do you plan to add functionality via JMX (starting/stopping
> > compaction, splitting....)
> >
> > 8.       Finally there were some request for allowing custom compaction,
> > part of this was given via the RegionObserver in HBASE-2001, nevertheless
> > do you consider adding support for custom compaction (providing real
> > pluggable compaction stategy not just observer)?
> >
> >
> > Regards,
> > Mikael.S
> >
> >
>
>
> --
> Mikael.S
>

Re: Major Compaction Concerns

Posted by Mikael Sitruk <mi...@gmail.com>.
I forgot to mention, I'm using HBase 0.90.1

Regards,
Mikael.S

On Sun, Jan 8, 2012 at 5:25 PM, Mikael Sitruk <mi...@gmail.com>wrote:

> Hi
>
>
>
> I have some concern regarding major compactions below...
>
>
>    1. According to best practices from the mailing list and from the
>    book, automatic major compaction should be disabled. This can be done by
>    setting the property ‘hbase.hregion.majorcompaction’ to ‘0’. Neverhteless
>    even after having doing this I STILL see “major compaction” messages in
>    logs. therefore it is unclear how can I manage major compactions. (The
>    system has heavy insert - uniformly on the cluster, and major compaction
>    affect the performance of the system).
>    If I'm not wrong it seems from the code that: even if not requested
>    and even if the indicator is set to '0' (no automatic major compaction),
>    major compaction can be triggered by the code in case all store files are
>    candidate for a compaction (from Store.compact(final boolean forceMajor)).
>    Shouldn't the code add a condition that automatic major compaction is
>    disabled??
>
>    2. I tried to check the parameter  ‘hbase.hregion.majorcompaction’  at
>    runtime using several approaches - to validate that the server indeed
>    loaded the parameter.
>
> a. Using a connection created from local config
>
> *conn = (HConnection) HConnectionManager.getConnection(m_hbConfig);*
>
> *conn.getConfiguration().getString(“hbase.hregion.majorcompaction”)*
>
> returns the parameter from local config and not from cluster. Is it a bug?
> If I set the property via the configuration shouldn’t all the cluster be
> aware of? (supposing that the connection indeed connected to the cluster)
>
> b.  fetching the property from the table descriptor
>
> *HTableDescriptor hTableDescriptor =
> conn.getHTableDescriptor(Bytes.toBytes("my table"));*
>
> *hTableDescriptor.getValue("hbase.hregion.majorcompaction")*
>
> This will returns the default parameter value (1 day) not the parameter
> from the configuration (on the cluster). It seems to be a bug, isn’t it?
> (the parameter from the config, should be the default if not set at the
> table level)
>
> c. The only way I could set the parameter to 0 and really see it is via
> the Admin API, updating the table descriptor or the column descriptor. Now
> I could see the parameter on the web UI. So is it the only way to set
> correctly the parameter? If setting the parameter via the configuration
> file, shouldn’t the webUI show this on any table created?
>
> d. I tried also to setup the parameter via hbase shell but setting such
> properties is not supported. (do you plan to add such support via the
> shell?)
>
> e. Generally is it possible to get via API the configuration used by the
> servers? (at cluster/server level)
>
>     3.  I ran both major compaction  requests from the shell or from API
> but since both are async there is no progress indication. Neither the JMX
> nor the Web will help here since you don’t know if a compaction task is
> running. Tailling the logs is not an efficient way to do this neither. The
> point is that I would like to automate the process and avoid compaction
> storm. So I want to do that region, region, but if I don’t know when a
> compaction started/ended I can’t automate it.
>
> 4.       In case there is no compaction files in queue (but still you have
> more than 1 storefile per store e.g. minor compaction just finished) then
> invoking major_compact will indeed decrease the number of store files, but
> the compaction queue will remain to 0 during the compaction task (shouldn’t
> the compaction queue increase by the number of file to compact and be
> reduced when the task ended?)
>
>
> 5.       I saw already HBASE-3965 for getting status of major compaction,
> nevertheless it has be removed from 0.92, is it possible to put it back?
> Even sooner than 0.92?
>
> 6.       In case a compaction (major) is running it seems there is no way
> to stop-it. Do you plan to add such feature?
>
> 7.       Do you plan to add functionality via JMX (starting/stopping
> compaction, splitting....)
>
> 8.       Finally there were some request for allowing custom compaction,
> part of this was given via the RegionObserver in HBASE-2001, nevertheless
> do you consider adding support for custom compaction (providing real
> pluggable compaction stategy not just observer)?
>
>
> Regards,
> Mikael.S
>
>


-- 
Mikael.S

Re: Major Compaction Concerns

Posted by Nicolas Spiegelberg <ns...@fb.com>.
>1. my CF were already working with BF they used ROWCOL, (i didn't pay
>attention to that at the time i wrote my answers)
>2. I see form the logs that the BF is already 100% - is it bad? should I
>had more memory for BF?

Since Bloom Filters are a probabilistic optimization, it's kinda hard to
analyze your efficiency.  Mostly, we rely on theory and a little bit of
experimentation.  Basically, you want your key queries to have a high miss
rate on HFiles.  This doesn't mean that the key doesn't exist in the
Store.  It just means that you're not constantly writing to it, so it
doesn't exist in all N StoreFiles.  Optimally, you want 1 of the blooms to
hit (key exists in file) and N-1 to miss. Metrics that you can look at
(not sure about the versions of when these were introduced):

keymaybeinbloomcnt : number of bloom hits
keynotinbloomcnt : number of bloom misses.
staticbloomsizekb : size that bloom data takes up in memory (HFileV1)


Note that per-CF metrics are added in 0.94 so you can watch bloom
efficiency in finer granularity.

>3. HLog compression (HBASE-4608) is not scheduled yet, is it by intention?

There's limited bandwidth and this is an open source project, so... :)

>4. Compaction.ratio is only for 0.92.x releases, so i cannot use it yet.

"hbase.hstore.compaction.ratio" is in 0.90
(https://svn.apache.org/repos/asf/hbase/branches/0.90/src/main/java/org/apa
che/hadoop/hbase/regionserver/Store.java)


>6. I have also noticed that in a workload of pure insert (no read, empty
>regions, new keys) the store files on the RS can reach more than 4500
>files, nevertheless with a update/read scenario the store files were not
>passing 1500 files per region (the throttling of the flush was active and
>not in insert) Is there an explanation for that?

That depends on the size of your major compacted data.  Updates will
dedupe and lower your compaction volume.


Re: Major Compaction Concerns

Posted by Mikael Sitruk <mi...@gmail.com>.
A follow-up...
1. my CF were already working with BF they used ROWCOL, (i didn't pay
attention to that at the time i wrote my answers)
2. I see form the logs that the BF is already 100% - is it bad? should I
had more memory for BF?
3. HLog compression (HBASE-4608) is not scheduled yet, is it by intention?
4. Compaction.ratio is only for 0.92.x releases, so i cannot use it yet.
5. all other patches are also for 0.92/0.94 so my situation will not be
better till then, beside playing with the log rolling size, and max number
of store files
6. I have also noticed that in a workload of pure insert (no read, empty
regions, new keys) the store files on the RS can reach more than 4500
files, nevertheless with a update/read scenario the store files were not
passing 1500 files per region (the throttling of the flush was active and
not in insert) Is there an explanation for that?
7. I also have a 0.92 fresh install, and checking there the behavior
(additional result soon, hopefully)

Mikael.S


On Sat, Jan 14, 2012 at 11:30 PM, Mikael Sitruk <mi...@gmail.com>wrote:

> Wow, thank you very much for all those precious explanations, pointers and
> examples. It's a lot to ingest... I will try them (at least what i can with
> 0.90.4 (yes i'm upgrading from 0.90.1 to 0.90.4)) and keep you informed.
> BTW I'm already using compression (GZ), the current data is randomized so
> I don't have so much gain as you mentioned ( i think i'm around 30% only).
> It seems that BF is one of the major thing i need to look up with the
> compaction.ratio, and i need a different setting for my different CF. (one
> CF has small set of column and each update will change 50% --> ROWCOL, the
> second CF has always a new column per update --> ROW)
> I'm not keeping more than one version neither, and you wrote this is not a
> point query.
>
> A suggestion is perhaps to take all those example/explanation and add them
> to the book for future reference.
>
> Regards,
> Mikael.S
>
>
> On Sat, Jan 14, 2012 at 4:06 AM, Nicolas Spiegelberg <ns...@fb.com>wrote:
>
>> >I'm sorry but i don't understand, of course i have a disk and network
>> >saturation and the flush stop to flush because he is waiting for
>> >compaction
>> >to finish. Since this a major compaction was triggered - all the
>> >stores (large number)  present on the disks (7 disk per RS) will be
>> >grabbed
>> >for major compact, and the I/O is affected. Network is also affected
>> since
>> >all are major compacting at the same time and replicating files on same
>> >time (1GB network).
>>
>> When you have an IO problem, there are multiple pieces at play that you
>> can adjust:
>>
>> Write: HLog, Flush, Compaction
>> Read: Point Query, Scan
>>
>> If your writes are far more than your reads, then you should relax one of
>> the write pieces.
>> - HLog: You can't really adjust HLog IO outside of key compression
>> (HBASE-4608)
>> - Flush: You can adjust your compression.  None->LZO == 5x compression.
>> LZO->GZ == 2x compression.  Both are at the expense of CPU.  HBASE-4241
>> minimizes flush IO significantly in the update-heavy use case (discussed
>> this in the last email).
>> - Compaction: You can lower the compaction ratio to minimize the amount of
>> rewrites over time.  That's why I suggested changing the ratio from 1.2 ->
>> 0.25.  This gives a ~50% IO reduction (blog post on this forthcoming @
>> http://www.facebook.com/UsingHBase ).
>>
>> However, you may have a lot more reads than you think.  For example, let's
>> say read:write ratio is 1:10, so significantly read dominated.  Without
>> any of the optimizations I listed in the previous email, your real read
>> ratio is multiplied by the StoreFile count (because you naively read all
>> StoreFiles).  So let say, during congestion, you have 20 StoreFiles.
>> 1*20:10 means that you're now 2:1 read dominated.  You need features to
>> reduce the number of StoreFiles you scan when the StoreFile count is high.
>>
>> - Point Query: bloom filters (HBASE-1200, HBASE-2794), lazy seek
>> (HBASE-4465), and seek optimizations (HBASE-4433, HBASE-4434, HBASE-4469,
>> HBASE-4532)
>> - Scan: not as many optimizations here.  Mostly revolve around proper
>> usage & seek-next optimization when using filters. Don't have JIRA numbers
>> here, but probably half-dozen small tweaks were added to 0.92.
>>
>> >I don't have an increment workload (the workload either update columns on
>> >a
>> >CF or add column on a CF for the same key), so how those patch will help?
>>
>> Increment & read->update workload end up roughly picking up the same
>> optimizations.  Adding a column to an existing row is no different than
>> adding a new row as far as optimizations are concerned because there's
>> nothing to de-dupe.
>>
>> >I don't say this is a bad thing, this is just an observation from our
>> >test,
>> >HBase will slow down the flush in case too many store file are present,
>> >and
>> >will add pressure on GC and memory affecting performance.
>> >The update workload does not send all the row content for a certain key
>> so
>> >only partial data is written, in order to get all the row i presume that
>> >reading the newest Store is not enough ("all" stores need to be read
>> >collecting the more up to date field a rebuild a full row), or i'm
>> missing
>> >something?
>>
>> Reading all row columns is the same as doing a scan.  You're not doing a
>> point query if you don't specify the exact key (columns) you're looking
>> for.  Setting versions to unlimited, then getting all versions of a
>> particular ROW+COL would also be considered a scan vs a point query as far
>> as optimizations are concerned.
>>
>> >1. If i did not set a specific property for bloom filter (BF), does it
>> >means that i'm not using them (the book only refer to BF with regards to
>> >CF)?
>>
>> By default, bloom filters are disabled, so you need to enable them to get
>> the optimizations.  This is by design.  Bloom Filters trade off cache
>> space for low-overhead probabilistic queries.  Default is 8-bytes per
>> bloom entry (key) & 1% false positive rate.  You can use 'bin/hbase
>> org.apache.hadoop.hbase.io.hfile.HFile' (look at help, then -f to specify
>> a StoreFile and then use -m for meta info) to see your StoreFile's average
>> KV size.  If size(KV) == 100 bytes, then blooms use 8% of the space in
>> cache, which is better than loading the StoreFile block only to get a
>> miss.
>>
>> Whether to use a ROW or ROWCOL bloom filter depends on your write & read
>> pattern.  If you read the entire row at a time, use a ROW bloom.  If you
>> point query, ROW or ROWCOL are both options.  If you write all columns for
>> a row at the same time, definitely use a ROW bloom.  If you have a small
>> column range and you update them at different rates/times, then a ROWCOL
>> bloom filter may be more helpful.  ROWCOL is really useful if a scan query
>> for a ROW will normally return results, but a point query for a ROWCOL may
>> have a high miss rate.  A perfect example is storing unique hash-values
>> for a user on disk.  You'd use 'user' as the row & the hash as the column.
>>  Most instances, the hash won't be a duplicate, so a ROWCOL bloom would be
>> better.
>>
>> >3. How can we ensure that compaction will not suck too much I/O if we
>> >cannot control major compaction?
>>
>> TCP Congestion Control will ensure that a single TCP socket won't consume
>> too much bandwidth, so that part of compactions is automatically handled.
>> The part that you need to handle is the number of simultaneous TCP sockets
>> (currently 1 until multi-threaded compactions) & the aggregate data volume
>> transferred over time.  As I said, this is controlled by compaction.ratio.
>>  If temporary high StoreFile counts cause you to bottleneck, slight
>> latency variance is an annoyance of the current compaction algorithm but
>> the underlying problem you should be looking at solving is the system's
>> inability to filter out the unnecessary StoreFiles.
>>
>>
>
>
>
>


-- 
Mikael.S

Re: Major Compaction Concerns

Posted by Doug Meil <do...@explorysmedical.com>.
re:  "A suggestion is perhaps to take all those example/explanation and
add them
to the book for future reference."

Absolutely!  I've been watching this thread with great interest.





On 1/14/12 4:30 PM, "Mikael Sitruk" <mi...@gmail.com> wrote:

>Wow, thank you very much for all those precious explanations, pointers and
>examples. It's a lot to ingest... I will try them (at least what i can
>with
>0.90.4 (yes i'm upgrading from 0.90.1 to 0.90.4)) and keep you informed.
>BTW I'm already using compression (GZ), the current data is randomized so
>I
>don't have so much gain as you mentioned ( i think i'm around 30% only).
>It seems that BF is one of the major thing i need to look up with the
>compaction.ratio, and i need a different setting for my different CF. (one
>CF has small set of column and each update will change 50% --> ROWCOL, the
>second CF has always a new column per update --> ROW)
>I'm not keeping more than one version neither, and you wrote this is not a
>point query.
>
>A suggestion is perhaps to take all those example/explanation and add them
>to the book for future reference.
>
>Regards,
>Mikael.S
>
>
>On Sat, Jan 14, 2012 at 4:06 AM, Nicolas Spiegelberg
><ns...@fb.com>wrote:
>
>> >I'm sorry but i don't understand, of course i have a disk and network
>> >saturation and the flush stop to flush because he is waiting for
>> >compaction
>> >to finish. Since this a major compaction was triggered - all the
>> >stores (large number)  present on the disks (7 disk per RS) will be
>> >grabbed
>> >for major compact, and the I/O is affected. Network is also affected
>>since
>> >all are major compacting at the same time and replicating files on same
>> >time (1GB network).
>>
>> When you have an IO problem, there are multiple pieces at play that you
>> can adjust:
>>
>> Write: HLog, Flush, Compaction
>> Read: Point Query, Scan
>>
>> If your writes are far more than your reads, then you should relax one
>>of
>> the write pieces.
>> - HLog: You can't really adjust HLog IO outside of key compression
>> (HBASE-4608)
>> - Flush: You can adjust your compression.  None->LZO == 5x compression.
>> LZO->GZ == 2x compression.  Both are at the expense of CPU.  HBASE-4241
>> minimizes flush IO significantly in the update-heavy use case (discussed
>> this in the last email).
>> - Compaction: You can lower the compaction ratio to minimize the amount
>>of
>> rewrites over time.  That's why I suggested changing the ratio from 1.2
>>->
>> 0.25.  This gives a ~50% IO reduction (blog post on this forthcoming @
>> http://www.facebook.com/UsingHBase ).
>>
>> However, you may have a lot more reads than you think.  For example,
>>let's
>> say read:write ratio is 1:10, so significantly read dominated.  Without
>> any of the optimizations I listed in the previous email, your real read
>> ratio is multiplied by the StoreFile count (because you naively read all
>> StoreFiles).  So let say, during congestion, you have 20 StoreFiles.
>> 1*20:10 means that you're now 2:1 read dominated.  You need features to
>> reduce the number of StoreFiles you scan when the StoreFile count is
>>high.
>>
>> - Point Query: bloom filters (HBASE-1200, HBASE-2794), lazy seek
>> (HBASE-4465), and seek optimizations (HBASE-4433, HBASE-4434,
>>HBASE-4469,
>> HBASE-4532)
>> - Scan: not as many optimizations here.  Mostly revolve around proper
>> usage & seek-next optimization when using filters. Don't have JIRA
>>numbers
>> here, but probably half-dozen small tweaks were added to 0.92.
>>
>> >I don't have an increment workload (the workload either update columns
>>on
>> >a
>> >CF or add column on a CF for the same key), so how those patch will
>>help?
>>
>> Increment & read->update workload end up roughly picking up the same
>> optimizations.  Adding a column to an existing row is no different than
>> adding a new row as far as optimizations are concerned because there's
>> nothing to de-dupe.
>>
>> >I don't say this is a bad thing, this is just an observation from our
>> >test,
>> >HBase will slow down the flush in case too many store file are present,
>> >and
>> >will add pressure on GC and memory affecting performance.
>> >The update workload does not send all the row content for a certain
>>key so
>> >only partial data is written, in order to get all the row i presume
>>that
>> >reading the newest Store is not enough ("all" stores need to be read
>> >collecting the more up to date field a rebuild a full row), or i'm
>>missing
>> >something?
>>
>> Reading all row columns is the same as doing a scan.  You're not doing a
>> point query if you don't specify the exact key (columns) you're looking
>> for.  Setting versions to unlimited, then getting all versions of a
>> particular ROW+COL would also be considered a scan vs a point query as
>>far
>> as optimizations are concerned.
>>
>> >1. If i did not set a specific property for bloom filter (BF), does it
>> >means that i'm not using them (the book only refer to BF with regards
>>to
>> >CF)?
>>
>> By default, bloom filters are disabled, so you need to enable them to
>>get
>> the optimizations.  This is by design.  Bloom Filters trade off cache
>> space for low-overhead probabilistic queries.  Default is 8-bytes per
>> bloom entry (key) & 1% false positive rate.  You can use 'bin/hbase
>> org.apache.hadoop.hbase.io.hfile.HFile' (look at help, then -f to
>>specify
>> a StoreFile and then use -m for meta info) to see your StoreFile's
>>average
>> KV size.  If size(KV) == 100 bytes, then blooms use 8% of the space in
>> cache, which is better than loading the StoreFile block only to get a
>>miss.
>>
>> Whether to use a ROW or ROWCOL bloom filter depends on your write & read
>> pattern.  If you read the entire row at a time, use a ROW bloom.  If you
>> point query, ROW or ROWCOL are both options.  If you write all columns
>>for
>> a row at the same time, definitely use a ROW bloom.  If you have a small
>> column range and you update them at different rates/times, then a ROWCOL
>> bloom filter may be more helpful.  ROWCOL is really useful if a scan
>>query
>> for a ROW will normally return results, but a point query for a ROWCOL
>>may
>> have a high miss rate.  A perfect example is storing unique hash-values
>> for a user on disk.  You'd use 'user' as the row & the hash as the
>>column.
>>  Most instances, the hash won't be a duplicate, so a ROWCOL bloom would
>>be
>> better.
>>
>> >3. How can we ensure that compaction will not suck too much I/O if we
>> >cannot control major compaction?
>>
>> TCP Congestion Control will ensure that a single TCP socket won't
>>consume
>> too much bandwidth, so that part of compactions is automatically
>>handled.
>> The part that you need to handle is the number of simultaneous TCP
>>sockets
>> (currently 1 until multi-threaded compactions) & the aggregate data
>>volume
>> transferred over time.  As I said, this is controlled by
>>compaction.ratio.
>>  If temporary high StoreFile counts cause you to bottleneck, slight
>> latency variance is an annoyance of the current compaction algorithm but
>> the underlying problem you should be looking at solving is the system's
>> inability to filter out the unnecessary StoreFiles.
>>
>>



Re: Major Compaction Concerns

Posted by Mikael Sitruk <mi...@gmail.com>.
Wow, thank you very much for all those precious explanations, pointers and
examples. It's a lot to ingest... I will try them (at least what i can with
0.90.4 (yes i'm upgrading from 0.90.1 to 0.90.4)) and keep you informed.
BTW I'm already using compression (GZ), the current data is randomized so I
don't have so much gain as you mentioned ( i think i'm around 30% only).
It seems that BF is one of the major thing i need to look up with the
compaction.ratio, and i need a different setting for my different CF. (one
CF has small set of column and each update will change 50% --> ROWCOL, the
second CF has always a new column per update --> ROW)
I'm not keeping more than one version neither, and you wrote this is not a
point query.

A suggestion is perhaps to take all those example/explanation and add them
to the book for future reference.

Regards,
Mikael.S


On Sat, Jan 14, 2012 at 4:06 AM, Nicolas Spiegelberg <ns...@fb.com>wrote:

> >I'm sorry but i don't understand, of course i have a disk and network
> >saturation and the flush stop to flush because he is waiting for
> >compaction
> >to finish. Since this a major compaction was triggered - all the
> >stores (large number)  present on the disks (7 disk per RS) will be
> >grabbed
> >for major compact, and the I/O is affected. Network is also affected since
> >all are major compacting at the same time and replicating files on same
> >time (1GB network).
>
> When you have an IO problem, there are multiple pieces at play that you
> can adjust:
>
> Write: HLog, Flush, Compaction
> Read: Point Query, Scan
>
> If your writes are far more than your reads, then you should relax one of
> the write pieces.
> - HLog: You can't really adjust HLog IO outside of key compression
> (HBASE-4608)
> - Flush: You can adjust your compression.  None->LZO == 5x compression.
> LZO->GZ == 2x compression.  Both are at the expense of CPU.  HBASE-4241
> minimizes flush IO significantly in the update-heavy use case (discussed
> this in the last email).
> - Compaction: You can lower the compaction ratio to minimize the amount of
> rewrites over time.  That's why I suggested changing the ratio from 1.2 ->
> 0.25.  This gives a ~50% IO reduction (blog post on this forthcoming @
> http://www.facebook.com/UsingHBase ).
>
> However, you may have a lot more reads than you think.  For example, let's
> say read:write ratio is 1:10, so significantly read dominated.  Without
> any of the optimizations I listed in the previous email, your real read
> ratio is multiplied by the StoreFile count (because you naively read all
> StoreFiles).  So let say, during congestion, you have 20 StoreFiles.
> 1*20:10 means that you're now 2:1 read dominated.  You need features to
> reduce the number of StoreFiles you scan when the StoreFile count is high.
>
> - Point Query: bloom filters (HBASE-1200, HBASE-2794), lazy seek
> (HBASE-4465), and seek optimizations (HBASE-4433, HBASE-4434, HBASE-4469,
> HBASE-4532)
> - Scan: not as many optimizations here.  Mostly revolve around proper
> usage & seek-next optimization when using filters. Don't have JIRA numbers
> here, but probably half-dozen small tweaks were added to 0.92.
>
> >I don't have an increment workload (the workload either update columns on
> >a
> >CF or add column on a CF for the same key), so how those patch will help?
>
> Increment & read->update workload end up roughly picking up the same
> optimizations.  Adding a column to an existing row is no different than
> adding a new row as far as optimizations are concerned because there's
> nothing to de-dupe.
>
> >I don't say this is a bad thing, this is just an observation from our
> >test,
> >HBase will slow down the flush in case too many store file are present,
> >and
> >will add pressure on GC and memory affecting performance.
> >The update workload does not send all the row content for a certain key so
> >only partial data is written, in order to get all the row i presume that
> >reading the newest Store is not enough ("all" stores need to be read
> >collecting the more up to date field a rebuild a full row), or i'm missing
> >something?
>
> Reading all row columns is the same as doing a scan.  You're not doing a
> point query if you don't specify the exact key (columns) you're looking
> for.  Setting versions to unlimited, then getting all versions of a
> particular ROW+COL would also be considered a scan vs a point query as far
> as optimizations are concerned.
>
> >1. If i did not set a specific property for bloom filter (BF), does it
> >means that i'm not using them (the book only refer to BF with regards to
> >CF)?
>
> By default, bloom filters are disabled, so you need to enable them to get
> the optimizations.  This is by design.  Bloom Filters trade off cache
> space for low-overhead probabilistic queries.  Default is 8-bytes per
> bloom entry (key) & 1% false positive rate.  You can use 'bin/hbase
> org.apache.hadoop.hbase.io.hfile.HFile' (look at help, then -f to specify
> a StoreFile and then use -m for meta info) to see your StoreFile's average
> KV size.  If size(KV) == 100 bytes, then blooms use 8% of the space in
> cache, which is better than loading the StoreFile block only to get a miss.
>
> Whether to use a ROW or ROWCOL bloom filter depends on your write & read
> pattern.  If you read the entire row at a time, use a ROW bloom.  If you
> point query, ROW or ROWCOL are both options.  If you write all columns for
> a row at the same time, definitely use a ROW bloom.  If you have a small
> column range and you update them at different rates/times, then a ROWCOL
> bloom filter may be more helpful.  ROWCOL is really useful if a scan query
> for a ROW will normally return results, but a point query for a ROWCOL may
> have a high miss rate.  A perfect example is storing unique hash-values
> for a user on disk.  You'd use 'user' as the row & the hash as the column.
>  Most instances, the hash won't be a duplicate, so a ROWCOL bloom would be
> better.
>
> >3. How can we ensure that compaction will not suck too much I/O if we
> >cannot control major compaction?
>
> TCP Congestion Control will ensure that a single TCP socket won't consume
> too much bandwidth, so that part of compactions is automatically handled.
> The part that you need to handle is the number of simultaneous TCP sockets
> (currently 1 until multi-threaded compactions) & the aggregate data volume
> transferred over time.  As I said, this is controlled by compaction.ratio.
>  If temporary high StoreFile counts cause you to bottleneck, slight
> latency variance is an annoyance of the current compaction algorithm but
> the underlying problem you should be looking at solving is the system's
> inability to filter out the unnecessary StoreFiles.
>
>

Re: Major Compaction Concerns

Posted by Nicolas Spiegelberg <ns...@fb.com>.
>I'm sorry but i don't understand, of course i have a disk and network
>saturation and the flush stop to flush because he is waiting for
>compaction
>to finish. Since this a major compaction was triggered - all the
>stores (large number)  present on the disks (7 disk per RS) will be
>grabbed
>for major compact, and the I/O is affected. Network is also affected since
>all are major compacting at the same time and replicating files on same
>time (1GB network).

When you have an IO problem, there are multiple pieces at play that you
can adjust:

Write: HLog, Flush, Compaction
Read: Point Query, Scan

If your writes are far more than your reads, then you should relax one of
the write pieces.  
- HLog: You can't really adjust HLog IO outside of key compression
(HBASE-4608)
- Flush: You can adjust your compression.  None->LZO == 5x compression.
LZO->GZ == 2x compression.  Both are at the expense of CPU.  HBASE-4241
minimizes flush IO significantly in the update-heavy use case (discussed
this in the last email).
- Compaction: You can lower the compaction ratio to minimize the amount of
rewrites over time.  That's why I suggested changing the ratio from 1.2 ->
0.25.  This gives a ~50% IO reduction (blog post on this forthcoming @
http://www.facebook.com/UsingHBase ).

However, you may have a lot more reads than you think.  For example, let's
say read:write ratio is 1:10, so significantly read dominated.  Without
any of the optimizations I listed in the previous email, your real read
ratio is multiplied by the StoreFile count (because you naively read all
StoreFiles).  So let say, during congestion, you have 20 StoreFiles.
1*20:10 means that you're now 2:1 read dominated.  You need features to
reduce the number of StoreFiles you scan when the StoreFile count is high.

- Point Query: bloom filters (HBASE-1200, HBASE-2794), lazy seek
(HBASE-4465), and seek optimizations (HBASE-4433, HBASE-4434, HBASE-4469,
HBASE-4532)
- Scan: not as many optimizations here.  Mostly revolve around proper
usage & seek-next optimization when using filters. Don't have JIRA numbers
here, but probably half-dozen small tweaks were added to 0.92.

>I don't have an increment workload (the workload either update columns on
>a
>CF or add column on a CF for the same key), so how those patch will help?

Increment & read->update workload end up roughly picking up the same
optimizations.  Adding a column to an existing row is no different than
adding a new row as far as optimizations are concerned because there's
nothing to de-dupe.

>I don't say this is a bad thing, this is just an observation from our
>test,
>HBase will slow down the flush in case too many store file are present,
>and
>will add pressure on GC and memory affecting performance.
>The update workload does not send all the row content for a certain key so
>only partial data is written, in order to get all the row i presume that
>reading the newest Store is not enough ("all" stores need to be read
>collecting the more up to date field a rebuild a full row), or i'm missing
>something?

Reading all row columns is the same as doing a scan.  You're not doing a
point query if you don't specify the exact key (columns) you're looking
for.  Setting versions to unlimited, then getting all versions of a
particular ROW+COL would also be considered a scan vs a point query as far
as optimizations are concerned.

>1. If i did not set a specific property for bloom filter (BF), does it
>means that i'm not using them (the book only refer to BF with regards to
>CF)?

By default, bloom filters are disabled, so you need to enable them to get
the optimizations.  This is by design.  Bloom Filters trade off cache
space for low-overhead probabilistic queries.  Default is 8-bytes per
bloom entry (key) & 1% false positive rate.  You can use 'bin/hbase
org.apache.hadoop.hbase.io.hfile.HFile' (look at help, then -f to specify
a StoreFile and then use -m for meta info) to see your StoreFile's average
KV size.  If size(KV) == 100 bytes, then blooms use 8% of the space in
cache, which is better than loading the StoreFile block only to get a miss.

Whether to use a ROW or ROWCOL bloom filter depends on your write & read
pattern.  If you read the entire row at a time, use a ROW bloom.  If you
point query, ROW or ROWCOL are both options.  If you write all columns for
a row at the same time, definitely use a ROW bloom.  If you have a small
column range and you update them at different rates/times, then a ROWCOL
bloom filter may be more helpful.  ROWCOL is really useful if a scan query
for a ROW will normally return results, but a point query for a ROWCOL may
have a high miss rate.  A perfect example is storing unique hash-values
for a user on disk.  You'd use 'user' as the row & the hash as the column.
 Most instances, the hash won't be a duplicate, so a ROWCOL bloom would be
better.

>3. How can we ensure that compaction will not suck too much I/O if we
>cannot control major compaction?

TCP Congestion Control will ensure that a single TCP socket won't consume
too much bandwidth, so that part of compactions is automatically handled.
The part that you need to handle is the number of simultaneous TCP sockets
(currently 1 until multi-threaded compactions) & the aggregate data volume
transferred over time.  As I said, this is controlled by compaction.ratio.
 If temporary high StoreFile counts cause you to bottleneck, slight
latency variance is an annoyance of the current compaction algorithm but
the underlying problem you should be looking at solving is the system's
inability to filter out the unnecessary StoreFiles.


Re: Major Compaction Concerns

Posted by Mikael Sitruk <mi...@gmail.com>.
Hi

Nicolas - Can you point on lazy seek patch referenced? Like Ted, from the
release note i can't find "lazy seek", unless you
are referring to HBASE-4434

> Filling up the logs faster than you can flush normally indicates that you
> have disk or network saturation.  If you have an increment workload, I
> know there are a number of patches in 0.92 that will drastically reduce
> your flush size (1: read memstore before going to disk, 2: don't flush all
> versions).  You don't have a compaction problem, you have a write/read
> problem.

I'm sorry but i don't understand, of course i have a disk and network
saturation and the flush stop to flush because he is waiting for compaction
to finish. Since this a major compaction was triggered - all the
stores (large number)  present on the disks (7 disk per RS) will be grabbed
for major compact, and the I/O is affected. Network is also affected since
all are major compacting at the same time and replicating files on same
time (1GB network).
I don't have an increment workload (the workload either update columns on a
CF or add column on a CF for the same key), so how those patch will help?

> Per-storefile compactions & multi-threaded compactions were added 0.92 to
> address this problem.  However, a high StoreFile count is not necessarily
> a bad thing.  For an update workload, you only have to read the newest
> StoreFile and lazy seek optimizes your situation a lot (again 0.92).
I don't say this is a bad thing, this is just an observation from our test,
HBase will slow down the flush in case too many store file are present, and
will add pressure on GC and memory affecting performance.
The update workload does not send all the row content for a certain key so
only partial data is written, in order to get all the row i presume that
reading the newest Store is not enough ("all" stores need to be read
collecting the more up to date field a rebuild a full row), or i'm missing
something?

> There are mostly 3 different workloads that require different
> optimizations (not necessarily compaction-related):
> 1. Read old data.  Should properly use bloom filters to filter out
> StoreFiles
> 2. R+W.  Will really benefit from lazy seeks & cache on write (0.92).  Far
> more than a compaction algorithm
> 3. Write mostly.  Don't really care about compactions here.  Just don't
> want them to be sucking too much IO

1. If i did not set a specific property for bloom filter (BF), does it
means that i'm not using them (the book only refer to BF with regards to
CF)?
3. How can we ensure that compaction will not suck too much I/O if we
cannot control major compaction?

Thanks  & Regards
Mikael.S

On Fri, Jan 13, 2012 at 12:20 AM, lars hofhansl <lh...@yahoo.com> wrote:

> HBASE-4465 is not needed for correctness.
> Personally I'd rather release 0.94 sooner rather than backporting
> non-trivial patches.
>
> I realize I am guilty of this myself (see HBASE-4838... although that was
> an important correctness fix)
>
> -- Lars
>
> ________________________________
> From: Ted Yu <yu...@gmail.com>
> To: dev@hbase.apache.org
> Cc: Mikael Sitruk <mi...@gmail.com>
> Sent: Thursday, January 12, 2012 2:09 PM
> Subject: Re: Major Compaction Concerns
>
> Thanks for the tips, Nicolas.
>
> About lazy seek, if you were referring to HBASE-4465, that was only
> integrated into TRUNK and 0.89-fb.
> I was thinking about backporting it to 0.92
>
> Cheers
>
> On Thu, Jan 12, 2012 at 1:44 PM, Nicolas Spiegelberg <nspiegelberg@fb.com
> >wrote:
>
> > Mikael,
> >
> > >The system is an OLTP system, with strict latency and throughput
> > >requirements, regions are pre-splitted and throughput is controlled.
> > >
> > >The system has heavy load period for few hours, during heavy load i mean
> > >high proportion insert/update and small proportion of read.
> >
> > I'm not sure about the production status of your system, but you sound
> > like you have critical need for dozens of optimization features coming
> out
> > in 0.92 and even some trunk patches.  In particular, update speed has
> been
> > drastically improved due to lazy seek.  Although you can get incremental
> > wins with a different compaction features, you will get exponential wins
> > from looking at other features right now.
> >
> > >we fall in the memstore flush throttling (
> > >will wait 90000 ms before flushing the memstore) retaining more logs,
> > >triggering more flush that can't be flushed.... adding pressure on the
> > >system memory (memstore is not flushed on time)
> >
> > Filling up the logs faster than you can flush normally indicates that you
> > have disk or network saturation.  If you have an increment workload, I
> > know there are a number of patches in 0.92 that will drastically reduce
> > your flush size (1: read memstore before going to disk, 2: don't flush
> all
> > versions).  You don't have a compaction problem, you have a write/read
> > problem.
> >
> > In 0.92, you can try setting your compaction.ratio down (0.25 is a good
> > start) to increase the StoreFile count to slow reads but save Network IO
> > on write.  This setting is very similar to the defaults suggested in the
> > BigTable paper.  However, this is only going to cut your Network IO in
> > half.  The LevelDB or BigTable algorithm can reduce your outlier
> StoreFile
> > count, but they wouldn't be able to cut this IO volume down much either.
> >
> > >Please remember i'm on 0.90.1 so when major compaction is running minor
> is
> > >blocked, when a memstore for a column family is flushed all other
> memstore
> > >(for other) column family are also (no matter if they are smaller or
> not).
> > >As you already wrote, the best way is to manage compaction, and it is
> what
> > >i tried to do.
> >
> > Per-storefile compactions & multi-threaded compactions were added 0.92 to
> > address this problem.  However, a high StoreFile count is not necessarily
> > a bad thing.  For an update workload, you only have to read the newest
> > StoreFile and lazy seek optimizes your situation a lot (again 0.92).
> >
> > >Regarding the compaction plug-ability needs.
> > >Let suppose that the data you are inserting in different column family
> has
> > >a different pattern, for example on CF1 (column family #1) you update
> > >fields in the same row key while in CF2 you add each time new fields or
> > >CF2 has new row and older rows are never updated won't you use different
> > >algorithms for compacting these CF?
> >
> > There are mostly 3 different workloads that require different
> > optimizations (not necessarily compaction-related):
> > 1. Read old data.  Should properly use bloom filters to filter out
> > StoreFiles
> > 2. R+W.  Will really benefit from lazy seeks & cache on write (0.92).
> Far
> > more than a compaction algorithm
> > 3. Write mostly.  Don't really care about compactions here.  Just don't
> > want them to be sucking too much IO
> >
> > >Finally the schema design is guided by the ACID property of a row, we
> have
> > >2 CF only both CF holds a different volume of data even if they are
> > >Updated approximately with the same amount of data (cell updated vs cell
> > >created).
> >
> > Note that 0.90 only had row-based write atomicity.  HBASE-2856 is
> > necessary for row-based read atomicity across column families.
> >
> >
>



-- 
Mikael.S

Re: Major Compaction Concerns

Posted by lars hofhansl <lh...@yahoo.com>.
HBASE-4465 is not needed for correctness.
Personally I'd rather release 0.94 sooner rather than backporting non-trivial patches.

I realize I am guilty of this myself (see HBASE-4838... although that was an important correctness fix)

-- Lars

________________________________
From: Ted Yu <yu...@gmail.com>
To: dev@hbase.apache.org 
Cc: Mikael Sitruk <mi...@gmail.com> 
Sent: Thursday, January 12, 2012 2:09 PM
Subject: Re: Major Compaction Concerns

Thanks for the tips, Nicolas.

About lazy seek, if you were referring to HBASE-4465, that was only
integrated into TRUNK and 0.89-fb.
I was thinking about backporting it to 0.92

Cheers

On Thu, Jan 12, 2012 at 1:44 PM, Nicolas Spiegelberg <ns...@fb.com>wrote:

> Mikael,
>
> >The system is an OLTP system, with strict latency and throughput
> >requirements, regions are pre-splitted and throughput is controlled.
> >
> >The system has heavy load period for few hours, during heavy load i mean
> >high proportion insert/update and small proportion of read.
>
> I'm not sure about the production status of your system, but you sound
> like you have critical need for dozens of optimization features coming out
> in 0.92 and even some trunk patches.  In particular, update speed has been
> drastically improved due to lazy seek.  Although you can get incremental
> wins with a different compaction features, you will get exponential wins
> from looking at other features right now.
>
> >we fall in the memstore flush throttling (
> >will wait 90000 ms before flushing the memstore) retaining more logs,
> >triggering more flush that can't be flushed.... adding pressure on the
> >system memory (memstore is not flushed on time)
>
> Filling up the logs faster than you can flush normally indicates that you
> have disk or network saturation.  If you have an increment workload, I
> know there are a number of patches in 0.92 that will drastically reduce
> your flush size (1: read memstore before going to disk, 2: don't flush all
> versions).  You don't have a compaction problem, you have a write/read
> problem.
>
> In 0.92, you can try setting your compaction.ratio down (0.25 is a good
> start) to increase the StoreFile count to slow reads but save Network IO
> on write.  This setting is very similar to the defaults suggested in the
> BigTable paper.  However, this is only going to cut your Network IO in
> half.  The LevelDB or BigTable algorithm can reduce your outlier StoreFile
> count, but they wouldn't be able to cut this IO volume down much either.
>
> >Please remember i'm on 0.90.1 so when major compaction is running minor is
> >blocked, when a memstore for a column family is flushed all other memstore
> >(for other) column family are also (no matter if they are smaller or not).
> >As you already wrote, the best way is to manage compaction, and it is what
> >i tried to do.
>
> Per-storefile compactions & multi-threaded compactions were added 0.92 to
> address this problem.  However, a high StoreFile count is not necessarily
> a bad thing.  For an update workload, you only have to read the newest
> StoreFile and lazy seek optimizes your situation a lot (again 0.92).
>
> >Regarding the compaction plug-ability needs.
> >Let suppose that the data you are inserting in different column family has
> >a different pattern, for example on CF1 (column family #1) you update
> >fields in the same row key while in CF2 you add each time new fields or
> >CF2 has new row and older rows are never updated won't you use different
> >algorithms for compacting these CF?
>
> There are mostly 3 different workloads that require different
> optimizations (not necessarily compaction-related):
> 1. Read old data.  Should properly use bloom filters to filter out
> StoreFiles
> 2. R+W.  Will really benefit from lazy seeks & cache on write (0.92).  Far
> more than a compaction algorithm
> 3. Write mostly.  Don't really care about compactions here.  Just don't
> want them to be sucking too much IO
>
> >Finally the schema design is guided by the ACID property of a row, we have
> >2 CF only both CF holds a different volume of data even if they are
> >Updated approximately with the same amount of data (cell updated vs cell
> >created).
>
> Note that 0.90 only had row-based write atomicity.  HBASE-2856 is
> necessary for row-based read atomicity across column families.
>
>

Re: Major Compaction Concerns

Posted by Ted Yu <yu...@gmail.com>.
Thanks for the tips, Nicolas.

About lazy seek, if you were referring to HBASE-4465, that was only
integrated into TRUNK and 0.89-fb.
I was thinking about backporting it to 0.92

Cheers

On Thu, Jan 12, 2012 at 1:44 PM, Nicolas Spiegelberg <ns...@fb.com>wrote:

> Mikael,
>
> >The system is an OLTP system, with strict latency and throughput
> >requirements, regions are pre-splitted and throughput is controlled.
> >
> >The system has heavy load period for few hours, during heavy load i mean
> >high proportion insert/update and small proportion of read.
>
> I'm not sure about the production status of your system, but you sound
> like you have critical need for dozens of optimization features coming out
> in 0.92 and even some trunk patches.  In particular, update speed has been
> drastically improved due to lazy seek.  Although you can get incremental
> wins with a different compaction features, you will get exponential wins
> from looking at other features right now.
>
> >we fall in the memstore flush throttling (
> >will wait 90000 ms before flushing the memstore) retaining more logs,
> >triggering more flush that can't be flushed.... adding pressure on the
> >system memory (memstore is not flushed on time)
>
> Filling up the logs faster than you can flush normally indicates that you
> have disk or network saturation.  If you have an increment workload, I
> know there are a number of patches in 0.92 that will drastically reduce
> your flush size (1: read memstore before going to disk, 2: don't flush all
> versions).  You don't have a compaction problem, you have a write/read
> problem.
>
> In 0.92, you can try setting your compaction.ratio down (0.25 is a good
> start) to increase the StoreFile count to slow reads but save Network IO
> on write.  This setting is very similar to the defaults suggested in the
> BigTable paper.  However, this is only going to cut your Network IO in
> half.  The LevelDB or BigTable algorithm can reduce your outlier StoreFile
> count, but they wouldn't be able to cut this IO volume down much either.
>
> >Please remember i'm on 0.90.1 so when major compaction is running minor is
> >blocked, when a memstore for a column family is flushed all other memstore
> >(for other) column family are also (no matter if they are smaller or not).
> >As you already wrote, the best way is to manage compaction, and it is what
> >i tried to do.
>
> Per-storefile compactions & multi-threaded compactions were added 0.92 to
> address this problem.  However, a high StoreFile count is not necessarily
> a bad thing.  For an update workload, you only have to read the newest
> StoreFile and lazy seek optimizes your situation a lot (again 0.92).
>
> >Regarding the compaction plug-ability needs.
> >Let suppose that the data you are inserting in different column family has
> >a different pattern, for example on CF1 (column family #1) you update
> >fields in the same row key while in CF2 you add each time new fields or
> >CF2 has new row and older rows are never updated won't you use different
> >algorithms for compacting these CF?
>
> There are mostly 3 different workloads that require different
> optimizations (not necessarily compaction-related):
> 1. Read old data.  Should properly use bloom filters to filter out
> StoreFiles
> 2. R+W.  Will really benefit from lazy seeks & cache on write (0.92).  Far
> more than a compaction algorithm
> 3. Write mostly.  Don't really care about compactions here.  Just don't
> want them to be sucking too much IO
>
> >Finally the schema design is guided by the ACID property of a row, we have
> >2 CF only both CF holds a different volume of data even if they are
> >Updated approximately with the same amount of data (cell updated vs cell
> >created).
>
> Note that 0.90 only had row-based write atomicity.  HBASE-2856 is
> necessary for row-based read atomicity across column families.
>
>

Re: Major Compaction Concerns

Posted by Nicolas Spiegelberg <ns...@fb.com>.
Mikael,

>The system is an OLTP system, with strict latency and throughput
>requirements, regions are pre-splitted and throughput is controlled.
>
>The system has heavy load period for few hours, during heavy load i mean
>high proportion insert/update and small proportion of read.

I'm not sure about the production status of your system, but you sound
like you have critical need for dozens of optimization features coming out
in 0.92 and even some trunk patches.  In particular, update speed has been
drastically improved due to lazy seek.  Although you can get incremental
wins with a different compaction features, you will get exponential wins
from looking at other features right now.

>we fall in the memstore flush throttling (
>will wait 90000 ms before flushing the memstore) retaining more logs,
>triggering more flush that can't be flushed.... adding pressure on the
>system memory (memstore is not flushed on time)

Filling up the logs faster than you can flush normally indicates that you
have disk or network saturation.  If you have an increment workload, I
know there are a number of patches in 0.92 that will drastically reduce
your flush size (1: read memstore before going to disk, 2: don't flush all
versions).  You don't have a compaction problem, you have a write/read
problem.

In 0.92, you can try setting your compaction.ratio down (0.25 is a good
start) to increase the StoreFile count to slow reads but save Network IO
on write.  This setting is very similar to the defaults suggested in the
BigTable paper.  However, this is only going to cut your Network IO in
half.  The LevelDB or BigTable algorithm can reduce your outlier StoreFile
count, but they wouldn't be able to cut this IO volume down much either.

>Please remember i'm on 0.90.1 so when major compaction is running minor is
>blocked, when a memstore for a column family is flushed all other memstore
>(for other) column family are also (no matter if they are smaller or not).
>As you already wrote, the best way is to manage compaction, and it is what
>i tried to do.

Per-storefile compactions & multi-threaded compactions were added 0.92 to
address this problem.  However, a high StoreFile count is not necessarily
a bad thing.  For an update workload, you only have to read the newest
StoreFile and lazy seek optimizes your situation a lot (again 0.92).

>Regarding the compaction plug-ability needs.
>Let suppose that the data you are inserting in different column family has
>a different pattern, for example on CF1 (column family #1) you update
>fields in the same row key while in CF2 you add each time new fields or
>CF2 has new row and older rows are never updated won't you use different
>algorithms for compacting these CF?

There are mostly 3 different workloads that require different
optimizations (not necessarily compaction-related):
1. Read old data.  Should properly use bloom filters to filter out
StoreFiles
2. R+W.  Will really benefit from lazy seeks & cache on write (0.92).  Far
more than a compaction algorithm
3. Write mostly.  Don't really care about compactions here.  Just don't
want them to be sucking too much IO

>Finally the schema design is guided by the ACID property of a row, we have
>2 CF only both CF holds a different volume of data even if they are
>Updated approximately with the same amount of data (cell updated vs cell
>created).

Note that 0.90 only had row-based write atomicity.  HBASE-2856 is
necessary for row-based read atomicity across column families.


Re: Major Compaction Concerns

Posted by Mikael Sitruk <mi...@gmail.com>.
Hi Nicolas

First thanks for the information and the pointers, second nice to have a
discussion with the compaction expert :-)

It seems that version 0.90.1 which I'm using is very different in the
compaction management than the current release or the planned 0.92 release.
The 3 triggers are clear but according to the code the majorCompaction
indicator is switched back to true because the number of files. The result
is that major compaction are running at a time this is less adequate.

Let me explain a little bit on the use case and my problem...

The system is an OLTP system, with strict latency and throughput
requirements, regions are pre-splitted and throughput is controlled.

The system has heavy load period for few hours, during heavy load i mean
high proportion insert/update and small proportion of read.

According to the default parameters of log rolling, log size and compaction
thresholds the system suffer from performance problem due to the following:

Log are being flushed every 30 secs (according to our load), and very soon
the memstore flush occur in order to clean the logs, nevertheless under
this heavy load a lot of memstore are created on the FS and will trigger a
major compaction (due to the number of file reached, the third option).
Since the volume of the store is large (>1G) it will take more or less
1'40'' to handle a single region compaction. I the meantime other store
files are created. As a result we fall in the memstore flush throttling (
will wait 90000 ms before flushing the memstore) retaining more logs,
triggering more flush that can't be flushed.... adding pressure on the
system memory (memstore is not flushed on time)
Since the cluster his uniformly loaded when compaction occurs in one RS it
will happen on all of them also, adding network traffic (and being
saturated)
As a result we have a degradation in performance!

Please remember i'm on 0.90.1 so when major compaction is running minor is
blocked, when a memstore for a column family is flushed all other memstore
(for other) column family are also (no matter if they are smaller or not).
As you already wrote, the best way is to manage compaction, and it is what
i tried to do.

Regarding the compaction plug-ability needs.
Let suppose that the data you are inserting in different column family has
a different pattern, for example on CF1 (column family #1) you update
fields in the same row key while in CF2 you add each time new fields or CF2
has new row and older rows are never updated won't you use different
algorithms for compacting these CF? one if merging the records cleaning
older version the other one is just appending the field to the record.
What i'm trying to say is that different approach can be used allowing
better needs to the data profile/application profile.

Regarding minor vs major compaction
isn't HBASE-3797 all about this? But support they work in similar way
(beside TTL and tombstones) a major compaction will work on all the store
files so it can make a single store by the end (per CF), minor will work
only up to an upper limit (as far as understand) of files configured. So if
you have a lot of storefiles, a major compaction will take longer to finish
(and will block memstore flush for longer period) than a compaction working
only on a partial set of the files.

Regarding the queue size - the behavior is understood, but if you want to
make a utility to major compact regions one after another (not all at the
same time) you need to know when a region/the server is under compaction or
not. So either a new indicator is needed or a new queue metric is needed
(like compactingFileSize).

Finally the schema design is guided by the ACID property of a row, we have
2 CF only both CF holds a different volume of data even if they are updated
approximately with the same amount of data (cell updated vs cell created).
Regarding config param, i'm sure they are not well tuned yet but they will
not resolve all the problems...

Regards,

Mikael.S

On Mon, Jan 9, 2012 at 9:42 PM, Nicolas Spiegelberg <ns...@fb.com>wrote:

> Significant compaction JIRAs:
>  - HBASE-2462 : original formulation of current compaction algorithm
>  - HBASE-3209 : implementation
>  - HBASE-1476 : multithreaded compactions
>  - HBASE-3797 : storefile-based compaction selection
>
>
> On 1/9/12 11:37 AM, "Ted Yu" <yu...@gmail.com> wrote:
>
> >Nicolas:
> >Thanks for your insight.
> >
> >Can you point Mikael to a few of the JIRAs where algorithm mentioned in #1
> >was implemented ?
> >
> >On Mon, Jan 9, 2012 at 10:55 AM, Nicolas Spiegelberg
> ><ns...@fb.com>wrote:
> >
> >> Mikael,
> >>
> >> Hi, I wrote the current compaction algorithm, so I should be able to
> >> answer most questions that you have about the feature.  It sounds like
> >> you're creating quite a task list of work to do, but I don't understand
> >> what your use case is so a lot of that work may be not be critical and
> >>you
> >> can leverage existing functionality.  A better description of your
> >>system
> >> requirements is a must to getting a good solution.
> >>
> >> 1. Major compactions are triggered by 3 methods: user issued, timed, and
> >> size-based.  You are probably hitting size-based compactions where your
> >> config is disabling time-based compactions.  Minor compactions are
> >>issued
> >> on a size-based threshold.  The algorithm sees if sum(file[0:i] *
> >>ratio) >
> >> file[i+1] and includes file[0:i+1] if so.  This is a reverse iteration,
> >>so
> >> the highest 'i' value is used.  If all files match, then you can remove
> >> delete markers [which is the difference between a major and minor
> >> compaction].  Major compactions aren't a bad or time-intensive thing,
> >>it's
> >> just delete marker removal.
> >>
> >> As a note, we use timed majors in an OLTP production environment.  They
> >> are less useful if you're doing bulk imports or have an OLAP environment
> >> where you're either running a read-intensive test or the cluster is
> >>idle.
> >> In that case, it's definitely best to disable compactions and run them
> >> when you're not using the cluster very much.
> >>
> >> 2. See HBASE-4418 for showing all configuration options in the Web UI.
> >> This is in 0.92 however.
> >>
> >> 4. The compaction queue shows compactions that are waiting to happen.
> >>If
> >> you invoke a compaction and the queue is empty, the thread will
> >> immediately pick up your request and the queue will remain empty.
> >>
> >> 8. A patch for pluggable compactions had been thrown up in the past.  It
> >> was not well-tested and the compaction algorithm was undergoing major
> >> design changes at the time that clashed with the patch.  I think it's
> >>been
> >> a low priority because there are many other ways to get big performance
> >> wins from HBase outside of pluggable compactions.  Most people don't
> >> understand how to optimize the current algorithm, which is well-known
> >> (very similar to BigTable's).  I think bigger wins can come from
> >>correctly
> >> laying out a good schema and understanding the config knobs currently at
> >> our disposal.
> >>
> >>
> >>
> >> On 1/8/12 7:25 AM, "Mikael Sitruk" <mi...@gmail.com> wrote:
> >>
> >> >Hi
> >> >
> >> >
> >> >
> >> >I have some concern regarding major compactions below...
> >> >
> >> >
> >> >   1. According to best practices from the mailing list and from the
> >>book,
> >> >   automatic major compaction should be disabled. This can be done by
> >> >setting
> >> >   the property Œhbase.hregion.majorcompaction¹ to Œ0¹. Neverhteless
> >>even
> >> >   after having doing this I STILL see ³major compaction² messages in
> >> >logs.
> >> >   therefore it is unclear how can I manage major compactions. (The
> >> >system has
> >> >   heavy insert - uniformly on the cluster, and major compaction affect
> >> >the
> >> >   performance of the system).
> >> >   If I'm not wrong it seems from the code that: even if not requested
> >>and
> >> >   even if the indicator is set to '0' (no automatic major compaction),
> >> >major
> >> >   compaction can be triggered by the code in case all store files are
> >> >   candidate for a compaction (from Store.compact(final boolean
> >> >forceMajor)).
> >> >   Shouldn't the code add a condition that automatic major compaction
> >>is
> >> >   disabled??
> >> >
> >> >   2. I tried to check the parameter  Œhbase.hregion.majorcompaction¹
> >>at
> >> >   runtime using several approaches - to validate that the server
> >>indeed
> >> >   loaded the parameter.
> >> >
> >> >a. Using a connection created from local config
> >> >
> >> >*conn = (HConnection) HConnectionManager.getConnection(m_hbConfig);*
> >> >
> >> >*conn.getConfiguration().getString(³hbase.hregion.majorcompaction²)*
> >> >
> >> >returns the parameter from local config and not from cluster. Is it a
> >>bug?
> >> >If I set the property via the configuration shouldn¹t all the cluster
> >>be
> >> >aware of? (supposing that the connection indeed connected to the
> >>cluster)
> >> >
> >> >b.  fetching the property from the table descriptor
> >> >
> >> >*HTableDescriptor hTableDescriptor =
> >> >conn.getHTableDescriptor(Bytes.toBytes("my table"));*
> >> >
> >> >*hTableDescriptor.getValue("hbase.hregion.majorcompaction")*
> >> >
> >> >This will returns the default parameter value (1 day) not the parameter
> >> >from the configuration (on the cluster). It seems to be a bug, isn¹t
> >>it?
> >> >(the parameter from the config, should be the default if not set at the
> >> >table level)
> >> >
> >> >c. The only way I could set the parameter to 0 and really see it is via
> >> >the
> >> >Admin API, updating the table descriptor or the column descriptor. Now
> >>I
> >> >could see the parameter on the web UI. So is it the only way to set
> >> >correctly the parameter? If setting the parameter via the configuration
> >> >file, shouldn¹t the webUI show this on any table created?
> >> >
> >> >d. I tried also to setup the parameter via hbase shell but setting such
> >> >properties is not supported. (do you plan to add such support via the
> >> >shell?)
> >> >
> >> >e. Generally is it possible to get via API the configuration used by
> >>the
> >> >servers? (at cluster/server level)
> >> >
> >> >    3.  I ran both major compaction  requests from the shell or from
> >>API
> >> >but since both are async there is no progress indication. Neither the
> >>JMX
> >> >nor the Web will help here since you don¹t know if a compaction task is
> >> >running. Tailling the logs is not an efficient way to do this neither.
> >>The
> >> >point is that I would like to automate the process and avoid compaction
> >> >storm. So I want to do that region, region, but if I don¹t know when a
> >> >compaction started/ended I can¹t automate it.
> >> >
> >> >4.       In case there is no compaction files in queue (but still you
> >>have
> >> >more than 1 storefile per store e.g. minor compaction just finished)
> >>then
> >> >invoking major_compact will indeed decrease the number of store files,
> >>but
> >> >the compaction queue will remain to 0 during the compaction task
> >> >(shouldn¹t
> >> >the compaction queue increase by the number of file to compact and be
> >> >reduced when the task ended?)
> >> >
> >> >
> >> >5.       I saw already HBASE-3965 for getting status of major
> >>compaction,
> >> >nevertheless it has be removed from 0.92, is it possible to put it
> >>back?
> >> >Even sooner than 0.92?
> >> >
> >> >6.       In case a compaction (major) is running it seems there is no
> >>way
> >> >to stop-it. Do you plan to add such feature?
> >> >
> >> >7.       Do you plan to add functionality via JMX (starting/stopping
> >> >compaction, splitting....)
> >> >
> >> >8.       Finally there were some request for allowing custom
> >>compaction,
> >> >part of this was given via the RegionObserver in HBASE-2001,
> >>nevertheless
> >> >do you consider adding support for custom compaction (providing real
> >> >pluggable compaction stategy not just observer)?
> >> >
> >> >
> >> >Regards,
> >> >Mikael.S
> >>
> >>
>
>

Re: Major Compaction Concerns

Posted by Nicolas Spiegelberg <ns...@fb.com>.
Significant compaction JIRAs:
 - HBASE-2462 : original formulation of current compaction algorithm
 - HBASE-3209 : implementation
 - HBASE-1476 : multithreaded compactions
 - HBASE-3797 : storefile-based compaction selection


On 1/9/12 11:37 AM, "Ted Yu" <yu...@gmail.com> wrote:

>Nicolas:
>Thanks for your insight.
>
>Can you point Mikael to a few of the JIRAs where algorithm mentioned in #1
>was implemented ?
>
>On Mon, Jan 9, 2012 at 10:55 AM, Nicolas Spiegelberg
><ns...@fb.com>wrote:
>
>> Mikael,
>>
>> Hi, I wrote the current compaction algorithm, so I should be able to
>> answer most questions that you have about the feature.  It sounds like
>> you're creating quite a task list of work to do, but I don't understand
>> what your use case is so a lot of that work may be not be critical and
>>you
>> can leverage existing functionality.  A better description of your
>>system
>> requirements is a must to getting a good solution.
>>
>> 1. Major compactions are triggered by 3 methods: user issued, timed, and
>> size-based.  You are probably hitting size-based compactions where your
>> config is disabling time-based compactions.  Minor compactions are
>>issued
>> on a size-based threshold.  The algorithm sees if sum(file[0:i] *
>>ratio) >
>> file[i+1] and includes file[0:i+1] if so.  This is a reverse iteration,
>>so
>> the highest 'i' value is used.  If all files match, then you can remove
>> delete markers [which is the difference between a major and minor
>> compaction].  Major compactions aren't a bad or time-intensive thing,
>>it's
>> just delete marker removal.
>>
>> As a note, we use timed majors in an OLTP production environment.  They
>> are less useful if you're doing bulk imports or have an OLAP environment
>> where you're either running a read-intensive test or the cluster is
>>idle.
>> In that case, it's definitely best to disable compactions and run them
>> when you're not using the cluster very much.
>>
>> 2. See HBASE-4418 for showing all configuration options in the Web UI.
>> This is in 0.92 however.
>>
>> 4. The compaction queue shows compactions that are waiting to happen.
>>If
>> you invoke a compaction and the queue is empty, the thread will
>> immediately pick up your request and the queue will remain empty.
>>
>> 8. A patch for pluggable compactions had been thrown up in the past.  It
>> was not well-tested and the compaction algorithm was undergoing major
>> design changes at the time that clashed with the patch.  I think it's
>>been
>> a low priority because there are many other ways to get big performance
>> wins from HBase outside of pluggable compactions.  Most people don't
>> understand how to optimize the current algorithm, which is well-known
>> (very similar to BigTable's).  I think bigger wins can come from
>>correctly
>> laying out a good schema and understanding the config knobs currently at
>> our disposal.
>>
>>
>>
>> On 1/8/12 7:25 AM, "Mikael Sitruk" <mi...@gmail.com> wrote:
>>
>> >Hi
>> >
>> >
>> >
>> >I have some concern regarding major compactions below...
>> >
>> >
>> >   1. According to best practices from the mailing list and from the
>>book,
>> >   automatic major compaction should be disabled. This can be done by
>> >setting
>> >   the property Œhbase.hregion.majorcompaction¹ to Œ0¹. Neverhteless
>>even
>> >   after having doing this I STILL see ³major compaction² messages in
>> >logs.
>> >   therefore it is unclear how can I manage major compactions. (The
>> >system has
>> >   heavy insert - uniformly on the cluster, and major compaction affect
>> >the
>> >   performance of the system).
>> >   If I'm not wrong it seems from the code that: even if not requested
>>and
>> >   even if the indicator is set to '0' (no automatic major compaction),
>> >major
>> >   compaction can be triggered by the code in case all store files are
>> >   candidate for a compaction (from Store.compact(final boolean
>> >forceMajor)).
>> >   Shouldn't the code add a condition that automatic major compaction
>>is
>> >   disabled??
>> >
>> >   2. I tried to check the parameter  Œhbase.hregion.majorcompaction¹
>>at
>> >   runtime using several approaches - to validate that the server
>>indeed
>> >   loaded the parameter.
>> >
>> >a. Using a connection created from local config
>> >
>> >*conn = (HConnection) HConnectionManager.getConnection(m_hbConfig);*
>> >
>> >*conn.getConfiguration().getString(³hbase.hregion.majorcompaction²)*
>> >
>> >returns the parameter from local config and not from cluster. Is it a
>>bug?
>> >If I set the property via the configuration shouldn¹t all the cluster
>>be
>> >aware of? (supposing that the connection indeed connected to the
>>cluster)
>> >
>> >b.  fetching the property from the table descriptor
>> >
>> >*HTableDescriptor hTableDescriptor =
>> >conn.getHTableDescriptor(Bytes.toBytes("my table"));*
>> >
>> >*hTableDescriptor.getValue("hbase.hregion.majorcompaction")*
>> >
>> >This will returns the default parameter value (1 day) not the parameter
>> >from the configuration (on the cluster). It seems to be a bug, isn¹t
>>it?
>> >(the parameter from the config, should be the default if not set at the
>> >table level)
>> >
>> >c. The only way I could set the parameter to 0 and really see it is via
>> >the
>> >Admin API, updating the table descriptor or the column descriptor. Now
>>I
>> >could see the parameter on the web UI. So is it the only way to set
>> >correctly the parameter? If setting the parameter via the configuration
>> >file, shouldn¹t the webUI show this on any table created?
>> >
>> >d. I tried also to setup the parameter via hbase shell but setting such
>> >properties is not supported. (do you plan to add such support via the
>> >shell?)
>> >
>> >e. Generally is it possible to get via API the configuration used by
>>the
>> >servers? (at cluster/server level)
>> >
>> >    3.  I ran both major compaction  requests from the shell or from
>>API
>> >but since both are async there is no progress indication. Neither the
>>JMX
>> >nor the Web will help here since you don¹t know if a compaction task is
>> >running. Tailling the logs is not an efficient way to do this neither.
>>The
>> >point is that I would like to automate the process and avoid compaction
>> >storm. So I want to do that region, region, but if I don¹t know when a
>> >compaction started/ended I can¹t automate it.
>> >
>> >4.       In case there is no compaction files in queue (but still you
>>have
>> >more than 1 storefile per store e.g. minor compaction just finished)
>>then
>> >invoking major_compact will indeed decrease the number of store files,
>>but
>> >the compaction queue will remain to 0 during the compaction task
>> >(shouldn¹t
>> >the compaction queue increase by the number of file to compact and be
>> >reduced when the task ended?)
>> >
>> >
>> >5.       I saw already HBASE-3965 for getting status of major
>>compaction,
>> >nevertheless it has be removed from 0.92, is it possible to put it
>>back?
>> >Even sooner than 0.92?
>> >
>> >6.       In case a compaction (major) is running it seems there is no
>>way
>> >to stop-it. Do you plan to add such feature?
>> >
>> >7.       Do you plan to add functionality via JMX (starting/stopping
>> >compaction, splitting....)
>> >
>> >8.       Finally there were some request for allowing custom
>>compaction,
>> >part of this was given via the RegionObserver in HBASE-2001,
>>nevertheless
>> >do you consider adding support for custom compaction (providing real
>> >pluggable compaction stategy not just observer)?
>> >
>> >
>> >Regards,
>> >Mikael.S
>>
>>


Re: Major Compaction Concerns

Posted by Ted Yu <yu...@gmail.com>.
Nicolas:
Thanks for your insight.

Can you point Mikael to a few of the JIRAs where algorithm mentioned in #1
was implemented ?

On Mon, Jan 9, 2012 at 10:55 AM, Nicolas Spiegelberg <ns...@fb.com>wrote:

> Mikael,
>
> Hi, I wrote the current compaction algorithm, so I should be able to
> answer most questions that you have about the feature.  It sounds like
> you're creating quite a task list of work to do, but I don't understand
> what your use case is so a lot of that work may be not be critical and you
> can leverage existing functionality.  A better description of your system
> requirements is a must to getting a good solution.
>
> 1. Major compactions are triggered by 3 methods: user issued, timed, and
> size-based.  You are probably hitting size-based compactions where your
> config is disabling time-based compactions.  Minor compactions are issued
> on a size-based threshold.  The algorithm sees if sum(file[0:i] * ratio) >
> file[i+1] and includes file[0:i+1] if so.  This is a reverse iteration, so
> the highest 'i' value is used.  If all files match, then you can remove
> delete markers [which is the difference between a major and minor
> compaction].  Major compactions aren't a bad or time-intensive thing, it's
> just delete marker removal.
>
> As a note, we use timed majors in an OLTP production environment.  They
> are less useful if you're doing bulk imports or have an OLAP environment
> where you're either running a read-intensive test or the cluster is idle.
> In that case, it's definitely best to disable compactions and run them
> when you're not using the cluster very much.
>
> 2. See HBASE-4418 for showing all configuration options in the Web UI.
> This is in 0.92 however.
>
> 4. The compaction queue shows compactions that are waiting to happen.  If
> you invoke a compaction and the queue is empty, the thread will
> immediately pick up your request and the queue will remain empty.
>
> 8. A patch for pluggable compactions had been thrown up in the past.  It
> was not well-tested and the compaction algorithm was undergoing major
> design changes at the time that clashed with the patch.  I think it's been
> a low priority because there are many other ways to get big performance
> wins from HBase outside of pluggable compactions.  Most people don't
> understand how to optimize the current algorithm, which is well-known
> (very similar to BigTable's).  I think bigger wins can come from correctly
> laying out a good schema and understanding the config knobs currently at
> our disposal.
>
>
>
> On 1/8/12 7:25 AM, "Mikael Sitruk" <mi...@gmail.com> wrote:
>
> >Hi
> >
> >
> >
> >I have some concern regarding major compactions below...
> >
> >
> >   1. According to best practices from the mailing list and from the book,
> >   automatic major compaction should be disabled. This can be done by
> >setting
> >   the property Œhbase.hregion.majorcompaction¹ to Œ0¹. Neverhteless even
> >   after having doing this I STILL see ³major compaction² messages in
> >logs.
> >   therefore it is unclear how can I manage major compactions. (The
> >system has
> >   heavy insert - uniformly on the cluster, and major compaction affect
> >the
> >   performance of the system).
> >   If I'm not wrong it seems from the code that: even if not requested and
> >   even if the indicator is set to '0' (no automatic major compaction),
> >major
> >   compaction can be triggered by the code in case all store files are
> >   candidate for a compaction (from Store.compact(final boolean
> >forceMajor)).
> >   Shouldn't the code add a condition that automatic major compaction is
> >   disabled??
> >
> >   2. I tried to check the parameter  Œhbase.hregion.majorcompaction¹  at
> >   runtime using several approaches - to validate that the server indeed
> >   loaded the parameter.
> >
> >a. Using a connection created from local config
> >
> >*conn = (HConnection) HConnectionManager.getConnection(m_hbConfig);*
> >
> >*conn.getConfiguration().getString(³hbase.hregion.majorcompaction²)*
> >
> >returns the parameter from local config and not from cluster. Is it a bug?
> >If I set the property via the configuration shouldn¹t all the cluster be
> >aware of? (supposing that the connection indeed connected to the cluster)
> >
> >b.  fetching the property from the table descriptor
> >
> >*HTableDescriptor hTableDescriptor =
> >conn.getHTableDescriptor(Bytes.toBytes("my table"));*
> >
> >*hTableDescriptor.getValue("hbase.hregion.majorcompaction")*
> >
> >This will returns the default parameter value (1 day) not the parameter
> >from the configuration (on the cluster). It seems to be a bug, isn¹t it?
> >(the parameter from the config, should be the default if not set at the
> >table level)
> >
> >c. The only way I could set the parameter to 0 and really see it is via
> >the
> >Admin API, updating the table descriptor or the column descriptor. Now I
> >could see the parameter on the web UI. So is it the only way to set
> >correctly the parameter? If setting the parameter via the configuration
> >file, shouldn¹t the webUI show this on any table created?
> >
> >d. I tried also to setup the parameter via hbase shell but setting such
> >properties is not supported. (do you plan to add such support via the
> >shell?)
> >
> >e. Generally is it possible to get via API the configuration used by the
> >servers? (at cluster/server level)
> >
> >    3.  I ran both major compaction  requests from the shell or from API
> >but since both are async there is no progress indication. Neither the JMX
> >nor the Web will help here since you don¹t know if a compaction task is
> >running. Tailling the logs is not an efficient way to do this neither. The
> >point is that I would like to automate the process and avoid compaction
> >storm. So I want to do that region, region, but if I don¹t know when a
> >compaction started/ended I can¹t automate it.
> >
> >4.       In case there is no compaction files in queue (but still you have
> >more than 1 storefile per store e.g. minor compaction just finished) then
> >invoking major_compact will indeed decrease the number of store files, but
> >the compaction queue will remain to 0 during the compaction task
> >(shouldn¹t
> >the compaction queue increase by the number of file to compact and be
> >reduced when the task ended?)
> >
> >
> >5.       I saw already HBASE-3965 for getting status of major compaction,
> >nevertheless it has be removed from 0.92, is it possible to put it back?
> >Even sooner than 0.92?
> >
> >6.       In case a compaction (major) is running it seems there is no way
> >to stop-it. Do you plan to add such feature?
> >
> >7.       Do you plan to add functionality via JMX (starting/stopping
> >compaction, splitting....)
> >
> >8.       Finally there were some request for allowing custom compaction,
> >part of this was given via the RegionObserver in HBASE-2001, nevertheless
> >do you consider adding support for custom compaction (providing real
> >pluggable compaction stategy not just observer)?
> >
> >
> >Regards,
> >Mikael.S
>
>

Re: Major Compaction Concerns

Posted by Nicolas Spiegelberg <ns...@fb.com>.
Mikael,

Hi, I wrote the current compaction algorithm, so I should be able to
answer most questions that you have about the feature.  It sounds like
you're creating quite a task list of work to do, but I don't understand
what your use case is so a lot of that work may be not be critical and you
can leverage existing functionality.  A better description of your system
requirements is a must to getting a good solution.

1. Major compactions are triggered by 3 methods: user issued, timed, and
size-based.  You are probably hitting size-based compactions where your
config is disabling time-based compactions.  Minor compactions are issued
on a size-based threshold.  The algorithm sees if sum(file[0:i] * ratio) >
file[i+1] and includes file[0:i+1] if so.  This is a reverse iteration, so
the highest 'i' value is used.  If all files match, then you can remove
delete markers [which is the difference between a major and minor
compaction].  Major compactions aren't a bad or time-intensive thing, it's
just delete marker removal.

As a note, we use timed majors in an OLTP production environment.  They
are less useful if you're doing bulk imports or have an OLAP environment
where you're either running a read-intensive test or the cluster is idle.
In that case, it's definitely best to disable compactions and run them
when you're not using the cluster very much.

2. See HBASE-4418 for showing all configuration options in the Web UI.
This is in 0.92 however.

4. The compaction queue shows compactions that are waiting to happen.  If
you invoke a compaction and the queue is empty, the thread will
immediately pick up your request and the queue will remain empty.

8. A patch for pluggable compactions had been thrown up in the past.  It
was not well-tested and the compaction algorithm was undergoing major
design changes at the time that clashed with the patch.  I think it's been
a low priority because there are many other ways to get big performance
wins from HBase outside of pluggable compactions.  Most people don't
understand how to optimize the current algorithm, which is well-known
(very similar to BigTable's).  I think bigger wins can come from correctly
laying out a good schema and understanding the config knobs currently at
our disposal.



On 1/8/12 7:25 AM, "Mikael Sitruk" <mi...@gmail.com> wrote:

>Hi
>
>
>
>I have some concern regarding major compactions below...
>
>
>   1. According to best practices from the mailing list and from the book,
>   automatic major compaction should be disabled. This can be done by
>setting
>   the property Œhbase.hregion.majorcompaction¹ to Œ0¹. Neverhteless even
>   after having doing this I STILL see ³major compaction² messages in
>logs.
>   therefore it is unclear how can I manage major compactions. (The
>system has
>   heavy insert - uniformly on the cluster, and major compaction affect
>the
>   performance of the system).
>   If I'm not wrong it seems from the code that: even if not requested and
>   even if the indicator is set to '0' (no automatic major compaction),
>major
>   compaction can be triggered by the code in case all store files are
>   candidate for a compaction (from Store.compact(final boolean
>forceMajor)).
>   Shouldn't the code add a condition that automatic major compaction is
>   disabled??
>
>   2. I tried to check the parameter  Œhbase.hregion.majorcompaction¹  at
>   runtime using several approaches - to validate that the server indeed
>   loaded the parameter.
>
>a. Using a connection created from local config
>
>*conn = (HConnection) HConnectionManager.getConnection(m_hbConfig);*
>
>*conn.getConfiguration().getString(³hbase.hregion.majorcompaction²)*
>
>returns the parameter from local config and not from cluster. Is it a bug?
>If I set the property via the configuration shouldn¹t all the cluster be
>aware of? (supposing that the connection indeed connected to the cluster)
>
>b.  fetching the property from the table descriptor
>
>*HTableDescriptor hTableDescriptor =
>conn.getHTableDescriptor(Bytes.toBytes("my table"));*
>
>*hTableDescriptor.getValue("hbase.hregion.majorcompaction")*
>
>This will returns the default parameter value (1 day) not the parameter
>from the configuration (on the cluster). It seems to be a bug, isn¹t it?
>(the parameter from the config, should be the default if not set at the
>table level)
>
>c. The only way I could set the parameter to 0 and really see it is via
>the
>Admin API, updating the table descriptor or the column descriptor. Now I
>could see the parameter on the web UI. So is it the only way to set
>correctly the parameter? If setting the parameter via the configuration
>file, shouldn¹t the webUI show this on any table created?
>
>d. I tried also to setup the parameter via hbase shell but setting such
>properties is not supported. (do you plan to add such support via the
>shell?)
>
>e. Generally is it possible to get via API the configuration used by the
>servers? (at cluster/server level)
>
>    3.  I ran both major compaction  requests from the shell or from API
>but since both are async there is no progress indication. Neither the JMX
>nor the Web will help here since you don¹t know if a compaction task is
>running. Tailling the logs is not an efficient way to do this neither. The
>point is that I would like to automate the process and avoid compaction
>storm. So I want to do that region, region, but if I don¹t know when a
>compaction started/ended I can¹t automate it.
>
>4.       In case there is no compaction files in queue (but still you have
>more than 1 storefile per store e.g. minor compaction just finished) then
>invoking major_compact will indeed decrease the number of store files, but
>the compaction queue will remain to 0 during the compaction task
>(shouldn¹t
>the compaction queue increase by the number of file to compact and be
>reduced when the task ended?)
>
>
>5.       I saw already HBASE-3965 for getting status of major compaction,
>nevertheless it has be removed from 0.92, is it possible to put it back?
>Even sooner than 0.92?
>
>6.       In case a compaction (major) is running it seems there is no way
>to stop-it. Do you plan to add such feature?
>
>7.       Do you plan to add functionality via JMX (starting/stopping
>compaction, splitting....)
>
>8.       Finally there were some request for allowing custom compaction,
>part of this was given via the RegionObserver in HBASE-2001, nevertheless
>do you consider adding support for custom compaction (providing real
>pluggable compaction stategy not just observer)?
>
>
>Regards,
>Mikael.S