You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@hbase.apache.org by st...@apache.org on 2009/09/17 05:26:02 UTC
svn commit: r816039 - in
/hadoop/hbase/branches/0.20/src/java/org/apache/hadoop/hbase/mapreduce:
Export.java Import.java
Author: stack
Date: Thu Sep 17 03:26:02 2009
New Revision: 816039
URL: http://svn.apache.org/viewvc?rev=816039&view=rev
Log:
HBASE-1684 Backup (Export/Import) contrib tool for 0.20
Added:
hadoop/hbase/branches/0.20/src/java/org/apache/hadoop/hbase/mapreduce/Export.java
hadoop/hbase/branches/0.20/src/java/org/apache/hadoop/hbase/mapreduce/Import.java
Added: hadoop/hbase/branches/0.20/src/java/org/apache/hadoop/hbase/mapreduce/Export.java
URL: http://svn.apache.org/viewvc/hadoop/hbase/branches/0.20/src/java/org/apache/hadoop/hbase/mapreduce/Export.java?rev=816039&view=auto
==============================================================================
--- hadoop/hbase/branches/0.20/src/java/org/apache/hadoop/hbase/mapreduce/Export.java (added)
+++ hadoop/hbase/branches/0.20/src/java/org/apache/hadoop/hbase/mapreduce/Export.java Thu Sep 17 03:26:02 2009
@@ -0,0 +1,120 @@
+/**
+ * Copyright 2009 The Apache Software Foundation
+ *
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.hadoop.hbase.mapreduce;
+
+import java.io.IOException;
+
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.hbase.HBaseConfiguration;
+import org.apache.hadoop.hbase.client.Result;
+import org.apache.hadoop.hbase.client.Scan;
+import org.apache.hadoop.hbase.io.ImmutableBytesWritable;
+import org.apache.hadoop.mapreduce.Job;
+import org.apache.hadoop.mapreduce.lib.output.FileOutputFormat;
+import org.apache.hadoop.mapreduce.lib.output.SequenceFileOutputFormat;
+import org.apache.hadoop.util.GenericOptionsParser;
+
+/**
+ * Export an HBase table.
+ * Writes content to sequence files up in HDFS. Use {@link Import} to read it
+ * back in again.
+ */
+public class Export {
+ final static String NAME = "export";
+
+ /**
+ * Mapper.
+ */
+ static class Exporter
+ extends TableMapper<ImmutableBytesWritable, Result> {
+ /**
+ * @param row The current table row key.
+ * @param value The columns.
+ * @param context The current context.
+ * @throws IOException When something is broken with the data.
+ * @see org.apache.hadoop.mapreduce.Mapper#map(KEYIN, VALUEIN,
+ * org.apache.hadoop.mapreduce.Mapper.Context)
+ */
+ @Override
+ public void map(ImmutableBytesWritable row, Result value,
+ Context context)
+ throws IOException {
+ try {
+ context.write(row, value);
+ } catch (InterruptedException e) {
+ e.printStackTrace();
+ }
+ }
+ }
+
+ /**
+ * Sets up the actual job.
+ *
+ * @param conf The current configuration.
+ * @param args The command line parameters.
+ * @return The newly created job.
+ * @throws IOException When setting up the job fails.
+ */
+ public static Job createSubmittableJob(Configuration conf, String[] args)
+ throws IOException {
+ String tableName = args[0];
+ Path outputDir = new Path(args[1]);
+ Job job = new Job(conf, NAME + "_" + tableName);
+ job.setJarByClass(Exporter.class);
+ // TODO: Allow passing filter and subset of rows/columns.
+ TableMapReduceUtil.initTableMapperJob(tableName, new Scan(),
+ Exporter.class, null, null, job);
+ // No reducers. Just write straight to output files.
+ job.setNumReduceTasks(0);
+ job.setOutputFormatClass(SequenceFileOutputFormat.class);
+ job.setOutputKeyClass(ImmutableBytesWritable.class);
+ job.setOutputValueClass(Result.class);
+ FileOutputFormat.setOutputPath(job, outputDir);
+ return job;
+ }
+
+ /*
+ * @param errorMsg Error message. Can be null.
+ */
+ private static void usage(final String errorMsg) {
+ if (errorMsg != null && errorMsg.length() > 0) {
+ System.err.println("ERROR: " + errorMsg);
+ }
+ System.err.println("Usage: Export <tablename> <outputdir>");
+ }
+
+ /**
+ * Main entry point.
+ *
+ * @param args The command line parameters.
+ * @throws Exception When running the job fails.
+ */
+ public static void main(String[] args) throws Exception {
+ HBaseConfiguration conf = new HBaseConfiguration();
+ String[] otherArgs = new GenericOptionsParser(conf, args).getRemainingArgs();
+ if (otherArgs.length < 2) {
+ usage("Wrong number of arguments: " + otherArgs.length);
+ System.exit(-1);
+ }
+ Job job = createSubmittableJob(conf, otherArgs);
+ System.exit(job.waitForCompletion(true) ? 0 : 1);
+ }
+}
\ No newline at end of file
Added: hadoop/hbase/branches/0.20/src/java/org/apache/hadoop/hbase/mapreduce/Import.java
URL: http://svn.apache.org/viewvc/hadoop/hbase/branches/0.20/src/java/org/apache/hadoop/hbase/mapreduce/Import.java?rev=816039&view=auto
==============================================================================
--- hadoop/hbase/branches/0.20/src/java/org/apache/hadoop/hbase/mapreduce/Import.java (added)
+++ hadoop/hbase/branches/0.20/src/java/org/apache/hadoop/hbase/mapreduce/Import.java Thu Sep 17 03:26:02 2009
@@ -0,0 +1,126 @@
+/**
+ * Copyright 2009 The Apache Software Foundation
+ *
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.hadoop.hbase.mapreduce;
+
+import java.io.IOException;
+
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.hbase.HBaseConfiguration;
+import org.apache.hadoop.hbase.KeyValue;
+import org.apache.hadoop.hbase.client.Put;
+import org.apache.hadoop.hbase.client.Result;
+import org.apache.hadoop.hbase.io.ImmutableBytesWritable;
+import org.apache.hadoop.mapreduce.Job;
+import org.apache.hadoop.mapreduce.lib.input.FileInputFormat;
+import org.apache.hadoop.mapreduce.lib.input.SequenceFileInputFormat;
+import org.apache.hadoop.util.GenericOptionsParser;
+
+/**
+ * Import data written by {@link Export}.
+ */
+public class Import {
+ final static String NAME = "import";
+
+ /**
+ * Write table content out to files in hdfs.
+ */
+ static class Importer
+ extends TableMapper<ImmutableBytesWritable, Put> {
+ /**
+ * @param row The current table row key.
+ * @param value The columns.
+ * @param context The current context.
+ * @throws IOException When something is broken with the data.
+ * @see org.apache.hadoop.mapreduce.Mapper#map(KEYIN, VALUEIN,
+ * org.apache.hadoop.mapreduce.Mapper.Context)
+ */
+ @Override
+ public void map(ImmutableBytesWritable row, Result value,
+ Context context)
+ throws IOException {
+ try {
+ context.write(row, resultToPut(row, value));
+ } catch (InterruptedException e) {
+ e.printStackTrace();
+ }
+ }
+
+ private static Put resultToPut(ImmutableBytesWritable key, Result result)
+ throws IOException {
+ Put put = new Put(key.get());
+ for (KeyValue kv : result.raw()) {
+ put.add(kv);
+ }
+ return put;
+ }
+ }
+
+ /**
+ * Sets up the actual job.
+ *
+ * @param conf The current configuration.
+ * @param args The command line parameters.
+ * @return The newly created job.
+ * @throws IOException When setting up the job fails.
+ */
+ public static Job createSubmittableJob(Configuration conf, String[] args)
+ throws IOException {
+ String tableName = args[0];
+ Path inputDir = new Path(args[1]);
+ Job job = new Job(conf, NAME + "_" + tableName);
+ job.setJarByClass(Importer.class);
+ FileInputFormat.setInputPaths(job, inputDir);
+ job.setInputFormatClass(SequenceFileInputFormat.class);
+ job.setMapperClass(Importer.class);
+ // No reducers. Just write straight to table. Call initTableReducerJob
+ // because it sets up the TableOutputFormat.
+ TableMapReduceUtil.initTableReducerJob(tableName, null, job);
+ job.setNumReduceTasks(0);
+ return job;
+ }
+
+ /*
+ * @param errorMsg Error message. Can be null.
+ */
+ private static void usage(final String errorMsg) {
+ if (errorMsg != null && errorMsg.length() > 0) {
+ System.err.println("ERROR: " + errorMsg);
+ }
+ System.err.println("Usage: Import <tablename> <inputdir>");
+ }
+
+ /**
+ * Main entry point.
+ *
+ * @param args The command line parameters.
+ * @throws Exception When running the job fails.
+ */
+ public static void main(String[] args) throws Exception {
+ HBaseConfiguration conf = new HBaseConfiguration();
+ String[] otherArgs = new GenericOptionsParser(conf, args).getRemainingArgs();
+ if (otherArgs.length < 2) {
+ usage("Wrong number of arguments: " + otherArgs.length);
+ System.exit(-1);
+ }
+ Job job = createSubmittableJob(conf, otherArgs);
+ System.exit(job.waitForCompletion(true) ? 0 : 1);
+ }
+}
\ No newline at end of file