You are viewing a plain text version of this content. The canonical link for it is here.
Posted to dev@phoenix.apache.org by "James Taylor (JIRA)" <ji...@apache.org> on 2017/05/12 23:03:04 UTC

[jira] [Commented] (PHOENIX-3576) The local index updates will be blocked when the region is preparing to split

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

James Taylor commented on PHOENIX-3576:
---------------------------------------

This looks like a duplicate of PHOENIX-3838 (FYI, [~gjacoby]). The fix for HBase 1.3+ is PHOENIX-3827, but should we apply something like this patch to other earlier versions? Is this patch complete, [~chia7712]? I'm not seeing where SimpleWriterIndexCommitter is used. Wouldn't it be easier to just create a different thread pool executor for same thread execution (Executors.sameThreadExecutor()) in the ParallelWriterIndexCommitter.setup() method instead of copy/pasting so much code? Then the two thread pools could coexist with the same thread executor being used for local indexes?
{code}
    @Override
    public void setup(IndexWriter parent, RegionCoprocessorEnvironment env, String name) {
        this.env = env;
        Configuration conf = env.getConfiguration();
        setup(IndexWriterUtils.getDefaultDelegateHTableFactory(env),
                ThreadPoolManager.getExecutor(
                        new ThreadPoolBuilder(name, conf).setMaxThread(NUM_CONCURRENT_INDEX_WRITER_THREADS_CONF_KEY,
                                DEFAULT_CONCURRENT_INDEX_WRITER_THREADS).setCoreTimeout(
                                INDEX_WRITER_KEEP_ALIVE_TIME_CONF_KEY), env), env.getRegionServerServices(), parent,
                CachingHTableFactory.getCacheSize(conf),env);
        this.kvBuilder = KeyValueBuilder.get(env.getHBaseVersion());
    }
{code}
Thoughts, [~rajeshbabu]?

> The local index updates will be blocked when the region is preparing to split
> -----------------------------------------------------------------------------
>
>                 Key: PHOENIX-3576
>                 URL: https://issues.apache.org/jira/browse/PHOENIX-3576
>             Project: Phoenix
>          Issue Type: Bug
>    Affects Versions: 4.9.0
>            Reporter: Chia-Ping Tsai
>             Fix For: 4.11.0
>
>         Attachments: hbase-root-regionserver-node06.ncku.edu.tw.log, PHOENIX-3576.v0.patch
>
>
> ParallelWriterIndexCommitter uses the AP to update the local index after the region#batchMutate fails. If the region is splitting, all later updates will fail (RegionTooBusyException) due to the write lock. The AP will retry the failed updates until it reaches the retry threshold, so the read lock of region is released after a long time. The read lock blocks the HRegion#close, so the region splitting is also blocked.
> {noformat}
>                     try {
>                         if (allowLocalUpdates
>                                 && env != null
>                                 && tableReference.getTableName().equals(
>                                     env.getRegion().getTableDesc().getNameAsString())) {
>                             try {
>                                 throwFailureIfDone();
>                                 IndexUtil.writeLocalUpdates(env.getRegion(), mutations, true);
>                                 return null;
>                             } catch (IOException ignord) {
>                                 // when it's failed we fall back to the standard & slow way
>                                 if (LOG.isDebugEnabled()) {
>                                     LOG.debug("indexRegion.batchMutate failed and fall back to HTable.batch(). Got error="
>                                             + ignord);
>                                 }
>                             }
>                         }
>                         table = factory.getTable(tableReference.get());
>                         throwFailureIfDone();
>                         table.batch(mutations);
>                     } catch (SingleIndexWriteFailureException e) {
>                         throw e;
>                     } catch (IOException e) {
>                         throw new SingleIndexWriteFailureException(tableReference.toString(), mutations, e);
>                     } catch (InterruptedException e) {
>                         // reset the interrupt status on the thread
>                         Thread.currentThread().interrupt();
>                         throw new SingleIndexWriteFailureException(tableReference.toString(), mutations, e);
>                     }
>                     finally{
>                         if (table != null) {
>                             table.close();
>                         }
>                     }
>                     return null;
> {noformat}



--
This message was sent by Atlassian JIRA
(v6.3.15#6346)