[
https://issues.apache.org/jira/browse/HBASE-8571?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=13661170#comment-13661170
]
stack commented on HBASE-8571:
------------------------------
This is the usage for RowCounter:
{code}
durruti:hbase-0.94.7 stack$ ./bin/hbase
org.apache.hadoop.hbase.mapreduce.RowCounter
ERROR: Wrong number of parameters: 0
Usage: RowCounter [options] <tablename> [--range=[startKey],[endKey]]
[<column1> <column2>...]
For performance consider the following options:
-Dhbase.client.scanner.caching=100
-Dmapred.map.tasks.speculative.execution=false
{code}
... and for CopyTable it is:
{code}
durruti:hbase-0.94.7 stack$ ./bin/hbase
org.apache.hadoop.hbase.mapreduce.CopyTable
Usage: CopyTable [general options] [--starttime=X] [--endtime=Y]
[--new.name=NEW] [--peer.adr=ADR] <tablename>
Options:
rs.class hbase.regionserver.class of the peer cluster
specify if different from current cluster
rs.impl hbase.regionserver.impl of the peer cluster
starttime beginning of the time range (unixtime in millis)
without endtime means from starttime to forever
endtime end of the time range. Ignored if no starttime specified.
versions number of cell versions to copy
new.name new table's name
peer.adr Address of the peer cluster given in the format
hbase.zookeeer.quorum:hbase.zookeeper.client.port:zookeeper.znode.parent
families comma-separated list of families to copy
To copy from cf1 to cf2, give sourceCfName:destCfName.
To keep the same name, just give "cfName"
all.cells also copy delete markers and deleted cells
Args:
tablename Name of the table to copy
Examples:
To copy 'TestTable' to a cluster that uses replication for a 1 hour window:
$ bin/hbase org.apache.hadoop.hbase.mapreduce.CopyTable
--starttime=1265875194289 --endtime=1265878794289
--peer.adr=server1,server2,server3:2181:/hbase
--families=myOldCf:myNewCf,cf2,cf3 TestTable
For performance consider the following general options:
-Dhbase.client.scanner.caching=100
-Dmapred.map.tasks.speculative.execution=false
{code}
The caching is called out explicitly.
Does that mitigate?
> CopyTable and RowCounter don't seem to use setCaching setting
> -------------------------------------------------------------
>
> Key: HBASE-8571
> URL: https://issues.apache.org/jira/browse/HBASE-8571
> Project: HBase
> Issue Type: Bug
> Reporter: Doug Meil
>
> Maybe it's just me, but I've been looking on trunk and I don't see where
> either RowCounter or CopyTable MapReduce can adjust the setCaching setting on
> the Scan instance.
> Example from RowCounter...
> {code}
> Job job = new Job(conf, NAME + "_" + tableName);
> job.setJarByClass(RowCounter.class);
> Scan scan = new Scan();
> scan.setCacheBlocks(false);
> Set<byte []> qualifiers = new TreeSet<byte[]>(Bytes.BYTES_COMPARATOR);
> if (startKey != null && !startKey.equals("")) {
> scan.setStartRow(Bytes.toBytes(startKey));
> }
> if (endKey != null && !endKey.equals("")) {
> scan.setStopRow(Bytes.toBytes(endKey));
> }
> scan.setFilter(new FirstKeyOnlyFilter());
> if (sb.length() > 0) {
> for (String columnName : sb.toString().trim().split(" ")) {
> String [] fields = columnName.split(":");
> if(fields.length == 1) {
> scan.addFamily(Bytes.toBytes(fields[0]));
> } else {
> byte[] qualifier = Bytes.toBytes(fields[1]);
> qualifiers.add(qualifier);
> scan.addColumn(Bytes.toBytes(fields[0]), qualifier);
> }
> }
> }
> // specified column may or may not be part of first key value for the row.
> // Hence do not use FirstKeyOnlyFilter if scan has columns, instead use
> // FirstKeyValueMatchingQualifiersFilter.
> if (qualifiers.size() == 0) {
> scan.setFilter(new FirstKeyOnlyFilter());
> } else {
> scan.setFilter(new FirstKeyValueMatchingQualifiersFilter(qualifiers));
> }
> job.setOutputFormatClass(NullOutputFormat.class);
> TableMapReduceUtil.initTableMapperJob(tableName, scan,
> RowCounterMapper.class, ImmutableBytesWritable.class, Result.class,
> job);
> job.setNumReduceTasks(0);
> return job;
> {code}
> TableMapReduceUtil only serializes the Scan into the job, it doesn't adjust
> any of the settings.
> Maybe I'm missing something, but this seems like a problem.
--
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