You are viewing a plain text version of this content. The canonical link for it is here.
Posted to dev@sqoop.apache.org by "Jarek Jarcec Cecho (JIRA)" <ji...@apache.org> on 2012/12/09 03:57:20 UTC

[jira] [Commented] (SQOOP-738) Sqoop is not importing all data in Sqoop 2

    [ https://issues.apache.org/jira/browse/SQOOP-738?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=13527325#comment-13527325 ] 

Jarek Jarcec Cecho commented on SQOOP-738:
------------------------------------------

I was able to catch up following mapper log from task that delivered empty file:

{code}
2012-12-08 18:35:28,031 WARN mapreduce.Counters: Group org.apache.hadoop.mapred.Task$Counter is deprecated. Use org.apache.hadoop.mapreduce.TaskCounter instead
2012-12-08 18:35:29,099 WARN org.apache.hadoop.conf.Configuration: session.id is deprecated. Instead, use dfs.metrics.session-id
2012-12-08 18:35:29,101 INFO org.apache.hadoop.metrics.jvm.JvmMetrics: Initializing JVM Metrics with processName=MAP, sessionId=
2012-12-08 18:35:39,582 INFO org.apache.hadoop.util.ProcessTree: setsid exited with exit code 0
2012-12-08 18:35:39,587 INFO org.apache.hadoop.mapred.Task:  Using ResourceCalculatorPlugin : org.apache.hadoop.util.LinuxResourceCalculatorPlugin@543bc20e
2012-12-08 18:35:48,655 INFO org.apache.hadoop.mapred.Task: Task:attempt_201212071653_0005_m_000004_0 is done. And is in the process of commiting
2012-12-08 18:35:49,787 INFO org.apache.hadoop.mapred.Task: Task attempt_201212071653_0005_m_000004_0 is allowed to commit now
2012-12-08 18:35:49,858 INFO org.apache.hadoop.mapreduce.lib.output.FileOutputCommitter: Saved output of task 'attempt_201212071653_0005_m_000004_0' to /user/root/texts
2012-12-08 18:35:49,864 INFO org.apache.hadoop.mapred.Task: Task 'attempt_201212071653_0005_m_000004_0' done.
2012-12-08 18:35:51,445 INFO org.apache.hadoop.mapred.TaskLogsTruncater: Initializing logs' truncater with mapRetainSize=-1 and reduceRetainSize=-1
2012-12-08 18:35:51,490 ERROR org.apache.sqoop.job.mr.SqoopOutputFormatLoadExecutor: Error while loading data out of MR job.
org.apache.sqoop.common.SqoopException: MAPRED_EXEC_0018:Error occurs during loader run
	at org.apache.sqoop.job.etl.HdfsTextImportLoader.load(HdfsTextImportLoader.java:98)
	at org.apache.sqoop.job.mr.SqoopOutputFormatLoadExecutor$ConsumerThread.run(SqoopOutputFormatLoadExecutor.java:193)
	at java.util.concurrent.Executors$RunnableAdapter.call(Executors.java:439)
	at java.util.concurrent.FutureTask$Sync.innerRun(FutureTask.java:303)
	at java.util.concurrent.FutureTask.run(FutureTask.java:138)
	at java.util.concurrent.ThreadPoolExecutor$Worker.runTask(ThreadPoolExecutor.java:886)
	at java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:908)
	at java.lang.Thread.run(Thread.java:662)
