| /** |
| * 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.sqoop.mapreduce; |
| |
| import java.io.IOException; |
| |
| import org.apache.commons.logging.Log; |
| import org.apache.commons.logging.LogFactory; |
| |
| import org.apache.hadoop.conf.Configuration; |
| import org.apache.hadoop.fs.FileSystem; |
| import org.apache.hadoop.fs.Path; |
| import org.apache.hadoop.io.NullWritable; |
| import org.apache.hadoop.io.Text; |
| import org.apache.hadoop.io.SequenceFile.CompressionType; |
| import org.apache.hadoop.io.compress.GzipCodec; |
| import org.apache.hadoop.mapreduce.Counters; |
| 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.mapreduce.lib.db.DBConfiguration; |
| import org.apache.hadoop.mapreduce.lib.db.DataDrivenDBInputFormat; |
| import org.apache.hadoop.mapreduce.lib.db.DBWritable; |
| |
| import org.apache.hadoop.sqoop.ConnFactory; |
| import org.apache.hadoop.sqoop.SqoopOptions; |
| import org.apache.hadoop.sqoop.manager.ConnManager; |
| import org.apache.hadoop.sqoop.orm.TableClassName; |
| import org.apache.hadoop.sqoop.util.ClassLoaderStack; |
| import org.apache.hadoop.sqoop.util.PerfCounters; |
| |
| /** |
| * Actually runs a jdbc import job using the ORM files generated by the sqoop.orm package. |
| * Uses DataDrivenDBInputFormat |
| */ |
| public class DataDrivenImportJob { |
| |
| public static final Log LOG = LogFactory.getLog(DataDrivenImportJob.class.getName()); |
| |
| private SqoopOptions options; |
| |
| public DataDrivenImportJob(final SqoopOptions opts) { |
| this.options = opts; |
| } |
| |
| /** |
| * Run an import job to read a table in to HDFS |
| * |
| * @param tableName the database table to read |
| * @param ormJarFile the Jar file to insert into the dcache classpath. (may be null) |
| * @param splitByCol the column of the database table to use to split the import |
| * @param conf A fresh Hadoop Configuration to use to build an MR job. |
| */ |
| public void runImport(String tableName, String ormJarFile, String splitByCol, |
| Configuration conf) throws IOException { |
| |
| LOG.info("Beginning data-driven import of " + tableName); |
| |
| String tableClassName = new TableClassName(options).getClassForTable(tableName); |
| |
| boolean isLocal = "local".equals(conf.get("mapreduce.jobtracker.address")) |
| || "local".equals(conf.get("mapred.job.tracker")); |
| ClassLoader prevClassLoader = null; |
| if (isLocal) { |
| // If we're using the LocalJobRunner, then instead of using the compiled jar file |
| // as the job source, we're running in the current thread. Push on another classloader |
| // that loads from that jar in addition to everything currently on the classpath. |
| prevClassLoader = ClassLoaderStack.addJarFile(ormJarFile, tableClassName); |
| } |
| |
| try { |
| Job job = new Job(conf); |
| |
| // Set the external jar to use for the job. |
| job.getConfiguration().set("mapred.jar", ormJarFile); |
| |
| String hdfsWarehouseDir = options.getWarehouseDir(); |
| Path outputPath; |
| |
| if (null != hdfsWarehouseDir) { |
| Path hdfsWarehousePath = new Path(hdfsWarehouseDir); |
| hdfsWarehousePath.makeQualified(FileSystem.get(job.getConfiguration())); |
| outputPath = new Path(hdfsWarehousePath, tableName); |
| } else { |
| outputPath = new Path(tableName); |
| } |
| |
| if (options.getFileLayout() == SqoopOptions.FileLayout.TextFile) { |
| job.setOutputFormatClass(RawKeyTextOutputFormat.class); |
| job.setMapperClass(TextImportMapper.class); |
| job.setOutputKeyClass(Text.class); |
| job.setOutputValueClass(NullWritable.class); |
| if (options.shouldUseCompression()) { |
| FileOutputFormat.setCompressOutput(job, true); |
| FileOutputFormat.setOutputCompressorClass(job, GzipCodec.class); |
| } |
| } else if (options.getFileLayout() == SqoopOptions.FileLayout.SequenceFile) { |
| job.setOutputFormatClass(SequenceFileOutputFormat.class); |
| job.setMapperClass(AutoProgressMapper.class); |
| if (options.shouldUseCompression()) { |
| SequenceFileOutputFormat.setCompressOutput(job, true); |
| SequenceFileOutputFormat.setOutputCompressionType(job, CompressionType.BLOCK); |
| } |
| job.getConfiguration().set("mapred.output.value.class", tableClassName); |
| } else { |
| LOG.warn("Unknown file layout specified: " + options.getFileLayout() + "; using text."); |
| } |
| |
| int numMapTasks = options.getNumMappers(); |
| if (numMapTasks < 1) { |
| numMapTasks = SqoopOptions.DEFAULT_NUM_MAPPERS; |
| LOG.warn("Invalid mapper count; using " + numMapTasks + " mappers."); |
| } |
| job.getConfiguration().setInt("mapred.map.tasks", numMapTasks); |
| job.setNumReduceTasks(0); |
| |
| job.setInputFormatClass(DataDrivenDBInputFormat.class); |
| |
| FileOutputFormat.setOutputPath(job, outputPath); |
| |
| ConnManager mgr = new ConnFactory(conf).getManager(options); |
| String username = options.getUsername(); |
| if (null == username || username.length() == 0) { |
| DBConfiguration.configureDB(job.getConfiguration(), mgr.getDriverClass(), |
| options.getConnectString()); |
| } else { |
| DBConfiguration.configureDB(job.getConfiguration(), mgr.getDriverClass(), |
| options.getConnectString(), username, options.getPassword()); |
| } |
| |
| String [] colNames = options.getColumns(); |
| if (null == colNames) { |
| colNames = mgr.getColumnNames(tableName); |
| } |
| |
| String [] sqlColNames = null; |
| if (null != colNames) { |
| sqlColNames = new String[colNames.length]; |
| for (int i = 0; i < colNames.length; i++) { |
| sqlColNames[i] = mgr.escapeColName(colNames[i]); |
| } |
| } |
| |
| // It's ok if the where clause is null in DBInputFormat.setInput. |
| String whereClause = options.getWhereClause(); |
| |
| // We can't set the class properly in here, because we may not have the |
| // jar loaded in this JVM. So we start by calling setInput() with DBWritable, |
| // and then overriding the string manually. |
| DataDrivenDBInputFormat.setInput(job, DBWritable.class, |
| mgr.escapeTableName(tableName), whereClause, |
| mgr.escapeColName(splitByCol), sqlColNames); |
| job.getConfiguration().set(DBConfiguration.INPUT_CLASS_PROPERTY, tableClassName); |
| |
| PerfCounters counters = new PerfCounters(); |
| counters.startClock(); |
| |
| try { |
| job.waitForCompletion(false); |
| counters.stopClock(); |
| counters.addBytes(job.getCounters().getGroup("FileSystemCounters") |
| .findCounter("HDFS_BYTES_WRITTEN").getValue()); |
| LOG.info("Transferred " + counters.toString()); |
| } catch (InterruptedException ie) { |
| throw new IOException(ie); |
| } catch (ClassNotFoundException cnfe) { |
| throw new IOException(cnfe); |
| } |
| } finally { |
| if (isLocal && null != prevClassLoader) { |
| // unload the special classloader for this jar. |
| ClassLoaderStack.setCurrentClassLoader(prevClassLoader); |
| } |
| } |
| } |
| } |