mirror of
https://github.com/apache/sqoop.git
synced 2025-05-10 07:31:32 +08:00
SQOOP-813: LoaderExecutor might get into deadlock when exception is raised outside Loader itself
(Jarek Jarcec Cecho via Cheolsoo Park)
This commit is contained in:
parent
612060139a
commit
5be8eb6808
@ -175,13 +175,13 @@ private class ConsumerThread implements Runnable {
|
||||
|
||||
@Override
|
||||
public void run() {
|
||||
LOG.info("SqoopOutputFormatLoadExecutor consumer thread is starting");
|
||||
try {
|
||||
DataReader reader = new OutputFormatDataReader();
|
||||
|
||||
Configuration conf = null;
|
||||
if (!isTest) {
|
||||
conf = context.getConfiguration();
|
||||
|
||||
|
||||
loaderName = conf.get(JobConstants.JOB_ETL_LOADER);
|
||||
}
|
||||
Loader loader = (Loader) ClassUtils.instantiate(loaderName);
|
||||
@ -204,15 +204,10 @@ public void run() {
|
||||
configJob = ConfigurationUtils.getFrameworkJob(conf);
|
||||
break;
|
||||
default:
|
||||
readerFinished = true;
|
||||
// Release so that the writer can tell the framework something went
|
||||
// wrong.
|
||||
free.release();
|
||||
throw new SqoopException(MapreduceExecutionError.MAPRED_EXEC_0023);
|
||||
}
|
||||
}
|
||||
|
||||
try {
|
||||
LOG.info("Running loader class " + loaderName);
|
||||
loader.load(subContext, configConnection, configJob, reader);
|
||||
LOG.info("Loader has finished");
|
||||
|
Loading…
Reference in New Issue
Block a user