Caused by: org.apache.hadoop.ipc.RemoteException(org.apache.hadoop.hdfs.server.namenode.LeaseExpiredException): No lease on /user/root/texts/_temporary/_attempt_201212071653_0005_m_000004_0/part-m-00004 File does not exist. [Lease.  Holder: DFSClient_NONMAPREDUCE_-243096719_1, pendingcreates: 1]
	at org.apache.hadoop.hdfs.server.namenode.FSNamesystem.checkLease(FSNamesystem.java:2308)
	at org.apache.hadoop.hdfs.server.namenode.FSNamesystem.checkLease(FSNamesystem.java:2299)
	at org.apache.hadoop.hdfs.server.namenode.FSNamesystem.completeFileInternal(FSNamesystem.java:2366)
	at org.apache.hadoop.hdfs.server.namenode.FSNamesystem.completeFile(FSNamesystem.java:2343)
	at org.apache.hadoop.hdfs.server.namenode.NameNodeRpcServer.complete(NameNodeRpcServer.java:526)
	at org.apache.hadoop.hdfs.protocolPB.ClientNamenodeProtocolServerSideTranslatorPB.complete(ClientNamenodeProtocolServerSideTranslatorPB.java:335)
	at org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos$ClientNamenodeProtocol$2.callBlockingMethod(ClientNamenodeProtocolProtos.java:44084)
	at org.apache.hadoop.ipc.ProtobufRpcEngine$Server$ProtoBufRpcInvoker.call(ProtobufRpcEngine.java:453)
	at org.apache.hadoop.ipc.RPC$Server.call(RPC.java:898)
	at org.apache.hadoop.ipc.Server$Handler$1.run(Server.java:1693)
	at org.apache.hadoop.ipc.Server$Handler$1.run(Server.java:1689)
	at java.security.AccessController.doPrivileged(Native Method)
	at javax.security.auth.Subject.doAs(Subject.java:396)
	at org.apache.hadoop.security.UserGroupInformation.doAs(UserGroupInformation.java:1332)
	at org.apache.hadoop.ipc.Server$Handler.run(Server.java:1687)

	at org.apache.hadoop.ipc.Client.call(Client.java:1160)
	at org.apache.hadoop.ipc.ProtobufRpcEngine$Invoker.invoke(ProtobufRpcEngine.java:202)
	at $Proxy10.complete(Unknown Source)
	at sun.reflect.NativeMethodAccessorImpl.invoke0(Native Method)
	at sun.reflect.NativeMethodAccessorImpl.invoke(NativeMethodAccessorImpl.java:39)
	at sun.reflect.DelegatingMethodAccessorImpl.invoke(DelegatingMethodAccessorImpl.java:25)
	at java.lang.reflect.Method.invoke(Method.java:597)
	at org.apache.hadoop.io.retry.RetryInvocationHandler.invokeMethod(RetryInvocationHandler.java:164)
	at org.apache.hadoop.io.retry.RetryInvocationHandler.invoke(RetryInvocationHandler.java:83)
	at $Proxy10.complete(Unknown Source)
	at org.apache.hadoop.hdfs.protocolPB.ClientNamenodeProtocolTranslatorPB.complete(ClientNamenodeProtocolTranslatorPB.java:329)
	at org.apache.hadoop.hdfs.DFSOutputStream.completeFile(DFSOutputStream.java:1769)
	at org.apache.hadoop.hdfs.DFSOutputStream.close(DFSOutputStream.java:1756)
	at org.apache.hadoop.fs.FSDataOutputStream$PositionCache.close(FSDataOutputStream.java:66)
	at org.apache.hadoop.fs.FSDataOutputStream.close(FSDataOutputStream.java:99)
	at sun.nio.cs.StreamEncoder.implClose(StreamEncoder.java:301)
	at sun.nio.cs.StreamEncoder.close(StreamEncoder.java:130)
	at java.io.OutputStreamWriter.close(OutputStreamWriter.java:216)
	at java.io.BufferedWriter.close(BufferedWriter.java:248)
	at org.apache.sqoop.job.etl.HdfsTextImportLoader.load(HdfsTextImportLoader.java:95)
	... 7 more
{code}

Please notice that Hadoop has committed the task (= move output file) before the exception which suggest that we were still writing output at the time the commit was happening. I believe that due to our synchronous way of moving data from mapper (reducer) to output format, it might happen that mapper finish before all data are written to disk. Sometimes when this happens, Hadoop will be fast enough to call task committer that will move output data file before we end writing, thus loosing unflushed data.
                
> Sqoop is not importing all data in Sqoop 2
> ------------------------------------------
>
>                 Key: SQOOP-738
>                 URL: https://issues.apache.org/jira/browse/SQOOP-738
>             Project: Sqoop
>          Issue Type: Bug
>    Affects Versions: 2.0.0
>            Reporter: Jarek Jarcec Cecho
>            Assignee: Jarek Jarcec Cecho
>            Priority: Blocker
>             Fix For: 1.99.1
>
>
> I've tried to import exactly 408,957 (nice rounded number right?) rows in 10 mappers and I've noticed that not all mappers will supply all the data all the time. For example in run I got 6 files with expected size of 10MB whereas the other 4 random files are completely empty. In another run I got 8 files of 10MB and just 2 files empty. I did not quite found any logic regarding how many and which files will end up empty. We definitely need to address this.

--
This message is automatically generated by JIRA.
If you think it was sent incorrectly, please contact your JIRA administrators
For more information on JIRA, see: http://www.atlassian.com/software/jira