hbase-issues mailing list archives

Site index · List index
Message view « Date » · « Thread »
Top « Date » · « Thread »
From "xinxin fan (JIRA)" <j...@apache.org>
Subject [jira] [Commented] (HBASE-18090) Improve TableSnapshotInputFormat to allow more multiple mappers per region
Date Fri, 08 Sep 2017 06:20:00 GMT

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

xinxin fan commented on HBASE-18090:
------------------------------------

Year, if the region is read-only, the writesEnabled will be false, and the regionSequenceIdFile
will not created. 

{code:java}
if (this.writestate.writesEnabled) {
      nextSeqid = WALSplitter.writeRegionSequenceIdFile(this.fs.getFileSystem(), this.fs
          .getRegionDir(), nextSeqid, (this.recovering ? (this.flushPerChanges + 10000000)
: 1));
    } else {
      nextSeqid++;
    }
{code}

And i notice the fact that a given region will be set read-only mode only when the table
is read-only or the region is not default replica. so it seem feasible that set the HTableDescriptor
read-only before the region open (in ClientSideRegionScanner.java):

{code:java}
public ClientSideRegionScanner(Configuration conf, FileSystem fs,
      Path rootDir, HTableDescriptor htd, HRegionInfo hri, Scan scan, ScanMetrics scanMetrics)
          throws IOException {

    // region is immutable, set isolation level
    scan.setIsolationLevel(IsolationLevel.READ_UNCOMMITTED);

    htd.setReadOnly(true);
    // open region from the snapshot directory
    this.region = HRegion.openHRegion(conf, fs, rootDir, hri, htd, null, null, null);
{code}

I have tested the plan, the split tasks works well and the exception disappear. 


> Improve TableSnapshotInputFormat to allow more multiple mappers per region
> --------------------------------------------------------------------------
>
>                 Key: HBASE-18090
>                 URL: https://issues.apache.org/jira/browse/HBASE-18090
>             Project: HBase
>          Issue Type: Improvement
>          Components: mapreduce
>    Affects Versions: 1.4.0
>            Reporter: Mikhail Antonov
>         Attachments: HBASE-18090-branch-1.3-v1.patch
>
>
> TableSnapshotInputFormat runs one map task per region in the table snapshot. This places
unnecessary restriction that the region layout of the original table needs to take the processing
resources available to MR job into consideration. Allowing to run multiple mappers per region
(assuming reasonably even key distribution) would be useful.



--
This message was sent by Atlassian JIRA
(v6.4.14#64029)

Mime
View raw message