hbase-user mailing list archives

Site index · List index
Message view « Date » · « Thread »
Top « Date » · « Thread »
From Ted Yu <yuzhih...@gmail.com>
Subject Re: MapReduce job runs fine, but nothing is written to HTable
Date Thu, 17 Jun 2010 18:23:03 GMT
You can override these methods of org.apache.hadoop.mapreduce.Mapper class.

  /**
   * Called once at the beginning of the task.
   */
  protected void setup(Context context
                       ) throws IOException, InterruptedException {
    // NOTHING
  }

  /**
   * Called once at the end of the task.
   */
  protected void cleanup(Context context
                         ) throws IOException, InterruptedException {
    // NOTHING
  }

On Thu, Jun 17, 2010 at 10:44 AM, Sharma, Avani <agsharma@ebay.com> wrote:

> Thanks, Dave. I have only 2 records in my HDFS file for testing.
> Could you give an example of which setup and cleanup functions you are
> referring to. This is my first MR HBase job using the new api.
> The other commented code in the email thread below runs fine, but it is
> non-MR. Is there any other setting needed for an MR job to update HBase?
>
> The MR jobs are run against `hadoop  jar <jar_name> <class_name>
> <hdfs_file_name>`
> And non-MR hbase jobs simply run against hbase `hbase <classname>`
> I am suspecting that I am missing some setting. I made sure that the
> CLASSPATHs are all good.
>
> This is how I configure the MR job -
>
>  public int run(String[] args) throws Exception {
>
>    Configuration conf = new Configuration();
>    conf.set(TableOutputFormat.OUTPUT_TABLE, "blogposts");
>
>    Job job = new Job(conf, NAME);
>    FileInputFormat.addInputPath(job, new Path(args[0]));
>    job.setJarByClass(mapRedImport_from_hdfs.class);
>    job.setMapperClass(myMap.class);
>    job.setNumReduceTasks(0);
>    job.setOutputFormatClass(NullOutputFormat.class);
>
>    job.waitForCompletion(true);
>
>    return 0;
>  }
>
>  public static void main(String[] args) throws Exception {
>    int errCode = ToolRunner.run(new mapRedImport_from_hdfs(), args);
>    System.exit(errCode);
>   }
>
> -----Original Message-----
> From: Buttler, David [mailto:buttler1@llnl.gov]
> Sent: Thursday, June 17, 2010 8:06 AM
> To: user@hbase.apache.org
> Subject: RE: MapReduce job runs fine, but nothing is written to HTable
>
> It looks to me as if you are not defining your input format correctly.
>  Notice that you only had two map input records.
> Other issues:
> You are not flushing
> You are creating a new htable on each map.  Put that in the setup and put
> the flush in the cleanup
> Dave
>
>
> -----Original Message-----
> From: Sharma, Avani [mailto:agsharma@ebay.com]
> Sent: Wednesday, June 16, 2010 7:06 PM
> To: user@hbase.apache.org
> Subject: MapReduce job runs fine, but nothing is written to HTable
>
> Hi,
>
> I am running a job to write some data from a HDFS file to and Hbase table
> using the new API.
> The job runs fine without any errors, but I do not see the rows added to
> the hbase table.
>
> This is what my code looks like -
> I am running this as hadoop jar <jar_file_name> <class_name>
> <hdfs_file_name>
>
>      private HTable table;
>                protected void  map(ImmutableBytesWritable key, Text value,
> Context context)
>                throws IOException, InterruptedException
>                        {
>                           table = new HTable( new HBaseConfiguration(),
> "blogposts");
>
>                             // Split input line on tab character
>                            String [] splits = value.toString().split("\t");
>                            String rowID = splits[0];
>                            String cellValue = splits[1];
>                          Put p = new Put(Bytes.toBytes(rowID));
>                        p.add(Bytes.toBytes("post"), Bytes.toBytes("title"),
> Bytes.toBytes(splits[1]));
>                        table.put(p);
>                        table.flushCommits();
> }
> /*
>       This commented code when run seprataely in a main program runs fine
> and does update to the table
>        HTable table = new HTable(new HBaseConfiguration(), "blogposts");
>
>        Put p = new Put(Bytes.toBytes("post3"));
>
>        p.add(Bytes.toBytes("post"), Bytes.toBytes("title"),
> Bytes.toBytes("abx"));
>        p.add(Bytes.toBytes("post"), Bytes.toBytes("author"),
> Bytes.toBytes("hadings"));
>        p.add(Bytes.toBytes("image"), Bytes.toBytes("body"),
> Bytes.toBytes("123.jpg"));
>        p.add(Bytes.toBytes("image"), Bytes.toBytes("header"),
> Bytes.toBytes("7657.jpg"));
>
>        table.put(p);
> */
>
> Run log
> 10/06/16 19:00:35 WARN mapred.JobClient: Use GenericOptionsParser for
> parsing the arguments. Applications should implement Tool for the same.
> 10/06/16 19:00:35 INFO input.FileInputFormat: Total input paths to process
> : 1
> 10/06/16 19:00:36 INFO mapred.JobClient: Running job: job_201003301510_0157
> 10/06/16 19:00:37 INFO mapred.JobClient:  map 0% reduce 0%
> 10/06/16 19:00:45 INFO mapred.JobClient:  map 100% reduce 0%
> 10/06/16 19:00:47 INFO mapred.JobClient: Job complete:
> job_201003301510_0157
> 10/06/16 19:00:47 INFO mapred.JobClient: Counters: 5
> 10/06/16 19:00:47 INFO mapred.JobClient:   Job Counters
> 10/06/16 19:00:47 INFO mapred.JobClient:     Rack-local map tasks=1
> 10/06/16 19:00:47 INFO mapred.JobClient:     Launched map tasks=1
> 10/06/16 19:00:47 INFO mapred.JobClient:   FileSystemCounters
> 10/06/16 19:00:47 INFO mapred.JobClient:     HDFS_BYTES_READ=31
> 10/06/16 19:00:47 INFO mapred.JobClient:   Map-Reduce Framework
> 10/06/16 19:00:47 INFO mapred.JobClient:     Map input records=2
> 10/06/16 19:00:47 INFO mapred.JobClient:     Spilled Records=0
>
> Thanks,
> Avani
>
>
>

Mime
  • Unnamed multipart/alternative (inline, None, 0 bytes)
View raw message