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

SQOOP-933: Properly log SQLExceptions using getNextException() method for chaining exceptions

(Jarek Jarcec Cecho via Abhijeet Gaikwad)
This commit is contained in:
Abhijeet Gaikwad 2013-03-11 10:30:41 +05:30
parent 1d0ebc2730
commit 05976e709d
13 changed files with 139 additions and 110 deletions

View File

@ -29,6 +29,7 @@
import org.apache.commons.logging.LogFactory;
import com.cloudera.sqoop.SqoopOptions;
import org.apache.sqoop.util.LoggingUtils;
/**
* Database manager that queries catalog tables directly
@ -66,23 +67,23 @@ public String[] listDatabases() {
c.rollback();
}
} catch (SQLException ce) {
LOG.error("Failed to rollback transaction", ce);
LoggingUtils.logAll(LOG, "Failed to rollback transaction", ce);
}
LOG.error("Failed to list databases", sqle);
LoggingUtils.logAll(LOG, "Failed to list databases", sqle);
throw new RuntimeException(sqle);
} finally {
if (rs != null) {
try {
rs.close();
} catch (SQLException re) {
LOG.error("Failed to close resultset", re);
LoggingUtils.logAll(LOG, "Failed to close resultset", re);
}
}
if (s != null) {
try {
s.close();
} catch (SQLException se) {
LOG.error("Failed to close statement", se);
LoggingUtils.logAll(LOG, "Failed to close statement", se);
}
}
}
@ -111,23 +112,23 @@ public String[] listTables() {
c.rollback();
}
} catch (SQLException ce) {
LOG.error("Failed to rollback transaction", ce);
LoggingUtils.logAll(LOG, "Failed to rollback transaction", ce);
}
LOG.error("Failed to list tables", sqle);
LoggingUtils.logAll(LOG, "Failed to list tables", sqle);
throw new RuntimeException(sqle);
} finally {
if (rs != null) {
try {
rs.close();
} catch (SQLException re) {
LOG.error("Failed to close resultset", re);
LoggingUtils.logAll(LOG, "Failed to close resultset", re);
}
}
if (s != null) {
try {
s.close();
} catch (SQLException se) {
LOG.error("Failed to close statement", se);
LoggingUtils.logAll(LOG, "Failed to close statement", se);
}
}
}
@ -157,23 +158,24 @@ public String[] getColumnNames(String tableName) {
c.rollback();
}
} catch (SQLException ce) {
LOG.error("Failed to rollback transaction", ce);
LoggingUtils.logAll(LOG, "Failed to rollback transaction", ce);
}
LOG.error("Failed to list columns from query: " + listColumnsQuery, sqle);
LoggingUtils.logAll(LOG, "Failed to list columns from query: "
+ listColumnsQuery, sqle);
throw new RuntimeException(sqle);
} finally {
if (rs != null) {
try {
rs.close();
} catch (SQLException re) {
LOG.error("Failed to close resultset", re);
LoggingUtils.logAll(LOG, "Failed to close resultset", re);
}
}
if (s != null) {
try {
s.close();
} catch (SQLException se) {
LOG.error("Failed to close statement", se);
LoggingUtils.logAll(LOG, "Failed to close statement", se);
}
}
}
@ -206,23 +208,23 @@ public String getPrimaryKey(String tableName) {
c.rollback();
}
} catch (SQLException ce) {
LOG.error("Failed to rollback transaction", ce);
LoggingUtils.logAll(LOG, "Failed to rollback transaction", ce);
}
LOG.error("Failed to list primary key", sqle);
LoggingUtils.logAll(LOG, "Failed to list primary key", sqle);
throw new RuntimeException(sqle);
} finally {
if (rs != null) {
try {
rs.close();
} catch (SQLException re) {
LOG.error("Failed to close resultset", re);
LoggingUtils.logAll(LOG, "Failed to close resultset", re);
}
}
if (s != null) {
try {
s.close();
} catch (SQLException se) {
LOG.error("Failed to close statement", se);
LoggingUtils.logAll(LOG, "Failed to close statement", se);
}
}
}

View File

@ -33,6 +33,7 @@
import com.cloudera.sqoop.mapreduce.JdbcExportJob;
import com.cloudera.sqoop.util.ExportException;
import com.cloudera.sqoop.util.ImportException;
import org.apache.sqoop.util.LoggingUtils;
/**
* Manages connections to DB2 databases. Requires the DB2 JDBC driver.
@ -106,16 +107,16 @@ public String[] listDatabases() {
conn.rollback();
}
} catch (SQLException ce) {
LOG.error("Failed to rollback transaction", ce);
LoggingUtils.logAll(LOG, "Failed to rollback transaction", ce);
}
LOG.error("Failed to list databases", sqle);
LoggingUtils.logAll(LOG, "Failed to list databases", sqle);
throw new RuntimeException(sqle);
} finally {
if (rset != null) {
try {
rset.close();
} catch (SQLException re) {
LOG.error("Failed to close resultset", re);
LoggingUtils.logAll(LOG, "Failed to close resultset", re);
}
}
}

View File

@ -30,13 +30,13 @@
import org.apache.avro.Schema.Type;
import org.apache.commons.logging.Log;
import org.apache.commons.logging.LogFactory;
import org.apache.hadoop.util.StringUtils;
import com.cloudera.sqoop.SqoopOptions;
import com.cloudera.sqoop.util.ImportException;
import com.cloudera.sqoop.util.ExportException;
import com.cloudera.sqoop.mapreduce.JdbcUpsertExportJob;
import org.apache.sqoop.mapreduce.mysql.MySQLUpsertOutputFormat;
import org.apache.sqoop.util.LoggingUtils;
/**
* Manages connections to MySQL databases.
@ -213,8 +213,7 @@ public void execAndPrint(String s) {
// Explicitly setting fetchSize to zero disables streaming.
results = super.execute(s, 0);
} catch (SQLException sqlE) {
LOG.error("Error executing statement: "
+ StringUtils.stringifyException(sqlE));
LoggingUtils.logAll(LOG, "Error executing statement: ", sqlE);
release();
return;
}

View File

@ -50,6 +50,7 @@
import com.cloudera.sqoop.mapreduce.db.OracleDataDrivenDBInputFormat;
import com.cloudera.sqoop.util.ExportException;
import com.cloudera.sqoop.util.ImportException;
import org.apache.sqoop.util.LoggingUtils;
/**
* Manages connections to Oracle databases.
@ -337,7 +338,7 @@ private void setSessionTimeZone(Connection conn) throws SQLException {
"setSessionTimeZone", new Class [] {String.class});
} catch (Exception ex) {
LOG.error("Could not find method setSessionTimeZone in "
+ conn.getClass().getName(), ex);
+ conn.getClass().getName(), ex);
// rethrow SQLException
throw new SQLException(ex);
}
@ -613,8 +614,8 @@ public String[] listDatabases() {
} catch (SQLException e) {
try {
conn.rollback();
} catch (Exception ex) {
LOG.error("Failed to rollback transaction", ex);
} catch (SQLException ex) {
LoggingUtils.logAll(LOG, "Failed to rollback transaction", ex);
}
if (e.getErrorCode() == ERROR_TABLE_OR_VIEW_DOES_NOT_EXIST) {
@ -623,28 +624,28 @@ public String[] listDatabases() {
+ "Please check privileges and try again.");
LOG.debug("Full trace for ORA-00942 exception", e);
} else {
LOG.error("Failed to list databases", e);
LoggingUtils.logAll(LOG, "Failed to list databases", e);
}
} finally {
if (rset != null) {
try {
rset.close();
} catch (SQLException ex) {
LOG.error("Failed to close resultset", ex);
LoggingUtils.logAll(LOG, "Failed to close resultset", ex);
}
}
if (stmt != null) {
try {
stmt.close();
} catch (Exception ex) {
LOG.error("Failed to close statement", ex);
} catch (SQLException ex) {
LoggingUtils.logAll(LOG, "Failed to close statement", ex);
}
}
try {
close();
} catch (SQLException ex) {
LOG.error("Unable to discard connection", ex);
LoggingUtils.logAll(LOG, "Unable to discard connection", ex);
}
}
@ -676,30 +677,30 @@ public String[] listTables() {
} catch (SQLException e) {
try {
conn.rollback();
} catch (Exception ex) {
LOG.error("Failed to rollback transaction", ex);
} catch (SQLException ex) {
LoggingUtils.logAll(LOG, "Failed to rollback transaction", ex);
}
LOG.error("Failed to list tables", e);
LoggingUtils.logAll(LOG, "Failed to list tables", e);
} finally {
if (rset != null) {
try {
rset.close();
} catch (SQLException ex) {
LOG.error("Failed to close resultset", ex);
LoggingUtils.logAll(LOG, "Failed to close resultset", ex);
}
}
if (pStmt != null) {
try {
pStmt.close();
} catch (Exception ex) {
LOG.error("Failed to close statement", ex);
} catch (SQLException ex) {
LoggingUtils.logAll(LOG, "Failed to close statement", ex);
}
}
try {
close();
} catch (SQLException ex) {
LOG.error("Unable to discard connection", ex);
LoggingUtils.logAll(LOG, "Unable to discard connection", ex);
}
}
@ -739,30 +740,30 @@ public String[] getColumnNames(String tableName) {
} catch (SQLException e) {
try {
conn.rollback();
} catch (Exception ex) {
LOG.error("Failed to rollback transaction", ex);
} catch (SQLException ex) {
LoggingUtils.logAll(LOG, "Failed to rollback transaction", ex);
}
LOG.error("Failed to list columns", e);
LoggingUtils.logAll(LOG, "Failed to list columns", e);
} finally {
if (rset != null) {
try {
rset.close();
} catch (SQLException ex) {
LOG.error("Failed to close resultset", ex);
LoggingUtils.logAll(LOG, "Failed to close resultset", ex);
}
}
if (pStmt != null) {
try {
pStmt.close();
} catch (Exception ex) {
LOG.error("Failed to close statement", ex);
} catch (SQLException ex) {
LoggingUtils.logAll(LOG, "Failed to close statement", ex);
}
}
try {
close();
} catch (SQLException ex) {
LOG.error("Unable to discard connection", ex);
LoggingUtils.logAll(LOG, "Unable to discard connection", ex);
}
}
@ -800,30 +801,30 @@ public String getPrimaryKey(String tableName) {
} catch (SQLException e) {
try {
conn.rollback();
} catch (Exception ex) {
LOG.error("Failed to rollback transaction", ex);
} catch (SQLException ex) {
LoggingUtils.logAll(LOG, "Failed to rollback transaction", ex);
}
LOG.error("Failed to list columns", e);
LoggingUtils.logAll(LOG, "Failed to list columns", e);
} finally {
if (rset != null) {
try {
rset.close();
} catch (SQLException ex) {
LOG.error("Failed to close resultset", ex);
LoggingUtils.logAll(LOG, "Failed to close resultset", ex);
}
}
if (pStmt != null) {
try {
pStmt.close();
} catch (Exception ex) {
LOG.error("Failed to close statement", ex);
} catch (SQLException ex) {
LoggingUtils.logAll(LOG, "Failed to close statement", ex);
}
}
try {
close();
} catch (SQLException ex) {
LOG.error("Unable to discard connection", ex);
LoggingUtils.logAll(LOG, "Unable to discard connection", ex);
}
}

View File

@ -52,8 +52,8 @@
import org.apache.commons.logging.Log;
import org.apache.commons.logging.LogFactory;
import org.apache.hadoop.util.StringUtils;
import org.apache.sqoop.mapreduce.JdbcCallExportJob;
import org.apache.sqoop.util.LoggingUtils;
import org.apache.sqoop.util.SqlTypeMap;
/**
@ -128,7 +128,8 @@ public String[] getColumnNamesForRawQuery(String stmt) {
try {
results = execute(stmt);
} catch (SQLException sqlE) {
LOG.error("Error executing statement: " + sqlE.toString(), sqlE);
LoggingUtils.logAll(LOG, "Error executing statement: " + sqlE.toString(),
sqlE);
release();
return null;
}
@ -149,7 +150,7 @@ public String[] getColumnNamesForRawQuery(String stmt) {
}
return columns.toArray(new String[0]);
} catch (SQLException sqlException) {
LOG.error("Error reading from database: "
LoggingUtils.logAll(LOG, "Error reading from database: "
+ sqlException.toString(), sqlException);
return null;
} finally {
@ -157,7 +158,8 @@ public String[] getColumnNamesForRawQuery(String stmt) {
results.close();
getConnection().commit();
} catch (SQLException sqlE) {
LOG.warn("SQLException closing ResultSet: " + sqlE.toString(), sqlE);
LoggingUtils.logAll(LOG, "SQLException closing ResultSet: "
+ sqlE.toString(), sqlE);
}
release();
@ -200,7 +202,7 @@ public String[] getColumnNamesForProcedure(String procedureName) {
getConnection().commit();
}
} catch (SQLException e) {
LOG.error("Error reading procedure metadata: ", e);
LoggingUtils.logAll(LOG, "Error reading procedure metadata: ", e);
throw new RuntimeException("Can't fetch column names for procedure.", e);
}
}
@ -234,7 +236,8 @@ protected Map<String, Integer> getColumnTypesForRawQuery(String stmt) {
try {
results = execute(stmt);
} catch (SQLException sqlE) {
LOG.error("Error executing statement: " + sqlE.toString(), sqlE);
LoggingUtils.logAll(LOG, "Error executing statement: " + sqlE.toString(),
sqlE);
release();
return null;
}
@ -262,14 +265,16 @@ protected Map<String, Integer> getColumnTypesForRawQuery(String stmt) {
return colTypes;
} catch (SQLException sqlException) {
LOG.error("Error reading from database: " + sqlException.toString());
LoggingUtils.logAll(LOG, "Error reading from database: "
+ sqlException.toString(), sqlException);
return null;
} finally {
try {
results.close();
getConnection().commit();
} catch (SQLException sqlE) {
LOG.warn("SQLException closing ResultSet: " + sqlE.toString());
LoggingUtils.logAll(LOG,
"SQLException closing ResultSet: " + sqlE.toString(), sqlE);
}
release();
@ -294,7 +299,8 @@ protected Map<String, String> getColumnTypeNamesForRawQuery(String stmt) {
try {
results = execute(stmt);
} catch (SQLException sqlE) {
LOG.error("Error executing statement: " + sqlE.toString(), sqlE);
LoggingUtils.logAll(LOG, "Error executing statement: " + sqlE.toString(),
sqlE);
release();
return null;
}
@ -317,14 +323,16 @@ protected Map<String, String> getColumnTypeNamesForRawQuery(String stmt) {
return colTypeNames;
} catch (SQLException sqlException) {
LOG.error("Error reading from database: " + sqlException.toString());
LoggingUtils.logAll(LOG, "Error reading from database: "
+ sqlException.toString(), sqlException);
return null;
} finally {
try {
results.close();
getConnection().commit();
} catch (SQLException sqlE) {
LOG.warn("SQLException closing ResultSet: " + sqlE.toString());
LoggingUtils.logAll(LOG, "SQLException closing ResultSet: "
+ sqlE.toString(), sqlE);
}
release();
@ -395,8 +403,8 @@ public Map<String, Integer> getColumnTypesForProcedure(String procedureName) {
getConnection().commit();
}
} catch (SQLException sqlException) {
LOG.error("Error reading primary key metadata: "
+ sqlException.toString());
LoggingUtils.logAll(LOG, "Error reading primary key metadata: "
+ sqlException.toString(), sqlException);
return null;
}
}
@ -410,8 +418,8 @@ public String[] listTables() {
DatabaseMetaData metaData = this.getConnection().getMetaData();
results = metaData.getTables(null, null, null, tableTypes);
} catch (SQLException sqlException) {
LOG.error("Error reading database metadata: "
+ sqlException.toString());
LoggingUtils.logAll(LOG, "Error reading database metadata: "
+ sqlException.toString(), sqlException);
return null;
}
@ -428,7 +436,8 @@ public String[] listTables() {
return tables.toArray(new String[0]);
} catch (SQLException sqlException) {
LOG.error("Error reading from database: " + sqlException.toString());
LoggingUtils.logAll(LOG, "Error reading from database: "
+ sqlException.toString(), sqlException);
return null;
}
} finally {
@ -437,7 +446,8 @@ public String[] listTables() {
results.close();
getConnection().commit();
} catch (SQLException sqlE) {
LOG.warn("Exception closing ResultSet: " + sqlE.toString());
LoggingUtils.logAll(LOG, "Exception closing ResultSet: "
+ sqlE.toString(), sqlE);
}
}
}
@ -463,8 +473,8 @@ public String getPrimaryKey(String tableName) {
getConnection().commit();
}
} catch (SQLException sqlException) {
LOG.error("Error reading primary key metadata: "
+ sqlException.toString());
LoggingUtils.logAll(LOG, "Error reading primary key metadata: "
+ sqlException.toString(), sqlException);
return null;
}
}
@ -656,7 +666,8 @@ protected void formatAndPrintResultSet(ResultSet results, PrintWriter pw) {
}
}
} catch (SQLException sqlE) {
LOG.error("SQLException reading result metadata: " + sqlE.toString());
LoggingUtils.logAll(LOG, "SQLException reading result metadata: "
+ sqlE.toString(), sqlE);
}
try {
@ -670,7 +681,8 @@ protected void formatAndPrintResultSet(ResultSet results, PrintWriter pw) {
results.close();
getConnection().commit();
} catch (SQLException sqlE) {
LOG.warn("SQLException closing ResultSet: " + sqlE.toString());
LoggingUtils.logAll(LOG, "SQLException closing ResultSet: "
+ sqlE.toString(), sqlE);
}
release();
@ -686,8 +698,7 @@ public void execAndPrint(String s) {
try {
results = execute(s);
} catch (SQLException sqlE) {
LOG.error("Error executing statement: "
+ StringUtils.stringifyException(sqlE));
LoggingUtils.logAll(LOG, "Error executing statement: ", sqlE);
release();
return;
}
@ -786,7 +797,8 @@ public void release() {
try {
this.lastStatement.close();
} catch (SQLException e) {
LOG.warn("Exception closing executed Statement: " + e);
LoggingUtils.logAll(LOG, "Exception closing executed Statement: "
+ e, e);
}
this.lastStatement = null;
@ -831,7 +843,8 @@ public Timestamp getCurrentDbTimestamp() {
return rs.getTimestamp(1);
} catch (SQLException sqlE) {
LOG.warn("SQL exception accessing current timestamp: " + sqlE);
LoggingUtils.logAll(LOG, "SQL exception accessing current timestamp: "
+ sqlE, sqlE);
return null;
} finally {
try {
@ -839,7 +852,8 @@ public Timestamp getCurrentDbTimestamp() {
rs.close();
}
} catch (SQLException sqlE) {
LOG.warn("SQL Exception closing resultset: " + sqlE);
LoggingUtils.logAll(LOG, "SQL Exception closing resultset: "
+ sqlE, sqlE);
}
try {
@ -847,7 +861,8 @@ public Timestamp getCurrentDbTimestamp() {
s.close();
}
} catch (SQLException sqlE) {
LOG.warn("SQL Exception closing statement: " + sqlE);
LoggingUtils.logAll(LOG, "SQL Exception closing statement: "
+ sqlE, sqlE);
}
}
}
@ -870,21 +885,22 @@ public long getTableRowCount(String tableName) throws SQLException {
rset.next();
result = rset.getLong(1);
} catch (SQLException ex) {
LOG.error("Unable to query count * for table " + tableName, ex);
LoggingUtils.logAll(LOG, "Unable to query count * for table "
+ tableName, ex);
throw ex;
} finally {
if (rset != null) {
try {
rset.close();
} catch (SQLException ex) {
LOG.error("Unable to close result set", ex);
LoggingUtils.logAll(LOG, "Unable to close result set", ex);
}
}
if (stmt != null) {
try {
stmt.close();
} catch (SQLException ex) {
LOG.error("Unable to close statement", ex);
LoggingUtils.logAll(LOG, "Unable to close statement", ex);
}
}
}
@ -907,14 +923,15 @@ public void deleteAllRecords(String tableName) throws SQLException {
conn.commit();
LOG.info("Deleted " + updateCount + " records from " + tableName);
} catch (SQLException ex) {
LOG.error("Unable to execute delete query: " + deleteQuery, ex);
LoggingUtils.logAll(LOG, "Unable to execute delete query: "
+ deleteQuery, ex);
throw ex;
} finally {
if (stmt != null) {
try {
stmt.close();
} catch (SQLException ex) {
LOG.error("Unable to close statement", ex);
LoggingUtils.logAll(LOG, "Unable to close statement", ex);
}
}
}
@ -953,7 +970,7 @@ public void migrateData(String fromTable, String toTable)
}
conn.commit();
} catch (SQLException ex) {
LOG.error("Unable to migrate data from "
LoggingUtils.logAll(LOG, "Unable to migrate data from "
+ fromTable + " to " + toTable, ex);
throw ex;
} finally {
@ -961,7 +978,7 @@ public void migrateData(String fromTable, String toTable)
try {
stmt.close();
} catch (SQLException ex) {
LOG.error("Unable to close statement", ex);
LoggingUtils.logAll(LOG, "Unable to close statement", ex);
}
}
}

View File

@ -36,6 +36,7 @@
import org.apache.hadoop.mapreduce.Mapper;
import org.apache.hadoop.mapreduce.OutputFormat;
import org.apache.hadoop.mapreduce.lib.input.FileInputFormat;
import org.apache.sqoop.util.LoggingUtils;
import org.apache.sqoop.util.PerfCounters;
import com.cloudera.sqoop.SqoopOptions;
import com.cloudera.sqoop.config.ConfigurationHelper;
@ -396,9 +397,9 @@ public void runExport() throws ExportException, IOException {
LOG.info("Starting to migrate data from staging table to destination.");
cmgr.migrateData(stagingTableName, outputTableName);
} catch (SQLException ex) {
LOG.error("Failed to move data from staging table ("
+ stagingTableName + ") to target table ("
+ outputTableName + ")", ex);
LoggingUtils.logAll(LOG, "Failed to move data from staging table ("
+ stagingTableName + ") to target table ("
+ outputTableName + ")", ex);
throw new ExportException(
"Failed to move data from staging table", ex);
}

View File

@ -38,6 +38,7 @@
import org.apache.sqoop.lib.SqoopRecord;
import org.apache.hadoop.mapreduce.Mapper.Context;
import org.apache.sqoop.mapreduce.db.DBConfiguration;
import org.apache.sqoop.util.LoggingUtils;
import org.apache.sqoop.util.PostgreSQLUtils;
import org.apache.sqoop.util.Executor;
import org.apache.sqoop.util.JdbcUrl;
@ -103,13 +104,13 @@ protected void setup(Context context)
LOG.error("Unable to load JDBC driver class", ex);
throw new IOException(ex);
} catch (SQLException ex) {
LOG.error("Unable to execute statement", ex);
LoggingUtils.logAll(LOG, "Unable to execute statement", ex);
throw new IOException(ex);
} finally {
try {
conn.close();
} catch (SQLException ex) {
LOG.error("Unable to close connection", ex);
LoggingUtils.logAll(LOG, "Unable to close connection", ex);
}
}
@ -257,7 +258,7 @@ protected int doExecuteUpdate(String query) throws IOException {
LOG.error("Unable to load JDBC driver class", ex);
throw new IOException(ex);
} catch (SQLException ex) {
LOG.error("Unable to connect to database", ex);
LoggingUtils.logAll(LOG, "Unable to connect to database", ex);
throw new IOException(ex);
}
Statement stmt = null;
@ -267,20 +268,20 @@ protected int doExecuteUpdate(String query) throws IOException {
conn.commit();
return ret;
} catch (SQLException ex) {
LOG.error("Unable to execute query: " + query, ex);
LoggingUtils.logAll(LOG, "Unable to execute query: " + query, ex);
throw new IOException(ex);
} finally {
if (stmt != null) {
try {
stmt.close();
} catch (SQLException ex) {
LOG.error("Unable to close statement", ex);
LoggingUtils.logAll(LOG, "Unable to close statement", ex);
}
}
try {
conn.close();
} catch (SQLException ex) {
LOG.error("Unable to close connection", ex);
LoggingUtils.logAll(LOG, "Unable to close connection", ex);
}
}
}

View File

@ -29,6 +29,7 @@
import org.apache.hadoop.io.NullWritable;
import org.apache.hadoop.io.Text;
import org.apache.sqoop.mapreduce.db.DBConfiguration;
import org.apache.sqoop.util.LoggingUtils;
/**
@ -60,7 +61,7 @@ protected void setup(Context context)
LOG.error("Unable to load JDBC driver class", ex);
throw new IOException(ex);
} catch (SQLException ex) {
LOG.error("Unable to connect to database", ex);
LoggingUtils.logAll(LOG, "Unable to connect to database", ex);
throw new IOException(ex);
}
}
@ -79,14 +80,14 @@ public void reduce(LongWritable key, Iterable<Text> values, Context context)
}
conn.commit();
} catch (SQLException ex) {
LOG.error("Unable to execute create query.", ex);
LoggingUtils.logAll(LOG, "Unable to execute create query.", ex);
throw new IOException(ex);
} finally {
if (stmt != null) {
try {
stmt.close();
} catch (SQLException ex) {
LOG.error("Unable to close statement", ex);
LoggingUtils.logAll(LOG, "Unable to close statement", ex);
}
}
}
@ -98,7 +99,7 @@ protected void cleanup(Context context)
try {
conn.close();
} catch (SQLException ex) {
LOG.error("Unable to load JDBC driver class", ex);
LoggingUtils.logAll(LOG, "Unable to load JDBC driver class", ex);
throw new IOException(ex);
}
}

View File

@ -37,6 +37,7 @@
import com.cloudera.sqoop.config.ConfigurationHelper;
import com.cloudera.sqoop.mapreduce.db.DBConfiguration;
import org.apache.sqoop.util.LoggingUtils;
/**
* A OutputFormat that sends the reduce output to a SQL table.
@ -218,7 +219,7 @@ public void close(TaskAttemptContext context) throws IOException {
statement.close();
connection.close();
} catch (SQLException ex) {
LOG.error("Unable to close connection", ex);
LoggingUtils.logAll(LOG, "Unable to close connection", ex);
}
}
}
@ -230,7 +231,7 @@ public void write(K key, V value) throws IOException {
key.write(statement);
statement.addBatch();
} catch (SQLException e) {
LOG.error("Exception encountered", e);
LoggingUtils.logAll(LOG, "Exception encountered", e);
}
}
}

View File

@ -36,7 +36,7 @@
import com.cloudera.sqoop.mapreduce.db.DBConfiguration;
import com.cloudera.sqoop.mapreduce.db.DBInputFormat;
import com.cloudera.sqoop.util.LoggingUtils;
import org.apache.sqoop.util.LoggingUtils;
/**
* A RecordReader that reads records from a SQL table.
@ -252,14 +252,14 @@ public boolean nextKeyValue() throws IOException {
try {
statement.close();
} catch (SQLException ex) {
LOG.error("Failed to close statement", ex);
LoggingUtils.logAll(LOG, "Failed to close statement", ex);
}
}
if (this.connection != null) {
try {
connection.close();
} catch (SQLException ex) {
LOG.error("Failed to close connection", ex);
LoggingUtils.logAll(LOG, "Failed to close connection", ex);
}
}

View File

@ -158,7 +158,10 @@ public List<InputSplit> getSplits(JobContext job) throws IOException {
DBSplitter splitter = getSplitter(sqlDataType);
if (null == splitter) {
throw new IOException("Sqoop does not have the splitter for the given SQL data type. Please use either different split column (argument --split-by) or lower the number of mappers to 1. Unknown SQL data type: " + sqlDataType);
throw new IOException("Sqoop does not have the splitter for the given"
+ " SQL data type. Please use either different split column (argument"
+ " --split-by) or lower the number of mappers to 1. Unknown SQL data"
+ " type: " + sqlDataType);
}
return splitter.split(job.getConfiguration(), results,

View File

@ -32,11 +32,15 @@ public final class LoggingUtils {
private LoggingUtils() { }
/**
* Log every exception in the chain if
* the exception is a chain of exceptions.
* Log entire chain of SQLExceptions using old SQLException.getNextException
* interface instead of new Throwable.getCause().
*/
public static void logAll(Log log, SQLException e) {
log.error("Top level exception: ", e);
logAll(log, null, e);
}
public static void logAll(Log log, String message, SQLException e) {
log.error(message == null ? "Top level exception: " : message, e);
e = e.getNextException();
int indx = 1;
while (e != null) {

View File

@ -26,7 +26,6 @@
import org.apache.commons.logging.Log;
import org.apache.commons.logging.LogFactory;
import org.apache.hadoop.util.StringUtils;
/**
* Utility methods to format and print ResultSet objects.
@ -111,8 +110,7 @@ public final void printResultSet(PrintWriter pw, ResultSet results)
pw.print(sbPad.toString());
} catch (SQLException sqlException) {
LOG.error("Error reading from database: "
+ StringUtils.stringifyException(sqlException));
LoggingUtils.logAll(LOG, "Error reading from database: ", sqlException);
}
}