5
0
mirror of https://github.com/apache/sqoop.git synced 2025-05-02 05:12:08 +08:00

SQOOP-133. Removing shim layer mechanism.

This change removes the ShimLoader and various Shim classes such as CDH3Shim
etc. It introduces a couple of new classes - ConfigurationConstants and
ConfigurationHelper - that provide a unique place for articulating interface
related details such as configuration keys that can likely change from version
to version of Hadoop.

From: Arvind Prabhakar <arvind@cloudera.com>

git-svn-id: https://svn.apache.org/repos/asf/incubator/sqoop/trunk@1149994 13f79535-47bb-0310-9956-ffa450edef68
This commit is contained in:
Andrew Bayer 2011-07-22 20:04:22 +00:00
parent df738df6c1
commit b22904cbfe
52 changed files with 439 additions and 1429 deletions

View File

@ -110,26 +110,6 @@ ant jar -Dhadoop.dist=cloudera
To switch between builds, you will need to clear Ivy's dependency
cache: +ant veryclean+
=== Using a Local Hadoop Installation
Sqoop can be compiled against a locally-installed version of Sqoop,
bypassing the maven repository. To do this you will need to set
three properties:
- hadoop.dist should be set to "local"
- hadoop.shim should be set to "cloudera" or "apache", to tell Sqoop whether
to build the Cloudera or Apache-specific shim jar
- hadoop.home should be set to the path where Hadoop is installed.
For example, the following will compile Sqoop against a locally-installed
version of CDH.
++++
ant jar jar-one-shim -Dhadoop.dist=local -Dhadoop.shim=cloudera \
-Dhadoop.home=/usr/lib/hadoop
++++
== Code Quality Analysis
We have three tools which can be used to analyze Sqoop's code quality.

View File

