[
https://issues.apache.org/jira/browse/HBASE-7725?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=13573833#comment-13573833
]
Jesse Yates commented on HBASE-7725:
------------------------------------
I'd need to control a compaction request from inception to execution. Looking
at the way we make compactions, there are the pre/postCompactSelection which
would need to get the attributes. Then we could definitely add a hook at the
end where we can override the compactionRequest:
{code}
...
// everything went better than expected. create a compaction request
int pri = getCompactPriority(priority);
ret = new CompactionRequest(region, this, filesToCompact, isMajor, pri,
compactionAttributes);
...
{code}
to something like:
{code}
...
// everything went better than expected. create a compaction request
int pri = getCompactPriority(priority);
if (region.getCoprocessorHost() != null) {
ret = region.getCoprocessorHost().preCompactionRequest(region, this,
filesToCompact, isMajor, pri, compactionAttributes)
}
if(ret == null){
ret = new CompactionRequest(region, this, filesToCompact, isMajor, pri,
compactionAttributes);
}
{code}
Then we'd just have to pass in the compaction request to all the compaction CP
hooks, rather than the compactionAttributes object so we can handle special
case compaction setup (for methods like preCompactScannerOpen and preCompact).
So the same overall usage and layout, but we don't need to modify
CompactionRequest and let people add their own finish hooks and store
attributes if the want.
I dig it Andrew. Sound reasonable to everyone else?
> Add generic attributes to CP initiated compaction request AND latch on
> compaction completion
> --------------------------------------------------------------------------------------------
>
> Key: HBASE-7725
> URL: https://issues.apache.org/jira/browse/HBASE-7725
> Project: HBase
> Issue Type: Bug
> Components: Compaction, Coprocessors, regionserver
> Reporter: Jesse Yates
> Assignee: Jesse Yates
> Fix For: 0.96.0, 0.94.6
>
> Attachments: example.java, hbase-7725_0.94-v0.patch,
> hbase-7725-v0.patch, hbase-7725-v1.patch,
> hbase-7725_with-attributes-0.94-v0.patch,
> hbase-7725_with-attributes-0.94-v1.patch
>
>
> You can request that a compaction be started, but you can't be sure when that
> compaction request completes. This is a simple update to the
> CompactionRequest interface and the compact-split thread on the RS that
> doesn't actually impact the RS exposed interface.
> This is particularly useful for CPs so they can control starting/running a
> compaction.
--
This message is automatically generated by JIRA.
If you think it was sent incorrectly, please contact your JIRA administrators
For more information on JIRA, see: http://www.atlassian.com/software/jira