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.sit...@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.sit...@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
>

Reply via email to