mirror of
https://github.com/apache/sqoop.git
synced 2025-05-04 06:10:18 +08:00
SQOOP-170. Capturing causal exception for logging.
From: Arvind Prabhakar <arvind@cloudera.com> git-svn-id: https://svn.apache.org/repos/asf/incubator/sqoop/trunk@1150024 13f79535-47bb-0310-9956-ffa450edef68
This commit is contained in:
parent
67dce5cc36
commit
3bbb4a6314
@ -215,7 +215,7 @@ conf, getConnection(), getDBConf(), conditions, fieldNames,
|
||||
tableName);
|
||||
}
|
||||
} catch (SQLException ex) {
|
||||
throw new IOException(ex.getMessage());
|
||||
throw new IOException(ex);
|
||||
}
|
||||
}
|
||||
|
||||
|
@ -98,13 +98,13 @@ public void close(TaskAttemptContext context) throws IOException {
|
||||
} catch (SQLException ex) {
|
||||
LOG.warn(StringUtils.stringifyException(ex));
|
||||
}
|
||||
throw new IOException(e.getMessage());
|
||||
throw new IOException(e);
|
||||
} finally {
|
||||
try {
|
||||
statement.close();
|
||||
connection.close();
|
||||
} catch (SQLException ex) {
|
||||
throw new IOException(ex.getMessage());
|
||||
LOG.error("Unable to close connection", ex);
|
||||
}
|
||||
}
|
||||
}
|
||||
@ -116,7 +116,7 @@ public void write(K key, V value) throws IOException {
|
||||
key.write(statement);
|
||||
statement.addBatch();
|
||||
} catch (SQLException e) {
|
||||
e.printStackTrace();
|
||||
LOG.error("Exception encountered", e);
|
||||
}
|
||||
}
|
||||
}
|
||||
@ -181,7 +181,7 @@ public RecordWriter<K, V> getRecordWriter(TaskAttemptContext context)
|
||||
constructQuery(tableName, fieldNames));
|
||||
return new DBRecordWriter(connection, statement);
|
||||
} catch (Exception ex) {
|
||||
throw new IOException(ex.getMessage());
|
||||
throw new IOException(ex);
|
||||
}
|
||||
}
|
||||
|
||||
|
@ -165,7 +165,7 @@ public void close() throws IOException {
|
||||
connection.close();
|
||||
}
|
||||
} catch (SQLException e) {
|
||||
throw new IOException(e.getMessage());
|
||||
throw new IOException(e);
|
||||
}
|
||||
}
|
||||
|
||||
|
@ -198,7 +198,7 @@ public List<InputSplit> getSplits(JobContext job) throws IOException {
|
||||
return splitter.split(job.getConfiguration(), results,
|
||||
getDBConf().getInputOrderBy());
|
||||
} catch (SQLException e) {
|
||||
throw new IOException(e.getMessage());
|
||||
throw new IOException(e);
|
||||
} finally {
|
||||
// More-or-less ignore SQL exceptions here, but log in case we need it.
|
||||
try {
|
||||
@ -293,7 +293,7 @@ conf, getConnection(), dbConf, dbConf.getInputConditions(),
|
||||
dbConf.getInputFieldNames(), dbConf.getInputTableName(),
|
||||
dbProductName);
|
||||
} catch (SQLException ex) {
|
||||
throw new IOException(ex.getMessage());
|
||||
throw new IOException(ex);
|
||||
}
|
||||
}
|
||||
|
||||
|
@ -65,7 +65,7 @@ protected RecordReader<LongWritable, T> createDBRecordReader(
|
||||
conf, getConnection(), dbConf, dbConf.getInputConditions(),
|
||||
dbConf.getInputFieldNames(), dbConf.getInputTableName());
|
||||
} catch (SQLException ex) {
|
||||
throw new IOException(ex.getMessage());
|
||||
throw new IOException(ex);
|
||||
}
|
||||
}
|
||||
}
|
||||
|
Loading…
Reference in New Issue
Block a user