[ https://issues.apache.org/jira/browse/IGNITE-3167?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=15435180#comment-15435180 ]
Taras Ledkov commented on IGNITE-3167: -------------------------------------- The found problems: - incorrect deserialization of the *HadoopJobInfoUpdateRequest*; - useless assert at the *HadoopV2Job* that checks the classloader is *HadoopClassLoader* - it is not true for external execution; - the test *HadoopExternalTaskExecutionSelfTest.testMapperException* checks the class of exception that is throw form hadoop job but now only string with the printed stacktrace is available; - unstable execution. The future is not completed sometimes. The remote executor process is failed with: {code} [19:04:16,605][ERROR][pool-1-thread-3][HadoopChildProcessRunner] Failed to initialize process: HadoopPrepareForJobRequest [jobId=357a0ecc-6d67-4e40-8e13-574dbe43f8e3_1, jobInfo=o.a.i.i.processors.hadoop.HadoopDefaultJobInfo@71a7f9f0, totalReducersCnt=1, locR class org.apache.ignite.IgniteCheckedException: Local job directory already exists: /tmp/ignite/work/hadoop/node-808b5711-7877-40e5-8c8b-1419b3f34c2b/job_357a0ecc-6d67-4e40-8e13-574dbe43f8e3_1 at org.apache.ignite.internal.processors.hadoop.v2.HadoopV2JobResourceManager.prepareJobEnvironment(HadoopV2JobResourceManager.java:120) at org.apache.ignite.internal.processors.hadoop.v2.HadoopV2Job.initialize(HadoopV2Job.java:305) at org.apache.ignite.internal.processors.hadoop.taskexecutor.external.child.HadoopChildProcessRunner.prepareProcess(HadoopChildProcessRunner.java:139) at org.apache.ignite.internal.processors.hadoop.taskexecutor.external.child.HadoopChildProcessRunner.access$1600(HadoopChildProcessRunner.java:62) at org.apache.ignite.internal.processors.hadoop.taskexecutor.external.child.HadoopChildProcessRunner$MessageListener.onMessageReceived(HadoopChildProcessRunner.java:420) at org.apache.ignite.internal.processors.hadoop.taskexecutor.external.communication.HadoopExternalCommunication.notifyListener(HadoopExternalCommunication.java:1105) at org.apache.ignite.internal.processors.hadoop.taskexecutor.external.communication.HadoopExternalCommunication$1.onMessage(HadoopExternalCommunication.java:153) at org.apache.ignite.internal.processors.hadoop.taskexecutor.external.communication.HadoopExternalCommunication$1.onMessage(HadoopExternalCommunication.java:120) at org.apache.ignite.internal.util.nio.GridNioFilterChain$TailFilter.onMessageReceived(GridNioFilterChain.java:270) at org.apache.ignite.internal.util.nio.GridNioFilterAdapter.proceedMessageReceived(GridNioFilterAdapter.java:107) at org.apache.ignite.internal.util.nio.GridNioAsyncNotifyFilter$3.body(GridNioAsyncNotifyFilter.java:95) at org.apache.ignite.internal.util.worker.GridWorker.run(GridWorker.java:110) at org.apache.ignite.internal.util.worker.GridWorkerPool$1.run(GridWorkerPool.java:70) at java.util.concurrent.ThreadPoolExecutor.runWorker(ThreadPoolExecutor.java:1145) at java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:615) at java.lang.Thread.run(Thread.java:745) [19:04:16,649][ERROR][pool-1-thread-3][HadoopExternalCommunication] Runtime error caught during grid runnable execution: GridWorker [name=message-received-notify, gridName=external, finished=false, isCancelled=false, hashCode=9105195, interrupted=false, runn java.lang.NullPointerException at org.apache.ignite.internal.processors.hadoop.taskexecutor.external.child.HadoopChildProcessRunner$2.apply(HadoopChildProcessRunner.java:248) at org.apache.ignite.internal.processors.hadoop.taskexecutor.external.child.HadoopChildProcessRunner$2.apply(HadoopChildProcessRunner.java:241) at org.apache.ignite.internal.util.future.GridFutureAdapter$ArrayListener.apply(GridFutureAdapter.java:441) at org.apache.ignite.internal.util.future.GridFutureAdapter$ArrayListener.apply(GridFutureAdapter.java:424) at org.apache.ignite.internal.util.future.GridFutureAdapter.notifyListener(GridFutureAdapter.java:263) at org.apache.ignite.internal.util.future.GridFutureAdapter.notifyListeners(GridFutureAdapter.java:251) at org.apache.ignite.internal.util.future.GridFutureAdapter.onDone(GridFutureAdapter.java:381) at org.apache.ignite.internal.util.future.GridFutureAdapter.onDone(GridFutureAdapter.java:347) at org.apache.ignite.internal.util.future.GridFutureAdapter.onDone(GridFutureAdapter.java:335) at org.apache.ignite.internal.processors.hadoop.taskexecutor.external.child.HadoopChildProcessRunner.prepareProcess(HadoopChildProcessRunner.java:155) at org.apache.ignite.internal.processors.hadoop.taskexecutor.external.child.HadoopChildProcessRunner.access$1600(HadoopChildProcessRunner.java:62) at org.apache.ignite.internal.processors.hadoop.taskexecutor.external.child.HadoopChildProcessRunner$MessageListener.onMessageReceived(HadoopChildProcessRunner.java:420) at org.apache.ignite.internal.processors.hadoop.taskexecutor.external.communication.HadoopExternalCommunication.notifyListener(HadoopExternalCommunication.java:1105) at org.apache.ignite.internal.processors.hadoop.taskexecutor.external.communication.HadoopExternalCommunication$1.onMessage(HadoopExternalCommunication.java:153) at org.apache.ignite.internal.processors.hadoop.taskexecutor.external.communication.HadoopExternalCommunication$1.onMessage(HadoopExternalCommunication.java:120) at org.apache.ignite.internal.util.nio.GridNioFilterChain$TailFilter.onMessageReceived(GridNioFilterChain.java:270) at org.apache.ignite.internal.util.nio.GridNioFilterAdapter.proceedMessageReceived(GridNioFilterAdapter.java:107) at org.apache.ignite.internal.util.nio.GridNioAsyncNotifyFilter$3.body(GridNioAsyncNotifyFilter.java:95) at org.apache.ignite.internal.util.worker.GridWorker.run(GridWorker.java:110) at org.apache.ignite.internal.util.worker.GridWorkerPool$1.run(GridWorkerPool.java:70) at java.util.concurrent.ThreadPoolExecutor.runWorker(ThreadPoolExecutor.java:1145) at java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:615) at java.lang.Thread.run(Thread.java:745) {code} > Hadoop: restore external execution. > ----------------------------------- > > Key: IGNITE-3167 > URL: https://issues.apache.org/jira/browse/IGNITE-3167 > Project: Ignite > Issue Type: Task > Components: hadoop > Affects Versions: 1.5.0.final > Reporter: Vladimir Ozerov > Assignee: Taras Ledkov > Priority: Critical > Labels: roadmap > > Some time ago we decided to get rid external execution mode. It appears to be > a wrong decision. > Hadoop users rely on its process-per-job nature in lot's places. One of such > case could be observed in HiBench Bayes benchmark: > 1) Job creates something in the local file system through Hadoop FileSystem > API. > 2) Then it tries to get this data using regular java.io.FileReader and > relative paths. > This doesn't work in embedded mode because our LocalFileSystem wrapper > assigns different work dirs for jobs, but process-wide working directory is > always the same. As a result, aforementioned benchmark doesn't work in > Ignite, but work with standard Hadoop job tracker. > It seems that we must return external execution back. -- This message was sent by Atlassian JIRA (v6.3.4#6332)