Thanks for the detailed example, David.
On Sun, Mar 31, 2013 at 10:14 AM, David Koch <[email protected]> wrote: > Hello Ted, > > I added the example here: > https://issues.apache.org/jira/browse/HBASE-8202#comment-13618389 > > To highlight the issue I used an HBase table rather than an HDFS as input > source. Otherwise - one could merely set hbase.zookeeper.quorum to the > destination cluster as a workaround and get away with it. > > Regads, > > /David > > > On Wed, Mar 27, 2013 at 6:40 PM, Ted Yu <[email protected]> wrote: > > > I logged HBASE-8202 > > > > Feel free to attach your example there. > > > > Cheers > > > > On Tue, Mar 26, 2013 at 8:03 AM, David Koch <[email protected]> > wrote: > > > > > Hello Ted, > > > > > > Yes, I'll put in a request and add a baseline example to reproduce the > > > issue. > > > > > > Thank you for helping me get to the bottom of this, > > > > > > /David > > > > > > > > > On Sun, Mar 24, 2013 at 3:35 PM, Ted Yu <[email protected]> wrote: > > > > > > > Looks like MultiTableOutputFormat doesn't support this use case - > > > > MultiTableOutputFormat doesn't extend TableOutputFormat: > > > > > > > > public class MultiTableOutputFormat > > > > extendsOutputFormat<ImmutableBytesWritable, Mutation> { > > > > Relevant configuration is setup in TableOutputFormat#setConf(): > > > > > > > > public void setConf(Configuration otherConf) { > > > > this.conf = HBaseConfiguration.create(otherConf); > > > > String tableName = this.conf.get(OUTPUT_TABLE); > > > > if(tableName == null || tableName.length() <= 0) { > > > > throw new IllegalArgumentException("Must specify table name"); > > > > } > > > > String address = this.conf.get(QUORUM_ADDRESS); > > > > int zkClientPort = conf.getInt(QUORUM_PORT, 0); > > > > String serverClass = this.conf.get(REGION_SERVER_CLASS); > > > > String serverImpl = this.conf.get(REGION_SERVER_IMPL); > > > > try { > > > > if (address != null) { > > > > ZKUtil.applyClusterKeyToConf(this.conf, address); > > > > } > > > > > > > > Mind filing a JIRA for enhancement ? > > > > > > > > On Sun, Mar 24, 2013 at 5:46 AM, David Koch <[email protected]> > > > wrote: > > > > > > > > > Hello, > > > > > > > > > > I want to import a file on HDFS from one cluster A (source) into > > HBase > > > > > tables on a different cluster B (destination) using a Mapper job > with > > > an > > > > > HBase sink. Both clusters run HBase. > > > > > > > > > > This setup works fine: > > > > > - Run Mapper job on cluster B (destination) > > > > > - "mapred.input.dir" --> hdfs://<cluster-A>/<path-to-file> (file on > > > > source > > > > > cluster) > > > > > - "hbase.zookeeper.quorum" --> <quorum-hostname-B> > > > > > - "hbase.zookeeper.property.clientPort" --> <quorum-port-B> > > > > > > > > > > I thought it should be possible to run the job on cluster A > (source) > > > and > > > > > using "hbase.mapred.output.quorum" to insert into the tables on > > cluster > > > > B. > > > > > This is what the CopyTable utility does. However, the following > does > > > not > > > > > work. HBase looks for the destination table(s) on cluster A and NOT > > > > cluster > > > > > B: > > > > > - Run Mapper job on cluster A (source) > > > > > - "mapred.input.dir" --> hdfs://<cluster-A>/<path-to-file> (file is > > > > local) > > > > > - "hbase.zookeeper.quorum" --> <quorum-hostname-A> > > > > > - "hbase.zookeeper.property.clientPort" --> <quorum-port-A> > > > > > - "hbase.mapred.output.quorum" -> <quorum-hostname-B>:2181:/hbase > > (same > > > > as > > > > > --peer.adr argument for CopyTable) > > > > > > > > > > Job setup inside the class MyJob is as follows, note I am using > > > > > MultiTableOutputFormat. > > > > > > > > > > Configuration conf = > HBaseConfiguration.addHbaseResources(getConf()); > > > > > Job job = new Job(conf); > > > > > job.setJarByClass(MyJob.class); > > > > > job.setMapperClass(JsonImporterMapper.class); > > > > > // Note, several output tables! > > > > > job.setOutputFormatClass(MultiTableOutputFormat.class); > > > > > job.setNumReduceTasks(0); > > > > > TableMapReduceUtil.addDependencyJars(job); > > > > > TableMapReduceUtil.addDependencyJars(job.getConfiguration()); > > > > > > > > > > Where The Mapper class has the following frame: > > > > > > > > > > public static class JsonImporterMapper extends > > > > > Mapper<LongWritable, Text, ImmutableBytesWritable, Put> { } > > > > > > > > > > Is this expected behaviour? How can I get the second scenario using > > > > > hbase.mapred.output.quorum" to work? Could the fact I am using > > > > > MultiTableOutputFormat instead of TableOutputFormat play a part? I > am > > > > using > > > > > HBase 0.92.1. > > > > > > > > > > Thank you, > > > > > > > > > > /David > > > > > > > > > > > > > > >