@ -62,17 +62,10 @@ fi
# Where to find the main Sqoop jar
SQOOP_JAR_DIR=$SQOOP_HOME
# Where to find the shim jars.
SQOOP_SHIM_DIR=$SQOOP_HOME/shims
# If there's a "build" subdir, override with this, so we use
# the newly-compiled copy.
if [ -d "$SQOOP_JAR_DIR/build" ]; then
SQOOP_JAR_DIR="${SQOOP_JAR_DIR}/build"
if [ -d "$SQOOP_JAR_DIR/shims" ]; then
SQOOP_SHIM_DIR="$SQOOP_JAR_DIR/shims"
fi
fi
function add_to_classpath() {
@ -115,9 +108,7 @@ add_to_classpath ${SQOOP_JAR_DIR}
export SQOOP_CLASSPATH
export SQOOP_CONF_DIR
export SQOOP_JAR_DIR
export SQOOP_SHIM_DIR
export HADOOP_CLASSPATH="${SQOOP_CLASSPATH}:${HADOOP_CLASSPATH}"
export HADOOP_HOME
export HBASE_HOME
export HADOOP_OPTS="-Dsqoop.shim.jar.dir=${SQOOP_SHIM_DIR} ${HADOOP_OPTS}"

258
build.xml
View File

@ -42,7 +42,7 @@
<property name="dest.jar" value="${artifact.name}.jar" />
<property name="test.jar" value="${name}-test-${version}.jar" />
<property name="git.hash" value="" />
<!-- programs used -->
<property name="python" value="python" />
@ -50,7 +50,6 @@
<property name="base.src.dir" location="${basedir}/src" />
<property name="src.dir" location="${base.src.dir}/java" />
<property name="test.dir" location="${base.src.dir}/test" />
<property name="shim.src.dir" location="${base.src.dir}/shims" />
<property name="perftest.src.dir" location="${base.src.dir}/perftest" />
<property name="lib.dir" location="${basedir}/lib" />
<property name="docs.src.dir" location="${base.src.dir}/docs" />
@ -71,10 +70,6 @@
<!-- compiled classes for the main sqoop artifact. -->
<property name="build.classes" location="${build.dir}/classes"/>
<!-- compiled outputs for the shim artifacts. -->
<property name="build.shim.dir" location="${build.dir}/shims"/>
<property name="build.shim.classes" location="${build.shim.dir}/classes"/>
<!-- root directory for output/intermediate data for testing -->
<property name="build.test" location="${build.dir}/test"/>
<property name="test.log.dir" location="${build.dir}/test/logs"/>
@ -107,7 +102,7 @@
<!-- controlling the Hadoop source -->
<!-- valid values for ${hadoop.dist} are 'apache' (trunk), 'apache21',
and 'cloudera' -->
<property name="hadoop.dist" value="cloudera" />
<!-- <property name="hadoop.dist" value="cloudera" /> -->
<!-- testing with JUnit -->
<property name="test.junit.output.format" value="plain"/>
@ -131,10 +126,6 @@
value="https://repository.cloudera.com/content/repositories/${mvn.repo}"/>
<property name="sqoop.pom" value="${mvn.build.dir}/sqoop.pom" />
<property name="sqooptest.pom" value="${mvn.build.dir}/sqoop-test.pom" />
<property name="apache.shim.pom"
value="${mvn.build.dir}/sqoop-shim-apache.pom" />
<property name="cloudera.shim.pom"
value="${mvn.build.dir}/sqoop-shim-cloudera.pom" />
<!-- code coverage -->
<property name="cobertura.dir" value="${build.dir}/cobertura" />
@ -162,8 +153,8 @@
<property name="ivysettings.xml" location="${ivy.dir}/ivysettings.xml"/>
<loadproperties srcfile="${ivy.dir}/libraries.properties"/>
<property name="ivy.jar" location="${lib.dir}/ivy-${ivy.version}.jar"/>
<property name="ivy_repo_url"
value="http://repo2.maven.org/maven2/org/apache/ivy/ivy/${ivy.version}/ivy-${ivy.version}.jar" />
<property name="ivy_repo_url"
value="http://repo2.maven.org/maven2/org/apache/ivy/ivy/${ivy.version}/ivy-${ivy.version}.jar" />
<property name="mvn_repo_url"
value="http://repo2.maven.org/maven2/org/apache/maven/maven-ant-tasks/${mvn.version}/maven-ant-tasks-${mvn.version}.jar"/>
<property name="mvn.jar"
@ -172,7 +163,7 @@
<property name="build.ivy.lib.dir" location="${build.ivy.dir}/lib" />
<property name="build.ivy.report.dir" location="${build.ivy.dir}/report" />
<property name="redist.ivy.lib.dir"
location="${build.ivy.lib.dir}/${name}/redist"/>
location="${build.ivy.lib.dir}/${name}/redist"/>
<!--this is the naming policy for artifacts we want pulled down-->
<property name="ivy.artifact.retrieve.pattern"
@ -211,10 +202,10 @@
</condition>
</target>
<target name="untar-hbase-zk" unless="hbase-zk-already-untared"
depends="download-hbase-zk">
<untar src="${build.dir}/${hbase.tar}" dest="${build.dir}"
compression="gzip"/>
<target name="untar-hbase-zk" unless="hbase-zk-already-untared"
depends="download-hbase-zk">
<untar src="${build.dir}/${hbase.tar}" dest="${build.dir}"
compression="gzip"/>
<untar src="${build.dir}/${zk.tar}" dest="${build.dir}" compression="gzip"/>
</target>
@ -267,60 +258,14 @@
</fileset>
</path>
<!-- "init" target used for setup purposes. -->
<!-- "init" target used for setup purposes. -->
<target name="init">
<!-- The hadoop.dist property determines which version of Hadoop to
retrieve; this may be "cloudera", "apache", or "apache21". But
multiple of these versions can use the same shim. We set the
hadoop.shim property here, based on that one.
-->
<if>
<equals arg1="${hadoop.dist}" arg2="apache" />
<then>
<property name="hadoop.shim" value="apache" />
<echo message="Hadoop distribution: apache -> apache (trunk) " />
</then>
<elseif>
<equals arg1="${hadoop.dist}" arg2="apache21" />
<then>
<property name="hadoop.shim" value="apache" />
<echo message="Hadoop distribution: apache21 -> apache (0.21)" />
</then>
</elseif>
<elseif>
<equals arg1="${hadoop.dist}" arg2="cloudera" />
<then>
<property name="hadoop.shim" value="cloudera" />
<echo message="Hadoop distribution: cloudera -> cloudera (CDH3)" />
</then>
</elseif>
<elseif>
<equals arg1="${hadoop.dist}" arg2="local" />
<then>
<echo message="Hadoop distribution set to local installation" />
<property name="hadoop.is.local" value="true" />
<fail unless="hadoop.shim">
You must explicitly set ${hadoop.shim} to 'apache' or 'cloudera'
to use a local distribution.
</fail>
<fail unless="hadoop.home">
You must set ${hadoop.home} to point to your local
Hadoop installation.
</fail>
</then>
</elseif>
<else>
<fail message="Invalid value for hadoop.dist: ${hadoop.dist}"/>
</else>
</if>
<!-- Path containing third-party libraries deployed directly with Sqoop.
This does not include anything that Ivy can retrieve for us.
-->
<path id="lib.path">
<fileset dir="${lib.dir}">
<include name="*.jar" />
<include name="${hadoop.shim}/*.jar" />
</fileset>
</path>
@ -329,44 +274,9 @@
<pathelement location="${build.test.classes}" />
<path refid="${name}.hadooptest.classpath" />
<path refid="compile.classpath" />
<pathelement location="${build.shim.classes}/${hadoop.shim}" />
</path>
</target>
<!-- Compile a shim class so Sqoop can run with the specified
hadoop.dist.
-->
<target name="compile-one-shim"
depends="init, ivy-retrieve-hadoop">
<mkdir dir="${build.shim.classes}/${hadoop.shim}" />
<javac
encoding="${build.encoding}"
destdir="${build.shim.classes}/${hadoop.shim}"
debug="${javac.debug}"
deprecation="${javac.deprecation}">
<classpath refid="compile.classpath"/>
<src path="${shim.src.dir}/${hadoop.shim}" />
<src path="${shim.src.dir}/common" />
<include name="**/*.java" />
</javac>
</target>
<!-- Compile one shim and fetch it into a jar file. -->
<target name="jar-one-shim" depends="init,compile-one-shim">
<jar jarfile="${build.shim.dir}/${name}-shim-${hadoop.shim}-${version}.jar"
basedir="${build.shim.classes}/${hadoop.shim}" />
</target>
<!-- Create jars for all shims -->
<target name="jar-all-shims" depends="init,compile">
<antcall target="jar-one-shim" inheritAll="false">
<param name="hadoop.dist" value="apache" />
</antcall>
<antcall target="jar-one-shim" inheritAll="false">
<param name="hadoop.dist" value="cloudera" />
</antcall>
</target>
<!-- generate the version information class. -->
<target name="gen-version" depends="init">
<exec executable="${script.src.dir}/write-version-info.sh"
@ -376,7 +286,7 @@
<arg value="${git.hash}" />
</exec>
</target>
<!-- Compile core classes for the project -->
<target name="compile"
depends="init, gen-version, ivy-retrieve-hadoop, untar-hbase-zk"
@ -410,7 +320,7 @@
</target>
<target name="compile-test"
depends="compile, compile-one-shim, ivy-retrieve-hadoop-test"
depends="compile, ivy-retrieve-hadoop-test"
description="Compile test classes">
<mkdir dir="${build.test.classes}" />
<javac
@ -451,11 +361,11 @@
<!-- Ensure that all source code can be built -->
<target name="compile-all"
depends="compile,compile-test,jar-all-shims,compile-perf-test"
depends="compile,compile-test,compile-perf-test"
description="Compile all sources"/>
<!-- Create all jars. Note this does not include the perftests. -->
<target name="jar-all" depends="jar,jar-test,jar-all-shims"
<target name="jar-all" depends="jar,jar-test"
description="Create all jar artifacts" />
<target name="scripts" depends="jar"
@ -501,14 +411,6 @@
</fileset>
</copy>
<!-- copy in all shim artifacts -->
<mkdir dir="${dist.dir}/shims" />
<copy todir="${dist.dir}/shims" includeEmptyDirs="false" flatten="true">
<fileset dir="${build.shim.dir}">
<include name="*.jar" />
</fileset>
</copy>
<!-- copy in various components of the initial source layout
so that the redistributable can bootstrap itself. -->
<copy todir="${dist.dir}" includeEmptyDirs="false" flatten="false">
@ -576,7 +478,7 @@
and copy it to sqoop-site.xml, overwriting any user-specified
sqoop-site.xml in there.
-->
<copy file="${dist.dir}/conf/sqoop-site-template.xml"
<copy file="${dist.dir}/conf/sqoop-site-template.xml"
tofile="${dist.dir}/conf/sqoop-site.xml"
overwrite="true" />
</target>
@ -617,15 +519,15 @@
<!-- To run third-party tests, run with -Dthirdparty=true -->
<!-- ================================================================== -->
<target name="test"
depends="compile-test,compile,jar-one-shim,test-prep,run-tests"
depends="compile-test,compile,test-prep,run-tests"
description="Run unit tests" />
<!-- actually run the selected unit tests -->
<target name="run-tests"
depends="compile-test,compile,jar-one-shim,test-prep">
depends="compile-test,compile,test-prep">
<antcall target="checkfailure" inheritRefs="true" />
</target>
<target name="test-core">
<!-- inner target only intended to be used via antcall.
Does not define its dependencies. Should be invoked through the
@ -655,8 +557,6 @@
<sysproperty key="test.build.data" value="${build.test}/data"/>
<sysproperty key="build.test" value="${build.test}"/>
<sysproperty key="sqoop.shim.jar.dir" value="${build.shim.dir}" />
<sysproperty key="net.sourceforge.cobertura.datafile"
value="${cobertura.dir}/cobertura-${cobertura.testset}.ser" />
@ -736,7 +636,7 @@
description="Audit license headers for release">
<fail unless="rat.present"
message="Failed to load class [${rat.reporting.classname}]." />
<java classname="${rat.reporting.classname}" fork="true"
<java classname="${rat.reporting.classname}" fork="true"
output="${build.dir}/rat.log">
<classpath refid="${name}.releaseaudit.classpath" />
<arg value="${dist.dir}" />
@ -810,7 +710,7 @@
<!-- Register mvn tasks -->
<path id="mvn-ant-task.classpath" path="${mvn.jar}" />
<typedef resource="org/apache/maven/artifact/ant/antlib.xml"
<typedef resource="org/apache/maven/artifact/ant/antlib.xml"
uri="urn:maven-artifact-ant"
classpathref="mvn-ant-task.classpath"/>
@ -827,17 +727,6 @@
<mapping conf="runtime" scope="runtime" />
</ivy:makepom>
<ivy:makepom ivyfile="ivy/apache-shim.xml" pomfile="${apache.shim.pom}"
settingsRef="${name}.ivy.settings">
<mapping conf="default" scope="compile" />
<mapping conf="runtime" scope="runtime" />
</ivy:makepom>
<ivy:makepom ivyfile="ivy/cloudera-shim.xml" pomfile="${cloudera.shim.pom}"
settingsRef="${name}.ivy.settings">
<mapping conf="default" scope="compile" />
<mapping conf="runtime" scope="runtime" />
</ivy:makepom>
<!-- Change the version in the pom file to reflect our claimed version. -->
<replaceregexp>
<regexp pattern="&lt;version&gt;.*&lt;/version&gt;" />
@ -858,14 +747,6 @@
<jar jarfile="${build.srcjar.dir}/${name}-test-${version}-sources.jar">
<fileset dir="${test.dir}" />
</jar>
<jar jarfile="${build.srcjar.dir}/${name}-shim-apache-${version}-sources.jar">
<fileset dir="${shim.src.dir}/apache/" />
<fileset dir="${shim.src.dir}/common/" />
</jar>
<jar jarfile="${build.srcjar.dir}/${name}-shim-cloudera-${version}-sources.jar">
<fileset dir="${shim.src.dir}/cloudera/" />
<fileset dir="${shim.src.dir}/common/" />
</jar>
</target>
<target name="mvn-install" depends="init,mvn-prep,jar-all,srcjars"
@ -883,24 +764,6 @@
<attach file="${build.srcjar.dir}/${name}-test-${version}-sources.jar"
classifier="sources" />
</artifact:install>
<artifact:pom file="${apache.shim.pom}" id="sqoop-apache-shim" />
<artifact:install
file="${build.shim.dir}/${name}-shim-apache-${version}.jar">
<pom refid="sqoop-apache-shim" />
<attach
file="${build.srcjar.dir}/${name}-shim-apache-${version}-sources.jar"
classifier="sources" />
</artifact:install>
<artifact:pom file="${cloudera.shim.pom}" id="sqoop-cloudera-shim" />
<artifact:install
file="${build.shim.dir}/${name}-shim-cloudera-${version}.jar">
<pom refid="sqoop-cloudera-shim" />
<attach
file="${build.srcjar.dir}/${name}-shim-cloudera-${version}-sources.jar"
classifier="sources" />
</artifact:install>
</target>
<target name="mvn-deploy" depends="init,mvn-prep,jar-all,srcjars"
@ -921,25 +784,6 @@
classifier="sources" />
</artifact:deploy>
<artifact:pom file="${apache.shim.pom}" id="sqoop-apache-shim" />
<artifact:deploy
file="${build.shim.dir}/${name}-shim-apache-${version}.jar">
<remoteRepository id="${mvn.repo.id}" url="${mvn.deploy.url}"/>
<pom refid="sqoop-apache-shim" />
<attach
file="${build.srcjar.dir}/${name}-shim-apache-${version}-sources.jar"
classifier="sources" />
</artifact:deploy>
<artifact:pom file="${cloudera.shim.pom}" id="sqoop-cloudera-shim" />
<artifact:deploy
file="${build.shim.dir}/${name}-shim-cloudera-${version}.jar">
<remoteRepository id="${mvn.repo.id}" url="${mvn.deploy.url}"/>
<pom refid="sqoop-cloudera-shim" />
<attach
file="${build.srcjar.dir}/${name}-shim-cloudera-${version}-sources.jar"
classifier="sources" />
</artifact:deploy>
</target>
<target name="docs" depends="real-docs,relnotes,javadoc"
@ -1041,7 +885,7 @@
</target>
<target name="cobertura"
depends="check-for-cobertura,warn-cobertura-unset,jar,compile-test,test-prep,jar-all-shims"
depends="check-for-cobertura,warn-cobertura-unset,jar,compile-test,test-prep"
if="cobertura.present" description="Run Cobertura (code coverage)">
<taskdef classpathref="cobertura.classpath"
resource="tasks.properties"/>
@ -1140,7 +984,7 @@
<ivy:resolve settingsRef="${name}.ivy.settings" conf="common" />
</target>
<target name="ivy-retrieve-common" depends="ivy-resolve-common">
<ivy:retrieve settingsRef="${name}.ivy.settings"
<ivy:retrieve settingsRef="${name}.ivy.settings"
pattern="${build.ivy.lib.dir}/${ivy.artifact.retrieve.pattern}" sync="true" />
<ivy:cachepath pathid="${name}.common.classpath" conf="common" />
</target>
@ -1170,65 +1014,28 @@
<!-- retrieve ivy-managed artifacts from the Hadoop distribution -->
<target name="ivy-resolve-hadoop" depends="ivy-init"
unless="hadoop.is.local">
<ivy:resolve settingsRef="${name}.ivy.settings" conf="${hadoop.dist}" />
<ivy:resolve settingsRef="${name}.ivy.settings" conf="cloudera" />
</target>
<target name="ivy-retrieve-hadoop" depends="ivy-init,ivy-resolve-hadoop">
<if>
<equals arg1="${hadoop.dist}" arg2="local" />
<then>
<!-- Use a local Hadoop distribution. Just retrieve the basic
'common' configuration, and add the Hadoop jars from
the local Hadoop install.
We can't use a subant here, because the refs wouldn't be
preserved when we return to executing this target.
-->
<ivy:resolve settingsRef="${name}.ivy.settings" conf="common" />
<ivy:retrieve settingsRef="${name}.ivy.settings"
pattern="${build.ivy.lib.dir}/${ivy.artifact.retrieve.pattern}"
sync="true" />
<ivy:cachepath pathid="${name}.hadoop.classpath" conf="common" />
</then>
<else>
<!-- retrieve hadoop refs normally. -->
<ivy:retrieve settingsRef="${name}.ivy.settings"
pattern="${build.ivy.lib.dir}/${ivy.artifact.retrieve.pattern}"
sync="true" />
<ivy:cachepath pathid="${name}.hadoop.classpath" conf="${hadoop.dist}" />
</else>
</if>
<!-- retrieve hadoop refs normally. -->
<ivy:retrieve settingsRef="${name}.ivy.settings"
pattern="${build.ivy.lib.dir}/${ivy.artifact.retrieve.pattern}"
sync="true" />
<ivy:cachepath pathid="${name}.hadoop.classpath" conf="cloudera" />
</target>
<!-- retrieve ivy-managed test artifacts from the Hadoop distribution -->
<target name="ivy-resolve-hadoop-test" depends="ivy-init"
unless="hadoop.is.local">
<ivy:resolve settingsRef="${name}.ivy.settings" conf="${hadoop.dist}test" />
<ivy:resolve settingsRef="${name}.ivy.settings" conf="clouderatest" />
</target>
<target name="ivy-retrieve-hadoop-test"
depends="ivy-init,ivy-resolve-hadoop-test">
<if>
<equals arg1="${hadoop.dist}" arg2="local" />
<then>
<!-- Use a local Hadoop distribution. Just retrieve the basic
'test' configuration, and add the Hadoop jars from
the local Hadoop install.
We can't use a subant here, because the refs wouldn't be
preserved when we return to executing this target.
-->
<ivy:resolve settingsRef="${name}.ivy.settings" conf="test" />
<ivy:retrieve settingsRef="${name}.ivy.settings"
pattern="${build.ivy.lib.dir}/${ivy.artifact.retrieve.pattern}"
sync="true" />
<ivy:cachepath pathid="${name}.hadooptest.classpath" conf="test" />
</then>
<else>
<!-- retrieve hadoop refs normally. -->
<ivy:retrieve settingsRef="${name}.ivy.settings"
<ivy:retrieve settingsRef="${name}.ivy.settings"
pattern="${build.ivy.lib.dir}/${ivy.artifact.retrieve.pattern}"
sync="true" />
<ivy:cachepath pathid="${name}.hadooptest.classpath"
conf="${hadoop.dist}test" />
</else>
</if>
conf="clouderatest" />
</target>
<!-- retrieve ivy-managed artifacts for checkstyle -->
@ -1263,9 +1070,6 @@
<classpath>
<source path="src/java" />
<source path="src/perftest" />
<source path="src/shims/apache" />
<source path="src/shims/common" />
<source path="src/shims/cloudera" />
<source path="src/test" />
<output path="${build.dir}/eclipse-build" />
<library pathref="test.classpath" />

View File

@ -139,8 +139,6 @@
rev="${commons-collections.version}" conf="releaseaudit->default"/>
<!-- dependencies on the shim jars we generate, for release poms -->
<dependency org="com.cloudera.sqoop" name="sqoop-shim-apache"
rev="${version}" conf="mvndeploy->default" />
<dependency org="com.cloudera.sqoop" name="sqoop-shim-cloudera"
rev="${version}" conf="mvndeploy->default" />
</dependencies>

View File

@ -1,49 +0,0 @@
<?xml version="1.0" ?>
<!--
Licensed to Cloudera, Inc. under one or more
contributor license agreements. See the NOTICE file distributed with
this work for additional information regarding copyright ownership.
Cloudera, Inc. 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.
-->
<ivy-module version="1.0">
<info organisation="com.cloudera.sqoop" module="${name}-shim-apache">
<license name="Apache 2.0"/>
<ivyauthor name="Sqoop Team" url="http://github.com/cloudera/sqoop" />
<description>
Sqoop shim for Apache Hadoop
</description>
</info>
<configurations defaultconfmapping="default">
<!--these match the Maven configurations-->
<conf name="default" extends="master,runtime"/>
<conf name="master" description="contains the artifact but no dependencies"/>
<conf name="runtime" description="runtime but not the artifact" />
<conf name="common" visibility="private"
extends="runtime"
description="artifacts needed to compile/test the application"/>
</configurations>
<publications>
<!--get the artifact from our module name-->
<artifact conf="master"/>
</publications>
<dependencies>
<!-- Dependencies for Apache Hadoop (trunk) -->
<dependency org="org.apache.hadoop" name="hadoop-common"
rev="${hadoop-common.apache.version}" conf="common->default"/>
<dependency org="org.apache.hadoop" name="hadoop-hdfs"
rev="${hadoop-hdfs.apache.version}" conf="common->default"/>
<dependency org="org.apache.hadoop" name="hadoop-mapred"
rev="${hadoop-mapred.apache.version}" conf="common->default"/>
</dependencies>
</ivy-module>

View File

@ -1,244 +0,0 @@
Apache License
Version 2.0, January 2004
http://www.apache.org/licenses/
TERMS AND CONDITIONS FOR USE, REPRODUCTION, AND DISTRIBUTION
1. Definitions.
"License" shall mean the terms and conditions for use, reproduction,
and distribution as defined by Sections 1 through 9 of this document.
"Licensor" shall mean the copyright owner or entity authorized by
the copyright owner that is granting the License.
"Legal Entity" shall mean the union of the acting entity and all
other entities that control, are controlled by, or are under common
control with that entity. For the purposes of this definition,
"control" means (i) the power, direct or indirect, to cause the
direction or management of such entity, whether by contract or
otherwise, or (ii) ownership of fifty percent (50%) or more of the
outstanding shares, or (iii) beneficial ownership of such entity.
"You" (or "Your") shall mean an individual or Legal Entity
exercising permissions granted by this License.
"Source" form shall mean the preferred form for making modifications,
including but not limited to software source code, documentation
source, and configuration files.
"Object" form shall mean any form resulting from mechanical
transformation or translation of a Source form, including but
not limited to compiled object code, generated documentation,
and conversions to other media types.
"Work" shall mean the work of authorship, whether in Source or
Object form, made available under the License, as indicated by a
copyright notice that is included in or attached to the work
(an example is provided in the Appendix below).
"Derivative Works" shall mean any work, whether in Source or Object
form, that is based on (or derived from) the Work and for which the
editorial revisions, annotations, elaborations, or other modifications
represent, as a whole, an original work of authorship. For the purposes
of this License, Derivative Works shall not include works that remain
separable from, or merely link (or bind by name) to the interfaces of,
the Work and Derivative Works thereof.
"Contribution" shall mean any work of authorship, including
the original version of the Work and any modifications or additions
to that Work or Derivative Works thereof, that is intentionally
submitted to Licensor for inclusion in the Work by the copyright owner
or by an individual or Legal Entity authorized to submit on behalf of
the copyright owner. For the purposes of this definition, "submitted"
means any form of electronic, verbal, or written communication sent
to the Licensor or its representatives, including but not limited to
communication on electronic mailing lists, source code control systems,
and issue tracking systems that are managed by, or on behalf of, the
Licensor for the purpose of discussing and improving the Work, but
excluding communication that is conspicuously marked or otherwise
designated in writing by the copyright owner as "Not a Contribution."
"Contributor" shall mean Licensor and any individual or Legal Entity
on behalf of whom a Contribution has been received by Licensor and
subsequently incorporated within the Work.
2. Grant of Copyright License. Subject to the terms and conditions of
this License, each Contributor hereby grants to You a perpetual,
worldwide, non-exclusive, no-charge, royalty-free, irrevocable
copyright license to reproduce, prepare Derivative Works of,
publicly display, publicly perform, sublicense, and distribute the
Work and such Derivative Works in Source or Object form.
3. Grant of Patent License. Subject to the terms and conditions of
this License, each Contributor hereby grants to You a perpetual,
worldwide, non-exclusive, no-charge, royalty-free, irrevocable
(except as stated in this section) patent license to make, have made,
use, offer to sell, sell, import, and otherwise transfer the Work,
where such license applies only to those patent claims licensable
by such Contributor that are necessarily infringed by their
Contribution(s) alone or by combination of their Contribution(s)
with the Work to which such Contribution(s) was submitted. If You
institute patent litigation against any entity (including a
cross-claim or counterclaim in a lawsuit) alleging that the Work
or a Contribution incorporated within the Work constitutes direct
or contributory patent infringement, then any patent licenses
granted to You under this License for that Work shall terminate
as of the date such litigation is filed.
4. Redistribution. You may reproduce and distribute copies of the
Work or Derivative Works thereof in any medium, with or without
modifications, and in Source or Object form, provided that You
meet the following conditions:
(a) You must give any other recipients of the Work or
Derivative Works a copy of this License; and
(b) You must cause any modified files to carry prominent notices
stating that You changed the files; and
(c) You must retain, in the Source form of any Derivative Works
that You distribute, all copyright, patent, trademark, and
attribution notices from the Source form of the Work,
excluding those notices that do not pertain to any part of
the Derivative Works; and
(d) If the Work includes a "NOTICE" text file as part of its
distribution, then any Derivative Works that You distribute must
include a readable copy of the attribution notices contained
within such NOTICE file, excluding those notices that do not
pertain to any part of the Derivative Works, in at least one
of the following places: within a NOTICE text file distributed
as part of the Derivative Works; within the Source form or
documentation, if provided along with the Derivative Works; or,
within a display generated by the Derivative Works, if and
wherever such third-party notices normally appear. The contents
of the NOTICE file are for informational purposes only and
do not modify the License. You may add Your own attribution
notices within Derivative Works that You distribute, alongside
or as an addendum to the NOTICE text from the Work, provided
that such additional attribution notices cannot be construed
as modifying the License.
You may add Your own copyright statement to Your modifications and
may provide additional or different license terms and conditions
for use, reproduction, or distribution of Your modifications, or
for any such Derivative Works as a whole, provided Your use,
reproduction, and distribution of the Work otherwise complies with
the conditions stated in this License.
5. Submission of Contributions. Unless You explicitly state otherwise,
any Contribution intentionally submitted for inclusion in the Work
by You to the Licensor shall be under the terms and conditions of
this License, without any additional terms or conditions.
Notwithstanding the above, nothing herein shall supersede or modify
the terms of any separate license agreement you may have executed
with Licensor regarding such Contributions.
6. Trademarks. This License does not grant permission to use the trade
names, trademarks, service marks, or product names of the Licensor,
except as required for reasonable and customary use in describing the
origin of the Work and reproducing the content of the NOTICE file.
7. Disclaimer of Warranty. Unless required by applicable law or
agreed to in writing, Licensor provides the Work (and each
Contributor provides its Contributions) on an "AS IS" BASIS,
WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or
implied, including, without limitation, any warranties or conditions
of TITLE, NON-INFRINGEMENT, MERCHANTABILITY, or FITNESS FOR A
PARTICULAR PURPOSE. You are solely responsible for determining the
appropriateness of using or redistributing the Work and assume any
risks associated with Your exercise of permissions under this License.
8. Limitation of Liability. In no event and under no legal theory,
whether in tort (including negligence), contract, or otherwise,
unless required by applicable law (such as deliberate and grossly
negligent acts) or agreed to in writing, shall any Contributor be
liable to You for damages, including any direct, indirect, special,
incidental, or consequential damages of any character arising as a
result of this License or out of the use or inability to use the
Work (including but not limited to damages for loss of goodwill,
work stoppage, computer failure or malfunction, or any and all
other commercial damages or losses), even if such Contributor
has been advised of the possibility of such damages.
9. Accepting Warranty or Additional Liability. While redistributing
the Work or Derivative Works thereof, You may choose to offer,
and charge a fee for, acceptance of support, warranty, indemnity,
or other liability obligations and/or rights consistent with this
License. However, in accepting such obligations, You may act only
on Your own behalf and on Your sole responsibility, not on behalf
of any other Contributor, and only if You agree to indemnify,
defend, and hold each Contributor harmless for any liability
incurred by, or claims asserted against, such Contributor by reason
of your accepting any such warranty or additional liability.
END OF TERMS AND CONDITIONS
APPENDIX: How to apply the Apache License to your work.
To apply the Apache License to your work, attach the following
boilerplate notice, with the fields enclosed by brackets "[]"
replaced with your own identifying information. (Don't include
the brackets!) The text should be enclosed in the appropriate
comment syntax for the file format. We also recommend that a
file or class name and description of purpose be included on the
same "printed page" as the copyright notice for easier
identification within third-party archives.
Copyright [yyyy] [name of copyright owner]
Licensed 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.
APACHE HADOOP SUBCOMPONENTS:
The Apache Hadoop project contains subcomponents with separate copyright
notices and license terms. Your use of the source code for the these
subcomponents is subject to the terms and conditions of the following
licenses.
For the org.apache.hadoop.util.bloom.* classes:
/**
*
* Copyright (c) 2005, European Commission project OneLab under contract
* 034819 (http://www.one-lab.org)
* All rights reserved.
* Redistribution and use in source and binary forms, with or
* without modification, are permitted provided that the following
* conditions are met:
* - Redistributions of source code must retain the above copyright
* notice, this list of conditions and the following disclaimer.
* - Redistributions in binary form must reproduce the above copyright
* notice, this list of conditions and the following disclaimer in
* the documentation and/or other materials provided with the distribution.
* - Neither the name of the University Catholique de Louvain - UCL
* nor the names of its contributors may be used to endorse or
* promote products derived from this software without specific prior
* written permission.
*
* THIS SOFTWARE IS PROVIDED BY THE COPYRIGHT HOLDERS AND CONTRIBUTORS
* "AS IS" AND ANY EXPRESS OR IMPLIED WARRANTIES, INCLUDING, BUT NOT
* LIMITED TO, THE IMPLIED WARRANTIES OF MERCHANTABILITY AND FITNESS
* FOR A PARTICULAR PURPOSE ARE DISCLAIMED. IN NO EVENT SHALL THE
* COPYRIGHT OWNER OR CONTRIBUTORS BE LIABLE FOR ANY DIRECT, INDIRECT,
* INCIDENTAL, SPECIAL, EXEMPLARY, OR CONSEQUENTIAL DAMAGES (INCLUDING,
* BUT NOT LIMITED TO, PROCUREMENT OF SUBSTITUTE GOODS OR SERVICES;
* LOSS OF USE, DATA, OR PROFITS; OR BUSINESS INTERRUPTION) HOWEVER
* CAUSED AND ON ANY THEORY OF LIABILITY, WHETHER IN CONTRACT, STRICT
* LIABILITY, OR TORT (INCLUDING NEGLIGENCE OR OTHERWISE) ARISING IN
* ANY WAY OUT OF THE USE OF THIS SOFTWARE, EVEN IF ADVISED OF THE
* POSSIBILITY OF SUCH DAMAGE.
*/

View File

@ -0,0 +1,83 @@
/**
* Licensed to Cloudera, Inc. under one
* or more contributor license agreements. See the NOTICE file
* distributed with this work for additional information
* regarding copyright ownership. Cloudera, Inc. 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 com.cloudera.sqoop.config;
/**
* Static constants that identify configuration keys, counter group names, and
* counter names.
*/
public final class ConfigurationConstants {
/**
* The Configuration property identifying the current task id.
*/
public static final String PROP_MAPRED_TASK_ID = "mapred.task.id";
/**
* The Configuration property identifying the job's local directory.
*/
public static final String PROP_JOB_LOCAL_DIRECTORY = "job.local.dir";
/**
* The Configuration property identifying the number of map tasks to be used.
*/
public static final String PROP_MAPRED_MAP_TASKS = "mapred.map.tasks";
/**
* The Configuration property identifying the speculative execution flag for
* map tasks.
*/
public static final String PROP_MAPRED_MAP_TASKS_SPECULATIVE_EXEC =
"mapred.map.tasks.speculative.execution";
/**
* The Configuration property identifying the speculative execution flag for
* reduce tasks.
*/
public static final String PROP_MAPRED_REDUCE_TASKS_SPECULATIVE_EXEC =
"mapred.reduce.tasks.speculative.execution";
/**
* The Configuration property identifying the job tracker address.
*/
public static final String PROP_MAPRED_JOB_TRACKER_ADDRESS =
"mapred.job.tracker";
/**
* The group name of task counters.
*/
public static final String COUNTER_GROUP_MAPRED_TASK_COUNTERS =
"org.apache.hadoop.mapred.Task$Counter";
/**
* The name of the counter that tracks output records from Map phase.
*/
public static final String COUNTER_MAP_OUTPUT_RECORDS =
"MAP_OUTPUT_RECORDS";
/**
* The name of the counter that tracks input records to the Map phase.
*/
public static final String COUNTER_MAP_INPUT_RECORDS =
"MAP_INPUT_RECORDS";
private ConfigurationConstants() {
// Disable Explicit Object Creation
}
}

View File

@ -0,0 +1,171 @@
/**
* Licensed to Cloudera, Inc. under one
* or more contributor license agreements. See the NOTICE file
* distributed with this work for additional information
* regarding copyright ownership. Cloudera, Inc. 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 com.cloudera.sqoop.config;
import java.io.IOException;
import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.mapreduce.Job;
import org.apache.hadoop.mapreduce.JobContext;
import org.apache.hadoop.util.GenericOptionsParser;
import com.cloudera.sqoop.mapreduce.db.DBConfiguration;
/**
* This class provides static helper methods that allow access and manipulation
* of job configuration. It is convenient to keep such access in one place in
* order to allow easy modifications when some of these aspects change from
* version to version of Hadoop.
*/
public final class ConfigurationHelper {
/**
* Set the (hinted) number of map tasks for a job.
*/
public static void setJobNumMaps(Job job, int numMapTasks) {
job.getConfiguration().setInt(
ConfigurationConstants.PROP_MAPRED_MAP_TASKS, numMapTasks);
}
/**
* Get the (hinted) number of map tasks for a job.
*/
public static int getJobNumMaps(JobContext job) {
return job.getConfiguration().getInt(
ConfigurationConstants.PROP_MAPRED_MAP_TASKS, 1);
}
/**
* @return the number of mapper output records from a job using its counters.
*/
public static long getNumMapOutputRecords(Job job)
throws IOException, InterruptedException {
return job.getCounters().findCounter(
ConfigurationConstants.COUNTER_GROUP_MAPRED_TASK_COUNTERS,
ConfigurationConstants.COUNTER_MAP_OUTPUT_RECORDS).getValue();
}
/**
* @return the number of mapper input records from a job using its counters.
*/
public static long getNumMapInputRecords(Job job)
throws IOException, InterruptedException {
return job.getCounters().findCounter(
ConfigurationConstants.COUNTER_GROUP_MAPRED_TASK_COUNTERS,
ConfigurationConstants.COUNTER_MAP_INPUT_RECORDS).getValue();
}
/**
* Get the (hinted) number of map tasks for a job.
*/
public static int getConfNumMaps(Configuration conf) {
return conf.getInt(ConfigurationConstants.PROP_MAPRED_MAP_TASKS, 1);
}
/**
* Set the mapper speculative execution property for a job.
*/
public static void setJobMapSpeculativeExecution(Job job, boolean isEnabled) {
job.getConfiguration().setBoolean(
ConfigurationConstants.PROP_MAPRED_MAP_TASKS_SPECULATIVE_EXEC,
isEnabled);
}
/**
* Set the reducer speculative execution property for a job.
*/
public static void setJobReduceSpeculativeExecution(
Job job, boolean isEnabled) {
job.getConfiguration().setBoolean(
ConfigurationConstants.PROP_MAPRED_REDUCE_TASKS_SPECULATIVE_EXEC,
isEnabled);
}
/**
* Sets the Jobtracker address to use for a job.
*/
public static void setJobtrackerAddr(Configuration conf, String addr) {
conf.set(ConfigurationConstants.PROP_MAPRED_JOB_TRACKER_ADDRESS, addr);
}
/**
* @return the Configuration property identifying a DBWritable to use.
*/
public static String getDbInputClassProperty() {
return DBConfiguration.INPUT_CLASS_PROPERTY;
}
/**
* @return the Configuration property identifying the DB username.
*/
public static String getDbUsernameProperty() {
return DBConfiguration.USERNAME_PROPERTY;
}
/**
* @return the Configuration property identifying the DB password.
*/
public static String getDbPasswordProperty() {
return DBConfiguration.PASSWORD_PROPERTY;
}
/**
* @return the Configuration property identifying the DB connect string.
*/
public static String getDbUrlProperty() {
return DBConfiguration.URL_PROPERTY;
}
/**
* @return the Configuration property identifying the DB input table.
*/
public static String getDbInputTableNameProperty() {
return DBConfiguration.INPUT_TABLE_NAME_PROPERTY;
}
/**
* @return the Configuration property specifying WHERE conditions for the
* db table.
*/
public static String getDbInputConditionsProperty() {
return DBConfiguration.INPUT_CONDITIONS_PROPERTY;
}
/**
* Parse arguments in 'args' via the GenericOptionsParser and
* embed the results in the supplied configuration.
* @param conf the configuration to populate with generic options.
* @param args the arguments to process.
* @return the unused args to be passed to the application itself.
*/
public static String [] parseGenericOptions(
Configuration conf, String [] args) throws IOException {
// This needs to be shimmed because in Apache Hadoop this can throw
// an IOException, but it does not do so in CDH. We just mandate in
// this method that an IOException is possible.
GenericOptionsParser genericParser = new GenericOptionsParser(
conf, args);
return genericParser.getRemainingArgs();
}
private ConfigurationHelper() {
// Disable explicit object creation
}
}

View File

@ -18,6 +18,8 @@
package com.cloudera.sqoop.manager;
import static com.cloudera.sqoop.lib.DelimiterSet.NULL_CHAR;
import java.io.BufferedWriter;
import java.io.File;
import java.io.FileOutputStream;
@ -27,8 +29,9 @@
import org.apache.commons.logging.Log;
import org.apache.commons.logging.LogFactory;
import org.apache.hadoop.conf.Configuration;
import static com.cloudera.sqoop.lib.DelimiterSet.NULL_CHAR;
import com.cloudera.sqoop.shims.HadoopShim;
import com.cloudera.sqoop.config.ConfigurationConstants;
import com.cloudera.sqoop.config.ConfigurationHelper;
import com.cloudera.sqoop.util.DirectImportUtils;
/**
@ -55,15 +58,15 @@ private MySQLUtils() {
public static final String OUTPUT_ENCLOSE_REQUIRED_KEY =
"sqoop.output.enclose.required";
public static final String TABLE_NAME_KEY =
HadoopShim.get().getDbInputTableNameProperty();
ConfigurationHelper.getDbInputTableNameProperty();
public static final String CONNECT_STRING_KEY =
HadoopShim.get().getDbUrlProperty();
ConfigurationHelper.getDbUrlProperty();
public static final String USERNAME_KEY =
HadoopShim.get().getDbUsernameProperty();
ConfigurationHelper.getDbUsernameProperty();
public static final String PASSWORD_KEY =
HadoopShim.get().getDbPasswordProperty();
ConfigurationHelper.getDbPasswordProperty();
public static final String WHERE_CLAUSE_KEY =
HadoopShim.get().getDbInputConditionsProperty();
ConfigurationHelper.getDbInputConditionsProperty();
public static final String EXTRA_ARGS_KEY =
"sqoop.mysql.extra.args";
@ -92,7 +95,7 @@ public static String writePasswordFile(Configuration conf)
throws IOException {
// Create the temp file to hold the user's password.
String tmpDir = conf.get(
HadoopShim.get().getJobLocalDirProperty(), "/tmp/");
ConfigurationConstants.PROP_JOB_LOCAL_DIRECTORY, "/tmp/");
File tempFile = File.createTempFile("mysql-cnf", ".cnf", new File(tmpDir));
// Make the password file only private readable.

View File

@ -19,6 +19,7 @@
package com.cloudera.sqoop.manager;
import java.io.IOException;
import java.lang.reflect.Method;
import java.sql.Connection;
import java.sql.DriverManager;
import java.sql.ResultSet;
@ -26,16 +27,14 @@
import java.sql.Timestamp;
import java.util.HashMap;
import java.util.Map;
import java.lang.reflect.Method;
import org.apache.commons.logging.Log;
import org.apache.commons.logging.LogFactory;
import org.apache.hadoop.mapreduce.OutputFormat;
import com.cloudera.sqoop.mapreduce.db.OracleDataDrivenDBInputFormat;
import com.cloudera.sqoop.SqoopOptions;
import com.cloudera.sqoop.mapreduce.JdbcExportJob;
import com.cloudera.sqoop.shims.ShimLoader;
import com.cloudera.sqoop.mapreduce.OracleExportOutputFormat;
import com.cloudera.sqoop.mapreduce.db.OracleDataDrivenDBInputFormat;
import com.cloudera.sqoop.util.ExportException;
import com.cloudera.sqoop.util.ImportException;
@ -166,7 +165,7 @@ protected synchronized void finalize() throws Throwable {
for (Connection c : connectionMap.values()) {
c.close();
}
super.finalize();
}
}
@ -304,15 +303,9 @@ public void importTable(ImportJobContext context)
public void exportTable(ExportJobContext context)
throws IOException, ExportException {
context.setConnManager(this);
try {
JdbcExportJob exportJob = new JdbcExportJob(context, null, null,
(Class<? extends OutputFormat>) ShimLoader.getShimClass(
"com.cloudera.sqoop.mapreduce.OracleExportOutputFormat"));
exportJob.runExport();
} catch (ClassNotFoundException cnfe) {
throw new ExportException("Could not start export; could not find class",
cnfe);
}
JdbcExportJob exportJob = new JdbcExportJob(context, null, null,
OracleExportOutputFormat.class);
exportJob.runExport();
}
@Override
@ -376,7 +369,7 @@ private String dbToJavaType(int sqlType) {
// return null if no java type was found for sqlType
return null;
}
/**
* Attempt to map sql type to hive type.
* @param sqlType sql data type

View File

@ -23,24 +23,22 @@
import org.apache.commons.logging.Log;
import org.apache.commons.logging.LogFactory;
import org.apache.hadoop.io.NullWritable;
import org.apache.hadoop.io.Text;
import org.apache.hadoop.mapreduce.InputFormat;
import org.apache.hadoop.mapreduce.Job;
import org.apache.hadoop.mapreduce.Mapper;
import org.apache.hadoop.mapreduce.OutputFormat;
import org.apache.hadoop.mapreduce.lib.output.SequenceFileOutputFormat;
import com.cloudera.sqoop.mapreduce.db.DBConfiguration;
import com.cloudera.sqoop.mapreduce.db.DataDrivenDBInputFormat;
import org.apache.hadoop.mapreduce.lib.db.DBWritable;
import org.apache.hadoop.mapreduce.lib.output.SequenceFileOutputFormat;
import com.cloudera.sqoop.SqoopOptions;
import com.cloudera.sqoop.manager.ConnManager;
import com.cloudera.sqoop.config.ConfigurationHelper;
import com.cloudera.sqoop.lib.LargeObjectLoader;
import com.cloudera.sqoop.shims.HadoopShim;
import com.cloudera.sqoop.shims.ShimLoader;
import com.cloudera.sqoop.manager.ConnManager;
import com.cloudera.sqoop.manager.ImportJobContext;
import com.cloudera.sqoop.mapreduce.db.DBConfiguration;
import com.cloudera.sqoop.mapreduce.db.DataDrivenDBInputFormat;
/**
* Actually runs a jdbc import job using the ORM files generated by the
@ -91,8 +89,7 @@ protected Class<? extends Mapper> getMapperClass() {
protected Class<? extends OutputFormat> getOutputFormatClass()
throws ClassNotFoundException {
if (options.getFileLayout() == SqoopOptions.FileLayout.TextFile) {
return (Class<? extends OutputFormat>) ShimLoader.getShimClass(
"com.cloudera.sqoop.mapreduce.RawKeyTextOutputFormat");
return RawKeyTextOutputFormat.class;
} else if (options.getFileLayout()
== SqoopOptions.FileLayout.SequenceFile) {
return SequenceFileOutputFormat.class;
@ -154,7 +151,7 @@ protected void configureInputFormat(Job job, String tableName,
}
LOG.debug("Using table class: " + tableClassName);
job.getConfiguration().set(HadoopShim.get().getDbInputClassProperty(),
job.getConfiguration().set(ConfigurationHelper.getDbInputClassProperty(),
tableClassName);
job.getConfiguration().setLong(LargeObjectLoader.MAX_INLINE_LOB_LEN_KEY,

View File

@ -38,11 +38,10 @@
import org.apache.hadoop.mapreduce.lib.input.FileInputFormat;
import com.cloudera.sqoop.SqoopOptions;
import com.cloudera.sqoop.config.ConfigurationHelper;
import com.cloudera.sqoop.lib.SqoopRecord;
import com.cloudera.sqoop.manager.ExportJobContext;
import com.cloudera.sqoop.orm.TableClassName;
import com.cloudera.sqoop.shims.HadoopShim;
import com.cloudera.sqoop.shims.ShimLoader;
import com.cloudera.sqoop.util.ExportException;
import com.cloudera.sqoop.util.PerfCounters;
@ -137,7 +136,7 @@ public static boolean isSequenceFiles(Configuration conf, Path p)
}
/**
* @param file a file to test.
* @param file a file to test.
* @return true if 'file' refers to a SequenceFile.
*/
private static boolean hasSequenceFileHeader(Path file, Configuration conf) {
@ -192,8 +191,7 @@ protected Class<? extends InputFormat> getInputFormatClass()
throws ClassNotFoundException {
Class<? extends InputFormat> configuredIF = super.getInputFormatClass();
if (null == configuredIF) {
return (Class<? extends InputFormat>) ShimLoader.getShimClass(
"com.cloudera.sqoop.mapreduce.ExportInputFormat");
return ExportInputFormat.class;
} else {
return configuredIF;
}
@ -204,8 +202,7 @@ protected Class<? extends OutputFormat> getOutputFormatClass()
throws ClassNotFoundException {
Class<? extends OutputFormat> configuredOF = super.getOutputFormatClass();
if (null == configuredOF) {
return (Class<? extends OutputFormat>) ShimLoader.getShimClass(
"com.cloudera.sqoop.mapreduce.ExportOutputFormat");
return ExportOutputFormat.class;
} else {
return configuredOF;
}
@ -218,7 +215,7 @@ protected void configureMapper(Job job, String tableName,
job.setMapperClass(getMapperClass());
// Concurrent writes of the same records would be problematic.
HadoopShim.get().setJobMapSpeculativeExecution(job, false);
ConfigurationHelper.setJobMapSpeculativeExecution(job, false);
job.setMapOutputKeyClass(SqoopRecord.class);
job.setMapOutputValueClass(NullWritable.class);
@ -249,13 +246,13 @@ protected boolean runJob(Job job) throws ClassNotFoundException, IOException,
perfCounters.addBytes(jobCounters.getGroup("FileSystemCounters")
.findCounter("HDFS_BYTES_READ").getValue());
LOG.info("Transferred " + perfCounters.toString());
long numRecords = HadoopShim.get().getNumMapInputRecords(job);
long numRecords = ConfigurationHelper.getNumMapInputRecords(job);
LOG.info("Exported " + numRecords + " records.");
}
return success;
}
/**
* Run an export job to dump a table from HDFS to a database.

View File

@ -43,7 +43,6 @@
import com.cloudera.sqoop.lib.SqoopRecord;
import com.cloudera.sqoop.manager.ConnManager;
import com.cloudera.sqoop.manager.ImportJobContext;
import com.cloudera.sqoop.shims.ShimLoader;
import com.cloudera.sqoop.util.ImportException;
/**
@ -76,14 +75,13 @@ protected Class<? extends Mapper> getMapperClass() {
@Override
protected Class<? extends OutputFormat> getOutputFormatClass()
throws ClassNotFoundException {
return (Class<? extends OutputFormat>) ShimLoader.getShimClass(
"com.cloudera.sqoop.mapreduce.DelegatingOutputFormat");
return DelegatingOutputFormat.class;
}
@Override
protected void configureOutputFormat(Job job, String tableName,
String tableClassName) throws ClassNotFoundException, IOException {
// Use the DelegatingOutputFormat with the HBasePutProcessor.
job.setOutputFormatClass(getOutputFormatClass());

View File

@ -37,9 +37,9 @@
import com.cloudera.sqoop.SqoopOptions;
import com.cloudera.sqoop.orm.TableClassName;
import com.cloudera.sqoop.shims.HadoopShim;
import com.cloudera.sqoop.util.ImportException;
import com.cloudera.sqoop.util.PerfCounters;
import com.cloudera.sqoop.config.ConfigurationHelper;
import com.cloudera.sqoop.manager.ImportJobContext;
/**
@ -49,7 +49,7 @@
public class ImportJobBase extends JobBase {
private ImportJobContext context;
public static final Log LOG = LogFactory.getLog(
ImportJobBase.class.getName());
@ -76,7 +76,7 @@ public ImportJobBase(final SqoopOptions opts,
@Override
protected void configureOutputFormat(Job job, String tableName,
String tableClassName) throws ClassNotFoundException, IOException {
job.setOutputFormatClass(getOutputFormatClass());
if (options.getFileLayout() == SqoopOptions.FileLayout.SequenceFile) {
@ -115,7 +115,7 @@ protected boolean runJob(Job job) throws ClassNotFoundException, IOException,
perfCounters.addBytes(jobCounters.getGroup("FileSystemCounters")
.findCounter("HDFS_BYTES_WRITTEN").getValue());
LOG.info("Transferred " + perfCounters.toString());
long numRecords = HadoopShim.get().getNumMapOutputRecords(job);
long numRecords = ConfigurationHelper.getNumMapOutputRecords(job);
LOG.info("Retrieved " + numRecords + " records.");
}
return success;

View File

@ -32,7 +32,6 @@
import com.cloudera.sqoop.manager.ConnManager;
import com.cloudera.sqoop.manager.ExportJobContext;
import com.cloudera.sqoop.shims.ShimLoader;
/**
* Run an update-based export using JDBC (JDBC-based UpdateOutputFormat).
@ -48,13 +47,7 @@ public class JdbcUpdateExportJob extends ExportJobBase {
*/
private static Class<? extends OutputFormat> getUpdateOutputFormat()
throws IOException {
try {
return (Class<? extends OutputFormat>) ShimLoader.getShimClass(
"com.cloudera.sqoop.mapreduce.UpdateOutputFormat");
} catch (ClassNotFoundException cnfe) {
throw new IOException("Could not load updating export OutputFormat",
cnfe);
}
return UpdateOutputFormat.class;
}
public JdbcUpdateExportJob(final ExportJobContext context)

View File

@ -40,8 +40,8 @@
import com.cloudera.sqoop.SqoopOptions;
import com.cloudera.sqoop.config.ConfigurationHelper;
import com.cloudera.sqoop.manager.ConnManager;
import com.cloudera.sqoop.shims.HadoopShim;
import com.cloudera.sqoop.util.ClassLoaderStack;
import com.cloudera.sqoop.util.Jars;
@ -133,12 +133,11 @@ public void setOptions(SqoopOptions opts) {
protected void cacheJars(Job job, ConnManager mgr)
throws IOException {
Configuration conf = job.getConfiguration();
Configuration conf = job.getConfiguration();
FileSystem fs = FileSystem.getLocal(conf);
Set<String> localUrls = new HashSet<String>();
addToCache(Jars.getSqoopJarPath(), fs, localUrls);
addToCache(Jars.getShimJarPath(), fs, localUrls);
if (null != mgr) {
addToCache(Jars.getDriverClassJar(mgr), fs, localUrls);
addToCache(Jars.getJarPathForClass(mgr.getClass()), fs, localUrls);
@ -159,7 +158,7 @@ protected void cacheJars(Job job, ConnManager mgr)
LOG.warn("SQOOP_HOME is unset. May not be able to find "
+ "all job dependencies.");
}
// If we didn't put anything in our set, then there's nothing to cache.
if (localUrls.isEmpty()) {
return;
@ -273,7 +272,7 @@ protected int configureNumTasks(Job job) throws IOException {
LOG.warn("Invalid mapper count; using " + numMapTasks + " mappers.");
}
HadoopShim.get().setJobNumMaps(job, numMapTasks);
ConfigurationHelper.setJobNumMaps(job, numMapTasks);
job.setNumReduceTasks(0);
return numMapTasks;
}

View File

@ -21,23 +21,17 @@
import java.io.IOException;
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.mapreduce.Job;
import org.apache.hadoop.mapreduce.OutputFormat;
import org.apache.hadoop.mapreduce.lib.input.FileInputFormat;
import org.apache.hadoop.mapreduce.lib.input.SequenceFileInputFormat;
import org.apache.hadoop.mapreduce.lib.output.FileOutputFormat;
import org.apache.hadoop.mapreduce.lib.output.SequenceFileOutputFormat;
import com.cloudera.sqoop.SqoopOptions;
import com.cloudera.sqoop.shims.ShimLoader;
import com.cloudera.sqoop.util.Jars;
/**
@ -78,7 +72,7 @@ public boolean runMergeJob() throws IOException {
String existingJar = options.getExistingJarName();
if (existingJar != null) {
// User explicitly identified a jar path.
LOG.debug("Setting job jar to user-specified jar: " + existingJar);
LOG.debug("Setting job jar to user-specified jar: " + existingJar);
job.getConfiguration().set("mapred.jar", existingJar);
} else {
// Infer it from the location of the specified class, if it's on the
@ -124,9 +118,7 @@ public boolean runMergeJob() throws IOException {
job.setMapperClass(MergeRecordMapper.class);
} else {
job.setMapperClass(MergeTextMapper.class);
job.setOutputFormatClass((Class<? extends OutputFormat>)
ShimLoader.getShimClass(
"com.cloudera.sqoop.mapreduce.RawKeyTextOutputFormat"));
job.setOutputFormatClass(RawKeyTextOutputFormat.class);
}
jobConf.set("mapred.output.key.class", userClassName);

View File

@ -22,21 +22,17 @@
import org.apache.commons.logging.Log;
import org.apache.commons.logging.LogFactory;
import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.io.NullWritable;
import org.apache.hadoop.mapreduce.InputFormat;
import org.apache.hadoop.mapreduce.Job;
import org.apache.hadoop.mapreduce.OutputFormat;
import com.cloudera.sqoop.mapreduce.db.DBConfiguration;
import com.cloudera.sqoop.mapreduce.db.DataDrivenDBInputFormat;
import org.apache.hadoop.mapreduce.lib.db.DBWritable;
import com.cloudera.sqoop.SqoopOptions;
import com.cloudera.sqoop.manager.ConnManager;
import com.cloudera.sqoop.manager.MySQLUtils;
import com.cloudera.sqoop.shims.ShimLoader;
import com.cloudera.sqoop.manager.ImportJobContext;
import com.cloudera.sqoop.manager.MySQLUtils;
import com.cloudera.sqoop.mapreduce.db.DBConfiguration;
import com.cloudera.sqoop.mapreduce.db.DataDrivenDBInputFormat;
/**
* Class that runs an import job using mysqldump in the mapper.
@ -48,11 +44,8 @@ public class MySQLDumpImportJob extends ImportJobBase {
public MySQLDumpImportJob(final SqoopOptions opts, ImportJobContext context)
throws ClassNotFoundException {
super(opts, MySQLDumpMapper.class,
(Class<? extends InputFormat>) ShimLoader.getShimClass(
"com.cloudera.sqoop.mapreduce.MySQLDumpInputFormat"),
(Class<? extends OutputFormat>) ShimLoader.getShimClass(
"com.cloudera.sqoop.mapreduce.RawKeyTextOutputFormat"), context);
super(opts, MySQLDumpMapper.class, MySQLDumpInputFormat.class,
RawKeyTextOutputFormat.class, context);
}
/**

View File

@ -30,7 +30,7 @@
import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.mapreduce.InputSplit;
import com.cloudera.sqoop.shims.HadoopShim;
import com.cloudera.sqoop.config.ConfigurationHelper;
/**
* Implement DBSplitter over BigDecimal values.
@ -46,9 +46,9 @@ public List<InputSplit> split(Configuration conf, ResultSet results,
String lowClausePrefix = colName + " >= ";
String highClausePrefix = colName + " < ";
BigDecimal numSplits = new BigDecimal(
HadoopShim.get().getConfNumMaps(conf));
ConfigurationHelper.getConfNumMaps(conf));
if (minVal == null && maxVal == null) {
// Range is null to null. Return a null split accordingly.

View File

@ -42,15 +42,15 @@
import org.apache.hadoop.conf.Configurable;
import org.apache.hadoop.conf.Configuration;
import com.cloudera.sqoop.shims.HadoopShim;
import com.cloudera.sqoop.config.ConfigurationHelper;
/**
* A InputFormat that reads input data from an SQL table.
* <p>
* DBInputFormat emits LongWritables containing the record number as
* key and DBWritables as value.
*
* The SQL query, and input class can be using one of the two
* DBInputFormat emits LongWritables containing the record number as
* key and DBWritables as value.
*
* The SQL query, and input class can be using one of the two
* setInput methods.
*/
public class DBInputFormat<T extends DBWritable>
@ -71,7 +71,7 @@ public void write(DataOutput out) throws IOException { }
@Override
public void write(PreparedStatement arg0) throws SQLException { }
}
/**
* A InputSplit that spans a set of rows.
*/
@ -172,7 +172,7 @@ public void setConf(Configuration conf) {
public Configuration getConf() {
return dbConf.getConf();
}
public DBConfiguration getDBConf() {
return dbConf;
}
@ -227,7 +227,7 @@ conf, getConnection(), getDBConf(), conditions, fieldNames,
@Override
/** {@inheritDoc} */
public RecordReader<LongWritable, T> createRecordReader(InputSplit split,
TaskAttemptContext context) throws IOException, InterruptedException {
TaskAttemptContext context) throws IOException, InterruptedException {
return createDBRecordReader((DBInputSplit) split,
context.getConfiguration());
@ -237,7 +237,7 @@ public RecordReader<LongWritable, T> createRecordReader(InputSplit split,
@Override
public List<InputSplit> getSplits(JobContext job) throws IOException {
ResultSet results = null;
ResultSet results = null;
Statement statement = null;
try {
statement = connection.createStatement();
@ -246,7 +246,7 @@ public List<InputSplit> getSplits(JobContext job) throws IOException {
results.next();
long count = results.getLong(1);
int chunks = HadoopShim.get().getJobNumMaps(job);
int chunks = ConfigurationHelper.getJobNumMaps(job);
long chunkSize = (count / chunks);
results.close();
@ -285,14 +285,14 @@ public List<InputSplit> getSplits(JobContext job) throws IOException {
}
}
/** Returns the query for getting the total number of rows,
/** Returns the query for getting the total number of rows,
* subclasses can override this for custom behaviour.*/
protected String getCountQuery() {
if(dbConf.getInputCountQuery() != null) {
return dbConf.getInputCountQuery();
}
StringBuilder query = new StringBuilder();
query.append("SELECT COUNT(*) FROM " + tableName);
@ -304,20 +304,20 @@ protected String getCountQuery() {
/**
* Initializes the map-part of the job with the appropriate input settings.
*
*
* @param job The map-reduce job
* @param inputClass the class object implementing DBWritable, which is the
* @param inputClass the class object implementing DBWritable, which is the
* Java object holding tuple fields.
* @param tableName The table to read data from
* @param conditions The condition which to select data with,
* @param conditions The condition which to select data with,
* eg. '(updated &gt; 20070101 AND length &gt; 0)'
* @param orderBy the fieldNames in the orderBy clause.
* @param fieldNames The field names in the table
* @see #setInput(Job, Class, String, String)
*/
public static void setInput(Job job,
public static void setInput(Job job,
Class<? extends DBWritable> inputClass,
String tableName, String conditions,
String tableName, String conditions,
String orderBy, String... fieldNames) {
job.setInputFormatClass(DBInputFormat.class);
DBConfiguration dbConf = new DBConfiguration(job.getConfiguration());
@ -327,17 +327,17 @@ public static void setInput(Job job,
dbConf.setInputConditions(conditions);
dbConf.setInputOrderBy(orderBy);
}
/**
* Initializes the map-part of the job with the appropriate input settings.
*
*
* @param job The map-reduce job
* @param inputClass the class object implementing DBWritable, which is the
* @param inputClass the class object implementing DBWritable, which is the
* Java object holding tuple fields.
* @param inputQuery the input query to select fields. Example :
* @param inputQuery the input query to select fields. Example :
* "SELECT f1, f2, f3 FROM Mytable ORDER BY f1"
* @param inputCountQuery the input query that returns
* the number of records in the table.
* @param inputCountQuery the input query that returns
* the number of records in the table.
* Example : "SELECT COUNT(f1) FROM Mytable"
* @see #setInput(Job, Class, String, String, String, String...)
*/

View File

@ -36,24 +36,24 @@
import org.apache.hadoop.mapreduce.lib.output.FileOutputFormat;
import org.apache.hadoop.util.StringUtils;
import com.cloudera.sqoop.shims.HadoopShim;
import com.cloudera.sqoop.config.ConfigurationHelper;
/**
* A OutputFormat that sends the reduce output to a SQL table.
* <p>
* {@link DBOutputFormat} accepts &lt;key,value&gt; pairs, where
* key has a type extending DBWritable. Returned {@link RecordWriter}
* writes <b>only the key</b> to the database with a batch SQL query.
*
* <p>
* {@link DBOutputFormat} accepts &lt;key,value&gt; pairs, where
* key has a type extending DBWritable. Returned {@link RecordWriter}
* writes <b>only the key</b> to the database with a batch SQL query.
*
*/
public class DBOutputFormat<K extends DBWritable, V>
public class DBOutputFormat<K extends DBWritable, V>
extends OutputFormat<K, V> {
private static final Log LOG = LogFactory.getLog(DBOutputFormat.class);
public void checkOutputSpecs(JobContext context)
public void checkOutputSpecs(JobContext context)
throws IOException, InterruptedException {}
public OutputCommitter getOutputCommitter(TaskAttemptContext context)
public OutputCommitter getOutputCommitter(TaskAttemptContext context)
throws IOException, InterruptedException {
return new FileOutputCommitter(FileOutputFormat.getOutputPath(context),
context);
@ -62,7 +62,7 @@ public OutputCommitter getOutputCommitter(TaskAttemptContext context)
/**
* A RecordWriter that writes the reduce output to a SQL table.
*/
public class DBRecordWriter
public class DBRecordWriter
extends RecordWriter<K, V> {
private Connection connection;
@ -81,11 +81,11 @@ public DBRecordWriter(Connection connection
public Connection getConnection() {
return connection;
}
public PreparedStatement getStatement() {
return statement;
}
@Override
/** {@inheritDoc} */
public void close(TaskAttemptContext context) throws IOException {
@ -123,7 +123,7 @@ public void write(K key, V value) throws IOException {
/**
* Constructs the query used as the prepared statement to insert data.
*
*
* @param table
* the table to insert into
* @param fieldNames
@ -163,20 +163,20 @@ public String constructQuery(String table, String[] fieldNames) {
@Override
/** {@inheritDoc} */
public RecordWriter<K, V> getRecordWriter(TaskAttemptContext context)
public RecordWriter<K, V> getRecordWriter(TaskAttemptContext context)
throws IOException {
DBConfiguration dbConf = new DBConfiguration(context.getConfiguration());
String tableName = dbConf.getOutputTableName();
String[] fieldNames = dbConf.getOutputFieldNames();
if(fieldNames == null) {
fieldNames = new String[dbConf.getOutputFieldCount()];
}
try {
Connection connection = dbConf.getConnection();
PreparedStatement statement = null;
statement = connection.prepareStatement(
constructQuery(tableName, fieldNames));
return new DBRecordWriter(connection, statement);
@ -188,12 +188,12 @@ public RecordWriter<K, V> getRecordWriter(TaskAttemptContext context)
/**
* Initializes the reduce-part of the job with
* the appropriate output settings.
*
*
* @param job The job
* @param tableName The table to insert data into
* @param fieldNames The field names in the table.
*/
public static void setOutput(Job job, String tableName,
public static void setOutput(Job job, String tableName,
String... fieldNames) throws IOException {
if(fieldNames.length > 0 && fieldNames[0] != null) {
DBConfiguration dbConf = setOutput(job, tableName);
@ -201,34 +201,34 @@ public static void setOutput(Job job, String tableName,
} else {
if (fieldNames.length > 0) {
setOutput(job, tableName, fieldNames.length);
} else {
} else {
throw new IllegalArgumentException(
"Field names must be greater than 0");
}
}
}
/**
* Initializes the reduce-part of the job
* Initializes the reduce-part of the job
* with the appropriate output settings.
*
*
* @param job The job
* @param tableName The table to insert data into
* @param fieldCount the number of fields in the table.
*/
public static void setOutput(Job job, String tableName,
public static void setOutput(Job job, String tableName,
int fieldCount) throws IOException {
DBConfiguration dbConf = setOutput(job, tableName);
dbConf.setOutputFieldCount(fieldCount);
}
private static DBConfiguration setOutput(Job job,
String tableName) throws IOException {
job.setOutputFormatClass(DBOutputFormat.class);
HadoopShim.get().setJobReduceSpeculativeExecution(job, false);
ConfigurationHelper.setJobReduceSpeculativeExecution(job, false);
DBConfiguration dbConf = new DBConfiguration(job.getConfiguration());
dbConf.setOutputTableName(tableName);
return dbConf;
}

View File

@ -42,7 +42,7 @@
import org.apache.hadoop.conf.Configurable;
import org.apache.hadoop.conf.Configuration;
import com.cloudera.sqoop.shims.HadoopShim;
import com.cloudera.sqoop.config.ConfigurationHelper;
/**
* A InputFormat that reads input data from an SQL table.
@ -167,7 +167,7 @@ protected DBSplitter getSplitter(int sqlDataType) {
/** {@inheritDoc} */
public List<InputSplit> getSplits(JobContext job) throws IOException {
int targetNumTasks = HadoopShim.get().getJobNumMaps(job);
int targetNumTasks = ConfigurationHelper.getJobNumMaps(job);
if (1 == targetNumTasks) {
// There's no need to run a bounding vals query; just return a split
// that separates nothing. This can be considerably more optimal for a
@ -313,9 +313,9 @@ conf, getConnection(), dbConf, dbConf.getInputConditions(),
* We reuse the same field, but it's not strictly ordering it
* -- just partitioning the results.
*/
public static void setInput(Job job,
public static void setInput(Job job,
Class<? extends DBWritable> inputClass,
String tableName, String conditions,
String tableName, String conditions,
String splitBy, String... fieldNames) {
DBInputFormat.setInput(job, inputClass, tableName, conditions,
splitBy, fieldNames);

View File

@ -27,11 +27,10 @@
import org.apache.commons.logging.Log;
import org.apache.commons.logging.LogFactory;
import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.mapreduce.InputSplit;
import com.cloudera.sqoop.shims.HadoopShim;
import com.cloudera.sqoop.config.ConfigurationHelper;
/**
* Implement DBSplitter over date/time values.
@ -55,7 +54,7 @@ public List<InputSplit> split(Configuration conf, ResultSet results,
String lowClausePrefix = colName + " >= ";
String highClausePrefix = colName + " < ";
int numSplits = HadoopShim.get().getConfNumMaps(conf);
int numSplits = ConfigurationHelper.getConfNumMaps(conf);
if (numSplits < 1) {
numSplits = 1;
}

View File

@ -29,7 +29,7 @@
import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.mapreduce.InputSplit;
import com.cloudera.sqoop.shims.HadoopShim;
import com.cloudera.sqoop.config.ConfigurationHelper;
/**
* Implement DBSplitter over floating-point values.
@ -62,7 +62,7 @@ public List<InputSplit> split(Configuration conf, ResultSet results,
// Use this as a hint. May need an extra task if the size doesn't
// divide cleanly.
int numSplits = HadoopShim.get().getConfNumMaps(conf);
int numSplits = ConfigurationHelper.getConfNumMaps(conf);
double splitSize = (maxVal - minVal) / (double) numSplits;
if (splitSize < MIN_INCREMENT) {

View File

@ -26,7 +26,7 @@
import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.mapreduce.InputSplit;
import com.cloudera.sqoop.shims.HadoopShim;
import com.cloudera.sqoop.config.ConfigurationHelper;
/**
* Implement DBSplitter over integer values.
@ -41,7 +41,7 @@ public List<InputSplit> split(Configuration conf, ResultSet results,
String lowClausePrefix = colName + " >= ";
String highClausePrefix = colName + " < ";
int numSplits = HadoopShim.get().getConfNumMaps(conf);
int numSplits = ConfigurationHelper.getConfNumMaps(conf);
if (numSplits < 1) {
numSplits = 1;
}

View File

@ -18,19 +18,18 @@
package com.cloudera.sqoop.mapreduce.db;
import java.math.BigDecimal;
import java.sql.ResultSet;
import java.sql.SQLException;
import java.math.BigDecimal;
import java.util.ArrayList;
import java.util.List;
import org.apache.commons.logging.Log;
import org.apache.commons.logging.LogFactory;
import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.mapreduce.InputSplit;
import com.cloudera.sqoop.shims.HadoopShim;
import com.cloudera.sqoop.config.ConfigurationHelper;
/**
* Implement DBSplitter over text strings.
@ -88,7 +87,7 @@ public List<InputSplit> split(Configuration conf, ResultSet results,
// Use this as a hint. May need an extra task if the size doesn't
// divide cleanly.
int numSplits = HadoopShim.get().getConfNumMaps(conf);
int numSplits = ConfigurationHelper.getConfNumMaps(conf);
String lowClausePrefix = colName + " >= '";
String highClausePrefix = colName + " < '";

View File

@ -1,147 +0,0 @@
/**
* Licensed to Cloudera, Inc. under one
* or more contributor license agreements. See the NOTICE file
* distributed with this work for additional information
* regarding copyright ownership. Cloudera, Inc. 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 com.cloudera.sqoop.shims;
import java.io.IOException;
import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.fs.Path;
import org.apache.hadoop.mapreduce.Job;
import org.apache.hadoop.mapreduce.JobContext;
import org.apache.hadoop.mapreduce.MapContext;
/**
* In order to be compatible with multiple versions of Hadoop, all parts
* of the Hadoop interface that are not cross-version compatible are
* encapsulated in an implementation of this class. Users should use
* the ShimLoader class as a factory to obtain an implementation of
* HadoopShims corresponding to the version of Hadoop currently on the
* classpath.
*/
public abstract class HadoopShim {
protected HadoopShim() {
}
/**
* Parse arguments in 'args' via the GenericOptionsParser and
* embed the results in the supplied configuration.
* @param conf the configuration to populate with generic options.
* @param args the arguments to process.
* @return the unused args to be passed to the application itself.
*/
public abstract String [] parseGenericOptions(Configuration conf,
String [] args) throws IOException;
/**
* @return the number of mapper output records from a job using its counters.
*/
public abstract long getNumMapOutputRecords(Job job)
throws IOException, InterruptedException;
/**
* @return the number of mapper input records from a job using its counters.
*/
public abstract long getNumMapInputRecords(Job job)
throws IOException, InterruptedException;
/**
* @return the Configuration property identifying the current task id.
*/
public abstract String getTaskIdProperty();
/**
* @return the Configuration property identifying the job's local dir.
*/
public abstract String getJobLocalDirProperty();
/**
* Set the (hinted) number of map tasks for a job.
*/
public abstract void setJobNumMaps(Job job, int numMapTasks);
/**
* Get the (hinted) number of map tasks for a job.
*/
public abstract int getJobNumMaps(JobContext job);
/**
* Get the (hinted) number of map tasks for a job.
*/
public abstract int getConfNumMaps(Configuration conf);
/**
* Set the mapper speculative execution property for a job.
*/
public abstract void setJobMapSpeculativeExecution(Job job,
boolean isEnabled);
/**
* Set the reducer speculative execution property for a job.
*/
public abstract void setJobReduceSpeculativeExecution(Job job,
boolean isEnabled);
/**
* Sets the Jobtracker address to use for a job.
*/
public abstract void setJobtrackerAddr(Configuration conf, String addr);
/**
* Returns the Configuration property identifying a DBWritable to use.
*/
public abstract String getDbInputClassProperty();
/**
* Returns the Configuration property identifying the DB username.
*/
public abstract String getDbUsernameProperty();
/**
* Returns the Configuration property identifying the DB password.
*/
public abstract String getDbPasswordProperty();
/**
* Returns the Configuration property identifying the DB connect string.
*/
public abstract String getDbUrlProperty();
/**
* Returns the Configuration property identifying the DB input table.
*/
public abstract String getDbInputTableNameProperty();
/**
* Returns the Configuration property specifying WHERE conditions for the
* db table.
*/
public abstract String getDbInputConditionsProperty();
/**
* Returns a mock MapContext that has both an OutputCommitter and an
* InputSplit wired to the specified path.
* Used for testing LargeObjectLoader.
*/
public abstract MapContext getMapContextForIOPath(
Configuration conf, Path p);
public static final synchronized HadoopShim get() {
return ShimLoader.getHadoopShim(null);
}
}

View File

@ -1,266 +0,0 @@
/**
* Licensed to Cloudera, Inc. under one
* or more contributor license agreements. See the NOTICE file
* distributed with this work for additional information
* regarding copyright ownership. Cloudera, Inc. 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 com.cloudera.sqoop.shims;
import java.io.File;
import java.io.IOException;
import java.util.ArrayList;
import java.util.List;
import org.apache.commons.logging.Log;
import org.apache.commons.logging.LogFactory;
import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.util.VersionInfo;
import com.cloudera.sqoop.util.ClassLoaderStack;
/**
* Provides a service locator for the appropriate shim, dynamically chosen
* based on the Hadoop version in the classpath.
*/
public abstract class ShimLoader {
private static HadoopShim hadoopShim;
public static final Log LOG = LogFactory.getLog(ShimLoader.class.getName());
/**
* Which directory Sqoop checks for shim jars.
*/
public static final String SHIM_JAR_DIR_PROPERTY = "sqoop.shim.jar.dir";
/**
* The names of the classes for shimming Hadoop.
* This list must be maintained in the same order as HADOOP_SHIM_MATCHES
*/
private static final List<String> HADOOP_SHIM_CLASSES =
new ArrayList<String>();
/**
* Patterns to match to identify which shim jar to load when shimming
* Hadoop.
* This list must be maintained in the same order as HADOOP_SHIM_MATCHES
*/
private static final List<String> HADOOP_SHIM_JARS =
new ArrayList<String>();
/**
* The regular expressions compared against the Hadoop version string
* when determining which shim class to load.
*/
private static final List<String> HADOOP_SHIM_MATCHES =
new ArrayList<String>();
static {
// These regular expressions will be evaluated in order until one matches.
// CDH3 distribution has versions as follows (with nnn being build number):
// 0.20.2+nnn: for CDH3B2 release
// 0.20.3-CDH3-SNAPSHOT: for CDH3B3 development builds
// 0.20.3+nnn: expected for CDH3B3 release
HADOOP_SHIM_MATCHES.add("0.20.(2|3)((\\+[0-9]+)|(-CDH3-SNAPSHOT))");
HADOOP_SHIM_CLASSES.add("com.cloudera.sqoop.shims.CDH3Shim");
HADOOP_SHIM_JARS.add("sqoop-shim-cloudera-.*.jar");
// Apache 0.22 trunk.
// Version may have the form "0.22-SNAPSHOT"
HADOOP_SHIM_MATCHES.add("0.22-.*");
HADOOP_SHIM_CLASSES.add("com.cloudera.sqoop.shims.Apache22HadoopShim");
HADOOP_SHIM_JARS.add("sqoop-shim-apache-.*.jar");
// ... or "0.22.n-SNAPSHOT"
HADOOP_SHIM_MATCHES.add("0.22.\\d+-.*");
HADOOP_SHIM_CLASSES.add("com.cloudera.sqoop.shims.Apache22HadoopShim");
HADOOP_SHIM_JARS.add("sqoop-shim-apache-.*.jar");
// Apache 0.22 trunk snapshots often compile with "Unknown" version,
// so we default to guessing Apache in this case.
HADOOP_SHIM_MATCHES.add("Unknown");
HADOOP_SHIM_CLASSES.add("com.cloudera.sqoop.shims.Apache22HadoopShim");
HADOOP_SHIM_JARS.add("sqoop-shim-apache-.*.jar");
// Apache 0.21 uses the same shim jars as 0.22
HADOOP_SHIM_MATCHES.add("0.21.\\d+(-.*)?");
HADOOP_SHIM_CLASSES.add("com.cloudera.sqoop.shims.Apache22HadoopShim");
HADOOP_SHIM_JARS.add("sqoop-shim-apache-.*.jar");
}
/**
* Factory method to get an instance of HadoopShim based on the
* version of Hadoop on the classpath.
* @param conf an optional Configuration whose internal ClassLoader
* should be updated with the jar containing the HadoopShim.
*/
public static synchronized HadoopShim getHadoopShim(Configuration conf) {
if (hadoopShim == null) {
hadoopShim = loadShim(HADOOP_SHIM_MATCHES, HADOOP_SHIM_CLASSES,
HADOOP_SHIM_JARS, HadoopShim.class, conf);
}
return hadoopShim;
}
/**
* Factory method to get an instance of HadoopShim based on the
* version of Hadoop on the classpath.
*/
public static synchronized HadoopShim getHadoopShim() {
return getHadoopShim(null);
}
@SuppressWarnings("unchecked")
/**
* Actually load the shim for the current Hadoop version.
* @param matchExprs a list of regexes against which the current Hadoop
* version is compared. The first one to hit defines which class/jar to
* use.
* @param classNames a list in the same order as matchExprs. This defines
* what class name to load as the shim class if the Hadoop version matches
* matchExprs[i].
* @param jarPatterns a list in the same order as matchExprs. This defines
* a pattern to select a jar file from which the shim classes should be
* loaded.
* @param xface the shim interface that the shim class must match.
* @param conf an optional Configuration whose context classloader should
* be updated to the current Thread's contextClassLoader after pushing a
* new ClassLoader on the stack to load this shim jar.
*/
private static <T> T loadShim(List<String> matchExprs,
List<String> classNames, List<String> jarPatterns, Class<T> xface,
Configuration conf) {
String version = VersionInfo.getVersion();
LOG.debug("Loading shims for class : " + xface.getName());
LOG.debug("Hadoop version: " + version);
for (int i = 0; i < matchExprs.size(); i++) {
LOG.debug("Checking: " + matchExprs.get(i));
if (version.matches(matchExprs.get(i))) {
String className = classNames.get(i);
String jarPattern = jarPatterns.get(i);
if (LOG.isDebugEnabled()) {
LOG.debug("Version matched regular expression: " + matchExprs.get(i));
LOG.debug("Trying to load class: " + className);
}
// Test to see if the class is already on the classpath.
try {
// If we can load the shim directly, we just do so. In this case,
// there's no need to update the Configuration's classloader,
// because we didn't modify the classloader stack.
return getShimInstance(className, xface);
} catch (Exception e) {
// Not already present. We'll need to load a jar for this.
// Ignore this exception.
}
try {
LOG.debug("Searching for jar matching: " + jarPattern);
loadMatchingShimJar(jarPattern, className);
LOG.debug("Loading shim from jar");
T shim = getShimInstance(className, xface);
if (null != conf) {
// Set the context classloader for the base Configuration to
// the current one, so we can load more classes from the shim jar.
conf.setClassLoader(Thread.currentThread().getContextClassLoader());
}
return shim;
} catch (Exception e) {
throw new RuntimeException("Could not load shim in class "
+ className, e);
}
}
}
throw new RuntimeException("Could not find appropriate Hadoop shim for "
+ version);
}
/**
* Check the current classloader to see if it can load the prescribed
* class name as an instance of 'xface'. If so, create an instance of
* the class and return it.
* @param className the shim class to attempt to instantiate.
* @param xface the interface it must implement.
* @return an instance of className.
*/
private static <T> T getShimInstance(String className, Class<T> xface)
throws ClassNotFoundException, InstantiationException,
IllegalAccessException {
ClassLoader cl = Thread.currentThread().getContextClassLoader();
Class clazz = Class.forName(className, true, cl);
return xface.cast(clazz.newInstance());
}
/**
* Look through the shim directory for a jar matching 'jarPattern'
* and classload it.
* @param jarPattern a regular expression which the shim jar's filename
* must match.
* @param className a class to classload from the jar.
*/
private static void loadMatchingShimJar(String jarPattern, String className)
throws IOException {
String jarFilename;
String shimDirName = System.getProperty(SHIM_JAR_DIR_PROPERTY, ".");
File shimDir = new File(shimDirName);
if (!shimDir.exists()) {
throw new IOException("No such shim directory: " + shimDirName);
}
String [] candidates = shimDir.list();
if (null == candidates) {
throw new IOException("Could not list shim directory: " + shimDirName);
}
for (String candidate : candidates) {
if (candidate.matches(jarPattern)) {
LOG.debug("Found jar matching pattern " + jarPattern + ": "
+ candidate);
File jarFile = new File(shimDir, candidate);
String jarFileName = jarFile.toString();
ClassLoaderStack.addJarFile(jarFileName, className);
LOG.debug("Successfully pushed classloader for jar: " + jarFileName);
return;
}
}
throw new IOException("Could not load shim jar for pattern: "
+ jarPattern);
}
private ShimLoader() {
// prevent instantiation
}
/**
* Given the name of a class, try to load the shim jars and return the Class
* object referring to that class.
* @param className a class to load out of the shim jar
* @return the class object loaded from the shim jar for the given class.
*/
public static <T> Class<? extends T> getShimClass(String className)
throws ClassNotFoundException {
getHadoopShim(); // Make sure shims are loaded.
return (Class<? extends T>) Class.forName(className,
true, Thread.currentThread().getContextClassLoader());
}
}

View File

@ -41,7 +41,6 @@
import com.cloudera.sqoop.lib.DelimiterSet;
import com.cloudera.sqoop.manager.ConnManager;
import com.cloudera.sqoop.metastore.JobData;
import com.cloudera.sqoop.shims.ShimLoader;
/**
* Layer on top of SqoopTool that provides some basic common code
@ -77,8 +76,8 @@ public abstract class BaseSqoopTool extends SqoopTool {
public static final String HIVE_HOME_ARG = "hive-home";
public static final String WAREHOUSE_DIR_ARG = "warehouse-dir";
public static final String TARGET_DIR_ARG = "target-dir";
public static final String APPEND_ARG = "append";
public static final String APPEND_ARG = "append";
public static final String FMT_SEQUENCEFILE_ARG = "as-sequencefile";
public static final String FMT_TEXTFILE_ARG = "as-textfile";
public static final String HIVE_IMPORT_ARG = "hive-import";
@ -170,10 +169,6 @@ protected void setManager(ConnManager mgr) {
* @return true on success, false on failure.
*/
protected boolean init(SqoopOptions sqoopOpts) {
// Make sure shim jar is classloaded early.
ShimLoader.getHadoopShim(sqoopOpts.getConf());
// Get the connection to the database.
try {
JobData data = new JobData(sqoopOpts, this);
@ -227,7 +222,7 @@ protected boolean hasUnrecognizedArgs(String [] argv, int offset, int len) {
LOG.error("Error parsing arguments for " + getToolName() + ":");
printedBanner = true;
}
LOG.error("Unrecognized argument: " + argv[i]);
LOG.error("Unrecognized argument: " + argv[i]);
unrecognized = true;
}
}
@ -537,7 +532,7 @@ protected RelatedOptions getHBaseOptions() {
}
/**
* Apply common command-line to the state.
*/
@ -650,7 +645,7 @@ protected void applyOutputFormatOptions(CommandLine in, SqoopOptions out)
in.getOptionValue(ESCAPED_BY_ARG)));
out.setExplicitDelims(true);
}
if (in.hasOption(MYSQL_DELIMITERS_ARG)) {
out.setOutputEncloseRequired(false);
out.setFieldsTerminatedBy(',');

View File

@ -36,7 +36,7 @@
import com.cloudera.sqoop.SqoopOptions.InvalidOptionsException;
import com.cloudera.sqoop.cli.SqoopParser;
import com.cloudera.sqoop.cli.ToolOptions;
import com.cloudera.sqoop.shims.ShimLoader;
import com.cloudera.sqoop.config.ConfigurationHelper;
/**
* Base class for Sqoop subprograms (e.g., SqoopImport, SqoopExport, etc.)
@ -158,7 +158,7 @@ protected void setToolName(String name) {
* configureOptions()/applyOptions().
* @return an integer return code for external programs to consume. 0
* represents success; nonzero means failure.
*/
*/
public abstract int run(SqoopOptions options);
/**
@ -246,7 +246,7 @@ public SqoopOptions parseArguments(String [] args,
String [] toolArgs = args; // args after generic parser is done.
if (useGenericOptions) {
try {
toolArgs = ShimLoader.getHadoopShim().parseGenericOptions(
toolArgs = ConfigurationHelper.parseGenericOptions(
out.getConf(), args);
} catch (IOException ioe) {
ParseException pe = new ParseException(

View File

@ -27,7 +27,6 @@
import org.apache.commons.logging.LogFactory;
import com.cloudera.sqoop.manager.ConnManager;
import com.cloudera.sqoop.shims.HadoopShim;
/**
* Utility class; returns the locations of various jars.
@ -47,17 +46,6 @@ public static String getSqoopJarPath() {
return getJarPathForClass(Jars.class);
}
/**
* @return the path to the currently-loaded shim jar.
*/
public static String getShimJarPath() {
HadoopShim h = HadoopShim.get();
if (null == h) {
return null;
}
return getJarPathForClass(h.getClass());
}
/**
* Return the jar file path that contains a particular class.
* Method mostly cloned from o.a.h.mapred.JobConf.findContainingJar().

View File

@ -23,7 +23,7 @@
import org.apache.hadoop.conf.Configuration;
import com.cloudera.sqoop.shims.HadoopShim;
import com.cloudera.sqoop.config.ConfigurationConstants;
/**
* Utility class; returns task attempt Id of the current job
@ -52,7 +52,8 @@ public static String get(Configuration conf, String defaultVal) {
* task attempt can be stored.
*/
public static File getLocalWorkPath(Configuration conf) throws IOException {
String tmpDir = conf.get(HadoopShim.get().getJobLocalDirProperty(),
String tmpDir = conf.get(
ConfigurationConstants.PROP_JOB_LOCAL_DIRECTORY,
"/tmp/");
// Create a local subdir specific to this task attempt.

View File

@ -39,7 +39,7 @@ fi
# Run main compilation step.
${ANT} clean jar-all-shims findbugs javadoc cobertura checkstyle \
${ANT} clean jar-all findbugs javadoc cobertura checkstyle \
-Divy.home=$IVY_HOME -Dhadoop.dist=${COMPILE_HADOOP_DIST} \
-Dcobertura.home=${COBERTURA_HOME} -Dcobertura.format=xml \
-Dfindbugs.home=${FINDBUGS_HOME} \

View File

@ -1,134 +0,0 @@
/**
* Licensed to Cloudera, Inc. under one
* or more contributor license agreements. See the NOTICE file
* distributed with this work for additional information
* regarding copyright ownership. Cloudera, Inc. 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 com.cloudera.sqoop.shims;
import java.io.IOException;
import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.fs.Path;
import org.apache.hadoop.mapreduce.Counters;
import org.apache.hadoop.mapreduce.InputSplit;
import org.apache.hadoop.mapreduce.Job;
import org.apache.hadoop.mapreduce.JobContext;
import org.apache.hadoop.mapreduce.MapContext;
import org.apache.hadoop.mapreduce.OutputCommitter;
import org.apache.hadoop.mapreduce.TaskAttemptID;
import org.apache.hadoop.mapreduce.TaskCounter;
import org.apache.hadoop.mapreduce.TaskType;
import org.apache.hadoop.mapreduce.lib.input.FileSplit;
import org.apache.hadoop.mapreduce.lib.output.FileOutputCommitter;
import org.apache.hadoop.mapreduce.server.jobtracker.JTConfig;
import org.apache.hadoop.mapreduce.task.MapContextImpl;
import org.apache.hadoop.mrunit.mapreduce.mock.MockReporter;
/**
* Hadoop Shim for Apache 0.22.
*/
public class Apache22HadoopShim extends CommonHadoopShim {
@Override
public long getNumMapOutputRecords(Job job)
throws IOException, InterruptedException {
return job.getCounters().findCounter(
TaskCounter.MAP_OUTPUT_RECORDS).getValue();
}
@Override
public long getNumMapInputRecords(Job job)
throws IOException, InterruptedException {
return job.getCounters().findCounter(
TaskCounter.MAP_INPUT_RECORDS).getValue();
}
@Override
public String getTaskIdProperty() {
return JobContext.TASK_ID;
}
@Override
public String getJobLocalDirProperty() {
return JobContext.JOB_LOCAL_DIR;
}
@Override
public void setJobNumMaps(Job job, int numMapTasks) {
job.getConfiguration().setInt(JobContext.NUM_MAPS, numMapTasks);
}
@Override
public int getJobNumMaps(JobContext job) {
return job.getConfiguration().getInt(JobContext.NUM_MAPS, 1);
}
@Override
public int getConfNumMaps(Configuration conf) {
return conf.getInt(JobContext.NUM_MAPS, 1);
}
@Override
public void setJobMapSpeculativeExecution(Job job, boolean isEnabled) {
job.setMapSpeculativeExecution(isEnabled);
}
@Override
public void setJobReduceSpeculativeExecution(Job job, boolean isEnabled) {
job.setReduceSpeculativeExecution(isEnabled);
}
@Override
public void setJobtrackerAddr(Configuration conf, String addr) {
conf.set(JTConfig.JT_IPC_ADDRESS, "local");
}
private static class MockMapContextWithCommitter
extends MapContextImpl<Object, Object, Object, Object> {
private Configuration conf;
private Path path;
public MockMapContextWithCommitter(Configuration c, Path p) {
super(c, new TaskAttemptID("jt", 0, TaskType.MAP, 0, 0),
null, null, null, new MockReporter(new Counters()), null);
this.conf = c;
this.path = p;
}
@Override
public InputSplit getInputSplit() {
return new FileSplit(new Path(path, "inputFile"), 0, 0, new String[0]);
}
@Override
public Configuration getConfiguration() {
return conf;
}
@Override
public OutputCommitter getOutputCommitter() {
try {
return new FileOutputCommitter(path, this);
} catch (IOException ioe) {
return null;
}
}
}
@Override
public MapContext getMapContextForIOPath(Configuration conf, Path p) {
return new MockMapContextWithCommitter(conf, p);
}
}

View File

@ -1,75 +0,0 @@
/**
* Licensed to Cloudera, Inc. under one
* or more contributor license agreements. See the NOTICE file
* distributed with this work for additional information
* regarding copyright ownership. Cloudera, Inc. 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 com.cloudera.sqoop.shims;
import java.io.IOException;
import org.apache.hadoop.conf.Configuration;
import com.cloudera.sqoop.mapreduce.db.DBConfiguration;
import org.apache.hadoop.util.GenericOptionsParser;
/**
* Contains code which belongs in all Hadoop shims which is syntactically
* identical in both, but needs to be recompiled against multiple different
* Hadoop versions (e.g., references to 'final static String' fields).
*/
public abstract class CommonHadoopShim extends HadoopShim {
@Override
public String [] parseGenericOptions(Configuration conf, String [] args)
throws IOException {
// This needs to be shimmed because in Apache Hadoop this can throw
// an IOException, but it does not do so in CDH. We just mandate in
// this method that an IOException is possible.
GenericOptionsParser genericParser = new GenericOptionsParser(
conf, args);
return genericParser.getRemainingArgs();
}
@Override
public String getDbInputClassProperty() {
return DBConfiguration.INPUT_CLASS_PROPERTY;
}
@Override
public String getDbUsernameProperty() {
return DBConfiguration.USERNAME_PROPERTY;
}
@Override
public String getDbPasswordProperty() {
return DBConfiguration.PASSWORD_PROPERTY;
}
@Override
public String getDbUrlProperty() {
return DBConfiguration.URL_PROPERTY;
}
@Override
public String getDbInputTableNameProperty() {
return DBConfiguration.INPUT_TABLE_NAME_PROPERTY;
}
@Override
public String getDbInputConditionsProperty() {
return DBConfiguration.INPUT_CONDITIONS_PROPERTY;
}
}

View File

@ -18,7 +18,9 @@
package com.cloudera.sqoop.lib;
import java.io.*;
import java.io.IOException;
import java.io.InputStream;
import java.io.Reader;
import java.sql.ResultSet;
import java.sql.SQLException;
@ -29,7 +31,8 @@
import org.apache.hadoop.fs.Path;
import org.apache.hadoop.mapreduce.MapContext;
import org.apache.hadoop.mapreduce.lib.output.FileOutputFormat;
import com.cloudera.sqoop.shims.HadoopShim;
import com.cloudera.sqoop.testutil.MockObjectFactory;
import com.cloudera.sqoop.testutil.MockResultSet;
/**
@ -61,7 +64,7 @@ public void setUp() throws IOException, InterruptedException {
* getInputSplit() to determine where to read our source data from--the same
* directory. We are repurposing the same context for both output and input.
*/
mapContext = HadoopShim.get().getMapContextForIOPath(conf, outDir);
mapContext = MockObjectFactory.getMapContextForIOPath(conf, outDir);
loader = new LargeObjectLoader(mapContext.getConfiguration(),
FileOutputFormat.getWorkOutputPath(mapContext));
}

View File

@ -32,7 +32,7 @@
import com.cloudera.sqoop.SqoopOptions;
import com.cloudera.sqoop.SqoopOptions.InvalidOptionsException;
import com.cloudera.sqoop.shims.HadoopShim;
import com.cloudera.sqoop.config.ConfigurationHelper;
import com.cloudera.sqoop.testutil.CommonArgs;
import com.cloudera.sqoop.testutil.HsqldbTestServer;
import com.cloudera.sqoop.testutil.ImportJobTestCase;
@ -50,7 +50,7 @@ public class TestParseMethods extends ImportJobTestCase {
* Create the argv to pass to Sqoop.
* @return the argv as an array of strings.
*/
private String [] getArgv(boolean includeHadoopFlags, String fieldTerminator,
private String [] getArgv(boolean includeHadoopFlags, String fieldTerminator,
String lineTerminator, String encloser, String escape,
boolean encloserRequired) {
@ -118,7 +118,7 @@ public void runParseTest(String fieldTerminator, String lineTerminator,
job.set(ReparseMapper.USER_TYPE_NAME_KEY, tableClassName);
// use local mode in the same JVM.
HadoopShim.get().setJobtrackerAddr(job, "local");
ConfigurationHelper.setJobtrackerAddr(job, "local");
job.set("fs.default.name", "file:///");
String warehouseDir = getWarehouseDir();
@ -203,7 +203,7 @@ public void testNumericTypes() throws IOException {
"92104916282869291837672829102857271948687.287475322",
"true",
};
createTableWithColTypes(types, vals);
runParseTest(",", "\\n", "\\\'", "\\", false);
}

View File

@ -39,7 +39,6 @@
import com.cloudera.sqoop.SqoopOptions;
import com.cloudera.sqoop.manager.ConnManager;
import com.cloudera.sqoop.metastore.JobData;
import com.cloudera.sqoop.shims.ShimLoader;
import com.cloudera.sqoop.tool.ImportTool;
import junit.framework.TestCase;
@ -168,7 +167,6 @@ protected SqoopOptions getSqoopOptions(Configuration conf) {
@Before
public void setUp() {
ShimLoader.getHadoopShim();
incrementTableNum();
if (!isLog4jConfigured) {

View File

@ -15,7 +15,8 @@
* See the License for the specific language governing permissions and
* limitations under the License.
*/
package com.cloudera.sqoop.shims;
package com.cloudera.sqoop.testutil;
import java.io.IOException;
@ -23,8 +24,6 @@
import org.apache.hadoop.fs.Path;
import org.apache.hadoop.mapreduce.Counters;
import org.apache.hadoop.mapreduce.InputSplit;
import org.apache.hadoop.mapreduce.Job;
import org.apache.hadoop.mapreduce.JobContext;
import org.apache.hadoop.mapreduce.MapContext;
import org.apache.hadoop.mapreduce.OutputCommitter;
import org.apache.hadoop.mapreduce.TaskAttemptID;
@ -33,65 +32,17 @@
import org.apache.hadoop.mrunit.mapreduce.mock.MockReporter;
/**
* Hadoop Shim for CDH3 (based on 0.20.2).
* Allows the creation of various mock objects for testing purposes.
*/
public class CDH3Shim extends CommonHadoopShim {
@Override
public long getNumMapOutputRecords(Job job)
throws IOException, InterruptedException {
return job.getCounters().findCounter(
"org.apache.hadoop.mapred.Task$Counter",
"MAP_OUTPUT_RECORDS").getValue();
}
public final class MockObjectFactory {
@Override
public long getNumMapInputRecords(Job job)
throws IOException, InterruptedException {
return job.getCounters().findCounter(
"org.apache.hadoop.mapred.Task$Counter",
"MAP_INPUT_RECORDS").getValue();
}
@Override
public String getTaskIdProperty() {
return "mapred.task.id";
}
@Override
public String getJobLocalDirProperty() {
return "job.local.dir";
}
@Override
public void setJobNumMaps(Job job, int numMapTasks) {
job.getConfiguration().setInt("mapred.map.tasks", numMapTasks);
}
@Override
public int getJobNumMaps(JobContext job) {
return job.getConfiguration().getInt("mapred.map.tasks", 1);
}
@Override
public int getConfNumMaps(Configuration conf) {
return conf.getInt("mapred.map.tasks", 1);
}
@Override
public void setJobMapSpeculativeExecution(Job job, boolean isEnabled) {
job.getConfiguration().setBoolean(
"mapred.map.tasks.speculative.execution", isEnabled);
}
@Override
public void setJobReduceSpeculativeExecution(Job job, boolean isEnabled) {
job.getConfiguration().setBoolean(
"mapred.reduce.tasks.speculative.execution", isEnabled);
}
@Override
public void setJobtrackerAddr(Configuration conf, String addr) {
conf.set("mapred.job.tracker", addr);
/**
* Returns a mock MapContext that has both an OutputCommitter and an
* InputSplit wired to the specified path.
* Used for testing LargeObjectLoader.
*/
public static MapContext getMapContextForIOPath(Configuration conf, Path p) {
return new MockMapContextWithCommitter(conf, p);
}
private static class MockMapContextWithCommitter
@ -101,7 +52,7 @@ private static class MockMapContextWithCommitter
public MockMapContextWithCommitter(Configuration c, Path p) {
super(c, new TaskAttemptID("jt", 0, true, 0, 0),
null, null, null, new MockReporter(new Counters()), null);
null, null, null, new MockReporter(new Counters()), null);
this.path = p;
this.conf = c;
@ -127,8 +78,7 @@ public Configuration getConfiguration() {
}
}
@Override
public MapContext getMapContextForIOPath(Configuration conf, Path p) {
return new MockMapContextWithCommitter(conf, p);
private MockObjectFactory() {
// Disable explicity object creation
}
}