[ https://issues.apache.org/jira/browse/HDDS-2936?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=17023762#comment-17023762 ]
Istvan Fajth commented on HDDS-2936: ------------------------------------ According to what we have find so far, it seems that the problem stems from the write path, and it is not the read which is causing the failure. We were able to read back the same data that was written to the DataNodes via standard CLI calls, and with ORC tools the meta verification, or reading the data from the file that was written to the DataNodes fails with the same readFully or an other ORC layer exception we both saw at the Hive queries. Examining the question further, it seems to be an ORC related internal which determines what will be the exception at the end. We compared the file sizes that are written by Hive to HDFS and to Ozone from the same original dataset, and we see that there are differences in the size of all files. Hive devs whom I have asked so far could not confirm if the partitions in ORC format has to be the same size in this case, so this might nor mean too much, however the readFully exception above is thrown if we read from a partition that differs 128KiB+ in size compared to HDFS, and we see a buffer size problem reported from the ORC layer if the difference is over 8KiB but under 128KiB. The next steps concentrate on to find a way to prove if a file is written with the corruption at data generation time, but files that are exhibit the problem with an exception are just a few percent of all the written files tops as we see so far. > Hive queries fail at readFully > ------------------------------ > > Key: HDDS-2936 > URL: https://issues.apache.org/jira/browse/HDDS-2936 > Project: Hadoop Distributed Data Store > Issue Type: Bug > Affects Versions: 0.5.0 > Reporter: Istvan Fajth > Assignee: Istvan Fajth > Priority: Critical > > When running Hive queries on a 1TB dataset for TPC-DS tests, we started to > see an exception coming out from FSInputStream.readFully. > This does not happen with a smaller 100GB dataset, so possibly multi block > long files are the cause of the trouble, and the issue was not seen with a > build from early december, so we most likely to blame a recent change since > then. The build I am running with is from the hash > 929f2f85d0379aab5aabeded8a4d3a5056777706 of master branch but with HDDS-2188 > reverted from the code. > The exception I see: > {code} > Error while running task ( failure ) : > attempt_1579615091731_0060_9_05_000029_3:java.lang.RuntimeException: > java.lang.RuntimeException: java.io.IOException: java.io.EOFException: End of > file reached before reading fully. > at > org.apache.hadoop.hive.ql.exec.tez.TezProcessor.initializeAndRunProcessor(TezProcessor.java:296) > at > org.apache.hadoop.hive.ql.exec.tez.TezProcessor.run(TezProcessor.java:250) > at > org.apache.tez.runtime.LogicalIOProcessorRuntimeTask.run(LogicalIOProcessorRuntimeTask.java:374) > at > org.apache.tez.runtime.task.TaskRunner2Callable$1.run(TaskRunner2Callable.java:73) > at > org.apache.tez.runtime.task.TaskRunner2Callable$1.run(TaskRunner2Callable.java:61) > at java.security.AccessController.doPrivileged(Native Method) > at javax.security.auth.Subject.doAs(Subject.java:422) > at > org.apache.hadoop.security.UserGroupInformation.doAs(UserGroupInformation.java:1876) > at > org.apache.tez.runtime.task.TaskRunner2Callable.callInternal(TaskRunner2Callable.java:61) > at > org.apache.tez.runtime.task.TaskRunner2Callable.callInternal(TaskRunner2Callable.java:37) > at org.apache.tez.common.CallableWithNdc.call(CallableWithNdc.java:36) > at > com.google.common.util.concurrent.TrustedListenableFutureTask$TrustedFutureInterruptibleTask.runInterruptibly(TrustedListenableFutureTask.java:108) > at > com.google.common.util.concurrent.InterruptibleTask.run(InterruptibleTask.java:41) > at > com.google.common.util.concurrent.TrustedListenableFutureTask.run(TrustedListenableFutureTask.java:77) > at > java.util.concurrent.ThreadPoolExecutor.runWorker(ThreadPoolExecutor.java:1149) > at > java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:624) > at java.lang.Thread.run(Thread.java:748) > Caused by: java.lang.RuntimeException: java.io.IOException: > java.io.EOFException: End of file reached before reading fully. > at > org.apache.hadoop.mapred.split.TezGroupedSplitsInputFormat$TezGroupedSplitsRecordReader.initNextRecordReader(TezGroupedSplitsInputFormat.java:206) > at > org.apache.hadoop.mapred.split.TezGroupedSplitsInputFormat$TezGroupedSplitsRecordReader.<init>(TezGroupedSplitsInputFormat.java:145) > at > org.apache.hadoop.mapred.split.TezGroupedSplitsInputFormat.getRecordReader(TezGroupedSplitsInputFormat.java:111) > at > org.apache.tez.mapreduce.lib.MRReaderMapred.setupOldRecordReader(MRReaderMapred.java:157) > at > org.apache.tez.mapreduce.lib.MRReaderMapred.setSplit(MRReaderMapred.java:83) > at > org.apache.tez.mapreduce.input.MRInput.initFromEventInternal(MRInput.java:703) > at > org.apache.tez.mapreduce.input.MRInput.initFromEvent(MRInput.java:662) > at > org.apache.tez.mapreduce.input.MRInputLegacy.checkAndAwaitRecordReaderInitialization(MRInputLegacy.java:150) > at > org.apache.tez.mapreduce.input.MRInputLegacy.init(MRInputLegacy.java:114) > at > org.apache.hadoop.hive.ql.exec.tez.MapRecordProcessor.getMRInput(MapRecordProcessor.java:532) > at > org.apache.hadoop.hive.ql.exec.tez.MapRecordProcessor.init(MapRecordProcessor.java:178) > at > org.apache.hadoop.hive.ql.exec.tez.TezProcessor.initializeAndRunProcessor(TezProcessor.java:266) > ... 16 more > Caused by: java.io.IOException: java.io.EOFException: End of file reached > before reading fully. > at > org.apache.hadoop.hive.io.HiveIOExceptionHandlerChain.handleRecordReaderCreationException(HiveIOExceptionHandlerChain.java:97) > at > org.apache.hadoop.hive.io.HiveIOExceptionHandlerUtil.handleRecordReaderCreationException(HiveIOExceptionHandlerUtil.java:57) > at > org.apache.hadoop.hive.ql.io.HiveInputFormat.getRecordReader(HiveInputFormat.java:422) > at > org.apache.hadoop.mapred.split.TezGroupedSplitsInputFormat$TezGroupedSplitsRecordReader.initNextRecordReader(TezGroupedSplitsInputFormat.java:203) > ... 27 more > Caused by: java.io.EOFException: End of file reached before reading fully. > at > org.apache.hadoop.fs.FSInputStream.readFully(FSInputStream.java:126) > at > org.apache.hadoop.fs.FSDataInputStream.readFully(FSDataInputStream.java:112) > at > org.apache.orc.impl.RecordReaderUtils$DefaultDataReader.readStripeFooter(RecordReaderUtils.java:269) > at > org.apache.orc.impl.RecordReaderImpl.readStripeFooter(RecordReaderImpl.java:308) > at > org.apache.orc.impl.RecordReaderImpl.beginReadStripe(RecordReaderImpl.java:1089) > at > org.apache.orc.impl.RecordReaderImpl.readStripe(RecordReaderImpl.java:1051) > at > org.apache.orc.impl.RecordReaderImpl.advanceStripe(RecordReaderImpl.java:1219) > at > org.apache.orc.impl.RecordReaderImpl.advanceToNextRow(RecordReaderImpl.java:1254) > at > org.apache.orc.impl.RecordReaderImpl.<init>(RecordReaderImpl.java:284) > at > org.apache.hadoop.hive.ql.io.orc.RecordReaderImpl.<init>(RecordReaderImpl.java:67) > at > org.apache.hadoop.hive.ql.io.orc.ReaderImpl.rowsOptions(ReaderImpl.java:83) > at > org.apache.hadoop.hive.ql.io.orc.VectorizedOrcAcidRowBatchReader.<init>(VectorizedOrcAcidRowBatchReader.java:145) > at > org.apache.hadoop.hive.ql.io.orc.VectorizedOrcAcidRowBatchReader.<init>(VectorizedOrcAcidRowBatchReader.java:135) > at > org.apache.hadoop.hive.ql.io.orc.OrcInputFormat.getRecordReader(OrcInputFormat.java:2046) > at > org.apache.hadoop.hive.ql.io.HiveInputFormat.getRecordReader(HiveInputFormat.java:419) > ... 28 more > {code} -- This message was sent by Atlassian Jira (v8.3.4#803005) --------------------------------------------------------------------- To unsubscribe, e-mail: ozone-issues-unsubscr...@hadoop.apache.org For additional commands, e-mail: ozone-issues-h...@hadoop.apache.org