mirror of
https://github.com/apache/sqoop.git
synced 2025-05-04 04:31:18 +08:00
SQOOP-934: JDBC Connection can timeout after import but before hive import
(Raghav Kumar Gautam via Jarek Jarcec Cecho)
This commit is contained in:
parent
92e94b911d
commit
217e2af3f8
@ -179,6 +179,8 @@ public void importTable(String inputTableName, String outputTableName,
|
|||||||
}
|
}
|
||||||
|
|
||||||
// generate the HQL statements to run.
|
// generate the HQL statements to run.
|
||||||
|
// reset the connection as it might have timed out
|
||||||
|
connManager.discardConnection(true);
|
||||||
TableDefWriter tableWriter = new TableDefWriter(options, connManager,
|
TableDefWriter tableWriter = new TableDefWriter(options, connManager,
|
||||||
inputTableName, outputTableName,
|
inputTableName, outputTableName,
|
||||||
configuration, !debugMode);
|
configuration, !debugMode);
|
||||||
|
@ -484,6 +484,14 @@ public abstract ResultSet readTable(String tableName, String [] columns)
|
|||||||
*/
|
*/
|
||||||
public abstract Connection getConnection() throws SQLException;
|
public abstract Connection getConnection() throws SQLException;
|
||||||
|
|
||||||
|
/**
|
||||||
|
* discard the database connection.
|
||||||
|
*/
|
||||||
|
public void discardConnection(boolean doClose) {
|
||||||
|
throw new UnsupportedOperationException("No discard connection support "
|
||||||
|
+ "for this database");
|
||||||
|
}
|
||||||
|
|
||||||
/**
|
/**
|
||||||
* @return a string identifying the driver class to load for this
|
* @return a string identifying the driver class to load for this
|
||||||
* JDBC connection type.
|
* JDBC connection type.
|
||||||
|
@ -65,9 +65,12 @@ protected boolean hasOpenConnection() {
|
|||||||
* connection first.
|
* connection first.
|
||||||
* @param doClose if true, try to close the connection before forgetting it.
|
* @param doClose if true, try to close the connection before forgetting it.
|
||||||
*/
|
*/
|
||||||
protected void discardConnection(boolean doClose) throws SQLException {
|
public void discardConnection(boolean doClose) {
|
||||||
if (doClose && hasOpenConnection()) {
|
if (doClose && hasOpenConnection()) {
|
||||||
this.connection.close();
|
try {
|
||||||
|
this.connection.close();
|
||||||
|
} catch(SQLException sqe) {
|
||||||
|
}
|
||||||
}
|
}
|
||||||
|
|
||||||
this.connection = null;
|
this.connection = null;
|
||||||
|
Loading…
Reference in New Issue
Block a user