You are viewing a plain text version of this content. The canonical link for it is here.
Posted to issues@drill.apache.org by "Khurram Faraaz (JIRA)" <ji...@apache.org> on 2015/05/13 20:56:02 UTC

[jira] [Commented] (DRILL-3043) Select over corrupt parquet file must return a better message

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

Khurram Faraaz commented on DRILL-3043:
---------------------------------------

Another case where querying a corrupt parquet file, leads to seeking beyond EOF.

{code}
0: jdbc:drill:schema=dfs.tmp> select key1, key2 from `corpt_Prq_02.parquet`;
Error: SYSTEM ERROR: java.io.EOFException: Seeking beyond EOF, file: /tmp/corpt_Prq_02.parquet, file length: 20480000, seeking to: 207814580

Fragment 0:0

[Error Id: 8fb96b03-264e-4ff2-8e9d-8885a2a32c96 on centos-01.qa.lab:31010] (state=,code=0)
{code}

Size in bytes of original parquet file 0_0_0.parquet and corrupt parquet file corpt_Prq_02.parquet

{code}
[root@centos-01 mal_formed_parquet]# head -c 10240000 0_0_0.parquet >> corpt_Prq_02.parquet
[root@centos-01 mal_formed_parquet]# tail -c 10240000 0_0_0.parquet >> corpt_Prq_02.parquet
{code}

{code}
-rw-r--r-- 1 root root 220980473 May 13 18:39 0_0_0.parquet
-rw-r--r-- 1 root root  20480000 May 13 18:41 corpt_Prq_02.parquet
{code}

Stack trace from drillbit.log

{code}
2015-05-13 18:44:27,652 [2aac6473-96a1-bb8f-6a72-b7b174caad01:frag:0:0] ERROR o.a.d.c.e.DrillRuntimeException - SYSTEM ERROR: java.io.EOFException: Seeking beyond EOF, file: /tmp/corpt_Prq_02.parquet, file length: 20480000, seeking to: 207814580

Fragment 0:0

[Error Id: 8fb96b03-264e-4ff2-8e9d-8885a2a32c96 on centos-01.qa.lab:31010]
org.apache.drill.common.exceptions.UserException: SYSTEM ERROR: java.io.EOFException: Seeking beyond EOF, file: /tmp/corpt_Prq_02.parquet, file length: 20480000, seeking to: 207814580

Fragment 0:0

[Error Id: 8fb96b03-264e-4ff2-8e9d-8885a2a32c96 on centos-01.qa.lab:31010]
        at org.apache.drill.common.exceptions.UserException$Builder.build(UserException.java:495) ~[drill-common-1.0.0-SNAPSHOT-rebuffed.jar:1.0.0-SNAPSHOT]
        at org.apache.drill.exec.work.fragment.FragmentExecutor.sendFinalState(FragmentExecutor.java:286) [drill-java-exec-1.0.0-SNAPSHOT-rebuffed.jar:1.0.0-SNAPSHOT]
        at org.apache.drill.exec.work.fragment.FragmentExecutor.run(FragmentExecutor.java:254) [drill-java-exec-1.0.0-SNAPSHOT-rebuffed.jar:1.0.0-SNAPSHOT]
        at org.apache.drill.common.SelfCleaningRunnable.run(SelfCleaningRunnable.java:38) [drill-common-1.0.0-SNAPSHOT-rebuffed.jar:1.0.0-SNAPSHOT]
        at java.util.concurrent.ThreadPoolExecutor.runWorker(ThreadPoolExecutor.java:1145) [na:1.7.0_45]
        at java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:615) [na:1.7.0_45]
        at java.lang.Thread.run(Thread.java:744) [na:1.7.0_45]
Caused by: org.apache.drill.common.exceptions.DrillRuntimeException: Error in parquet record reader.
Message: Failure in setting up reader
Parquet Metadata: ParquetMetaData{FileMetaData{schema: message root {
  optional double key1;
  optional binary key2 (UTF8);
}
, metadata: {}}, blocks: [BlockMetaData{26212355, 222873868 [ColumnMetaData{SNAPPY [key1] DOUBLE  [RLE, BIT_PACKED, PLAIN], 4}, ColumnMetaData{SNAPPY [key2] BINARY  [PLAIN_DICTIONARY, RLE, BIT_PACKED], 207814580}]}]}
        at org.apache.drill.exec.store.parquet.columnreaders.ParquetRecordReader.handleAndRaise(ParquetRecordReader.java:339) ~[drill-java-exec-1.0.0-SNAPSHOT-rebuffed.jar:1.0.0-SNAPSHOT]
        at org.apache.drill.exec.store.parquet.columnreaders.ParquetRecordReader.setup(ParquetRecordReader.java:332) ~[drill-java-exec-1.0.0-SNAPSHOT-rebuffed.jar:1.0.0-SNAPSHOT]
        at org.apache.drill.exec.physical.impl.ScanBatch.<init>(ScanBatch.java:99) ~[drill-java-exec-1.0.0-SNAPSHOT-rebuffed.jar:1.0.0-SNAPSHOT]
        at org.apache.drill.exec.store.parquet.ParquetScanBatchCreator.getBatch(ParquetScanBatchCreator.java:165) ~[drill-java-exec-1.0.0-SNAPSHOT-rebuffed.jar:1.0.0-SNAPSHOT]
        at org.apache.drill.exec.store.parquet.ParquetScanBatchCreator.getBatch(ParquetScanBatchCreator.java:55) ~[drill-java-exec-1.0.0-SNAPSHOT-rebuffed.jar:1.0.0-SNAPSHOT]
        at org.apache.drill.exec.physical.impl.ImplCreator.getRecordBatch(ImplCreator.java:140) ~[drill-java-exec-1.0.0-SNAPSHOT-rebuffed.jar:1.0.0-SNAPSHOT]
        at org.apache.drill.exec.physical.impl.ImplCreator.getChildren(ImplCreator.java:163) ~[drill-java-exec-1.0.0-SNAPSHOT-rebuffed.jar:1.0.0-SNAPSHOT]
        at org.apache.drill.exec.physical.impl.ImplCreator.getRecordBatch(ImplCreator.java:121) ~[drill-java-exec-1.0.0-SNAPSHOT-rebuffed.jar:1.0.0-SNAPSHOT]
        at org.apache.drill.exec.physical.impl.ImplCreator.getChildren(ImplCreator.java:163) ~[drill-java-exec-1.0.0-SNAPSHOT-rebuffed.jar:1.0.0-SNAPSHOT]
        at org.apache.drill.exec.physical.impl.ImplCreator.getRecordBatch(ImplCreator.java:121) ~[drill-java-exec-1.0.0-SNAPSHOT-rebuffed.jar:1.0.0-SNAPSHOT]
        at org.apache.drill.exec.physical.impl.ImplCreator.getChildren(ImplCreator.java:163) ~[drill-java-exec-1.0.0-SNAPSHOT-rebuffed.jar:1.0.0-SNAPSHOT]
        at org.apache.drill.exec.physical.impl.ImplCreator.getRecordBatch(ImplCreator.java:121) ~[drill-java-exec-1.0.0-SNAPSHOT-rebuffed.jar:1.0.0-SNAPSHOT]
        at org.apache.drill.exec.physical.impl.ImplCreator.getChildren(ImplCreator.java:163) ~[drill-java-exec-1.0.0-SNAPSHOT-rebuffed.jar:1.0.0-SNAPSHOT]
        at org.apache.drill.exec.physical.impl.ImplCreator.getRootExec(ImplCreator.java:96) ~[drill-java-exec-1.0.0-SNAPSHOT-rebuffed.jar:1.0.0-SNAPSHOT]
        at org.apache.drill.exec.physical.impl.ImplCreator.getExec(ImplCreator.java:77) ~[drill-java-exec-1.0.0-SNAPSHOT-rebuffed.jar:1.0.0-SNAPSHOT]
        at org.apache.drill.exec.work.fragment.FragmentExecutor.run(FragmentExecutor.java:199) [drill-java-exec-1.0.0-SNAPSHOT-rebuffed.jar:1.0.0-SNAPSHOT]
        ... 4 common frames omitted
Caused by: org.apache.drill.common.exceptions.ExecutionSetupException: Error opening or reading metadata for parquet file at location: corpt_Prq_02.parquet
        at org.apache.drill.exec.store.parquet.columnreaders.PageReader.<init>(PageReader.java:120) ~[drill-java-exec-1.0.0-SNAPSHOT-rebuffed.jar:1.0.0-SNAPSHOT]
        at org.apache.drill.exec.store.parquet.columnreaders.ColumnReader.<init>(ColumnReader.java:87) ~[drill-java-exec-1.0.0-SNAPSHOT-rebuffed.jar:1.0.0-SNAPSHOT]
        at org.apache.drill.exec.store.parquet.columnreaders.VarLengthColumn.<init>(VarLengthColumn.java:39) ~[drill-java-exec-1.0.0-SNAPSHOT-rebuffed.jar:1.0.0-SNAPSHOT]
        at org.apache.drill.exec.store.parquet.columnreaders.VarLengthValuesColumn.<init>(VarLengthValuesColumn.java:43) ~[drill-java-exec-1.0.0-SNAPSHOT-rebuffed.jar:1.0.0-SNAPSHOT]
        at org.apache.drill.exec.store.parquet.columnreaders.NullableVarLengthValuesColumn.<init>(NullableVarLengthValuesColumn.java:39) ~[drill-java-exec-1.0.0-SNAPSHOT-rebuffed.jar:1.0.0-SNAPSHOT]
        at org.apache.drill.exec.store.parquet.columnreaders.VarLengthColumnReaders$NullableVarCharColumn.<init>(VarLengthColumnReaders.java:213) ~[drill-java-exec-1.0.0-SNAPSHOT-rebuffed.jar:1.0.0-SNAPSHOT]
        at org.apache.drill.exec.store.parquet.columnreaders.ColumnReaderFactory.getReader(ColumnReaderFactory.java:181) ~[drill-java-exec-1.0.0-SNAPSHOT-rebuffed.jar:1.0.0-SNAPSHOT]
        at org.apache.drill.exec.store.parquet.columnreaders.ParquetRecordReader.setup(ParquetRecordReader.java:312) ~[drill-java-exec-1.0.0-SNAPSHOT-rebuffed.jar:1.0.0-SNAPSHOT]
        ... 18 common frames omitted
Caused by: java.io.EOFException: Seeking beyond EOF, file: /tmp/corpt_Prq_02.parquet, file length: 20480000, seeking to: 207814580
        at com.mapr.fs.MapRFsInStream.seek(MapRFsInStream.java:306) ~[maprfs-4.0.1.28318-mapr.jar:4.0.1.28318-mapr]
        at com.mapr.fs.MapRFsDataInputStream.seek(MapRFsDataInputStream.java:31) ~[maprfs-4.0.1.28318-mapr.jar:4.0.1.28318-mapr]
        at org.apache.drill.exec.store.dfs.DrillFSDataInputStream.seek(DrillFSDataInputStream.java:57) ~[drill-java-exec-1.0.0-SNAPSHOT-rebuffed.jar:1.0.0-SNAPSHOT]
        at org.apache.drill.exec.store.parquet.ColumnDataReader.<init>(ColumnDataReader.java:41) ~[drill-java-exec-1.0.0-SNAPSHOT-rebuffed.jar:1.0.0-SNAPSHOT]
        at org.apache.drill.exec.store.parquet.columnreaders.PageReader.<init>(PageReader.java:116) ~[drill-java-exec-1.0.0-SNAPSHOT-rebuffed.jar:1.0.0-SNAPSHOT]
        ... 25 common frames omitted
2015-05-13 18:44:27,658 [2aac6473-96a1-bb8f-6a72-b7b174caad01:frag:0:0] INFO  o.a.drill.exec.work.foreman.Foreman - State change requested.  RUNNING --> FAILED
org.apache.drill.common.exceptions.UserRemoteException: SYSTEM ERROR: java.io.EOFException: Seeking beyond EOF, file: /tmp/corpt_Prq_02.parquet, file length: 20480000, seeking to: 207814580

Fragment 0:0

[Error Id: 8fb96b03-264e-4ff2-8e9d-8885a2a32c96 on centos-01.qa.lab:31010]
        at org.apache.drill.exec.work.foreman.QueryManager$1.statusUpdate(QueryManager.java:460) [drill-java-exec-1.0.0-SNAPSHOT-rebuffed.jar:1.0.0-SNAPSHOT]
        at org.apache.drill.exec.work.foreman.QueryManager$RootStatusReporter.statusChange(QueryManager.java:440) [drill-java-exec-1.0.0-SNAPSHOT-rebuffed.jar:1.0.0-SNAPSHOT]
        at org.apache.drill.exec.work.fragment.AbstractStatusReporter.fail(AbstractStatusReporter.java:90) [drill-java-exec-1.0.0-SNAPSHOT-rebuffed.jar:1.0.0-SNAPSHOT]
        at org.apache.drill.exec.work.fragment.AbstractStatusReporter.fail(AbstractStatusReporter.java:86) [drill-java-exec-1.0.0-SNAPSHOT-rebuffed.jar:1.0.0-SNAPSHOT]
        at org.apache.drill.exec.work.fragment.FragmentExecutor.sendFinalState(FragmentExecutor.java:290) [drill-java-exec-1.0.0-SNAPSHOT-rebuffed.jar:1.0.0-SNAPSHOT]
        at org.apache.drill.exec.work.fragment.FragmentExecutor.run(FragmentExecutor.java:254) [drill-java-exec-1.0.0-SNAPSHOT-rebuffed.jar:1.0.0-SNAPSHOT]
        at org.apache.drill.common.SelfCleaningRunnable.run(SelfCleaningRunnable.java:38) [drill-common-1.0.0-SNAPSHOT-rebuffed.jar:1.0.0-SNAPSHOT]
        at java.util.concurrent.ThreadPoolExecutor.runWorker(ThreadPoolExecutor.java:1145) [na:1.7.0_45]
        at java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:615) [na:1.7.0_45]
        at java.lang.Thread.run(Thread.java:744) [na:1.7.0_45]
2015-05-13 18:44:27,666 [2aac6473-96a1-bb8f-6a72-b7b174caad01:frag:0:0] WARN  o.a.d.exec.memory.AtomicRemainder - Tried to close remainder, but it has already been closed
java.lang.Exception: null
        at org.apache.drill.exec.memory.AtomicRemainder.close(AtomicRemainder.java:196) [drill-java-exec-1.0.0-SNAPSHOT-rebuffed.jar:1.0.0-SNAPSHOT]
        at org.apache.drill.exec.memory.Accountor.close(Accountor.java:386) [drill-java-exec-1.0.0-SNAPSHOT-rebuffed.jar:1.0.0-SNAPSHOT]
        at org.apache.drill.exec.memory.TopLevelAllocator$ChildAllocator.close(TopLevelAllocator.java:310) [drill-java-exec-1.0.0-SNAPSHOT-rebuffed.jar:1.0.0-SNAPSHOT]
        at org.apache.drill.exec.ops.FragmentContext.suppressingClose(FragmentContext.java:397) [drill-java-exec-1.0.0-SNAPSHOT-rebuffed.jar:1.0.0-SNAPSHOT]
        at org.apache.drill.exec.ops.FragmentContext.close(FragmentContext.java:391) [drill-java-exec-1.0.0-SNAPSHOT-rebuffed.jar:1.0.0-SNAPSHOT]
        at org.apache.drill.exec.work.fragment.FragmentExecutor.closeOutResources(FragmentExecutor.java:311) [drill-java-exec-1.0.0-SNAPSHOT-rebuffed.jar:1.0.0-SNAPSHOT]
        at org.apache.drill.exec.work.fragment.FragmentExecutor.cancel(FragmentExecutor.java:135) [drill-java-exec-1.0.0-SNAPSHOT-rebuffed.jar:1.0.0-SNAPSHOT]
        at org.apache.drill.exec.work.foreman.QueryManager.cancelExecutingFragments(QueryManager.java:187) [drill-java-exec-1.0.0-SNAPSHOT-rebuffed.jar:1.0.0-SNAPSHOT]
        at org.apache.drill.exec.work.foreman.Foreman$StateSwitch.processEvent(Foreman.java:824) [drill-java-exec-1.0.0-SNAPSHOT-rebuffed.jar:1.0.0-SNAPSHOT]
        at org.apache.drill.exec.work.foreman.Foreman$StateSwitch.processEvent(Foreman.java:768) [drill-java-exec-1.0.0-SNAPSHOT-rebuffed.jar:1.0.0-SNAPSHOT]
        at org.apache.drill.common.EventProcessor.sendEvent(EventProcessor.java:73) [drill-common-1.0.0-SNAPSHOT-rebuffed.jar:1.0.0-SNAPSHOT]
        at org.apache.drill.exec.work.foreman.Foreman$StateSwitch.moveToState(Foreman.java:770) [drill-java-exec-1.0.0-SNAPSHOT-rebuffed.jar:1.0.0-SNAPSHOT]
        at org.apache.drill.exec.work.foreman.Foreman.moveToState(Foreman.java:871) [drill-java-exec-1.0.0-SNAPSHOT-rebuffed.jar:1.0.0-SNAPSHOT]
        at org.apache.drill.exec.work.foreman.Foreman.access$2700(Foreman.java:107) [drill-java-exec-1.0.0-SNAPSHOT-rebuffed.jar:1.0.0-SNAPSHOT]
        at org.apache.drill.exec.work.foreman.Foreman$StateListener.moveToState(Foreman.java:1132) [drill-java-exec-1.0.0-SNAPSHOT-rebuffed.jar:1.0.0-SNAPSHOT]
        at org.apache.drill.exec.work.foreman.QueryManager$1.statusUpdate(QueryManager.java:460) [drill-java-exec-1.0.0-SNAPSHOT-rebuffed.jar:1.0.0-SNAPSHOT]
        at org.apache.drill.exec.work.foreman.QueryManager$RootStatusReporter.statusChange(QueryManager.java:440) [drill-java-exec-1.0.0-SNAPSHOT-rebuffed.jar:1.0.0-SNAPSHOT]
        at org.apache.drill.exec.work.fragment.AbstractStatusReporter.fail(AbstractStatusReporter.java:90) [drill-java-exec-1.0.0-SNAPSHOT-rebuffed.jar:1.0.0-SNAPSHOT]
        at org.apache.drill.exec.work.fragment.AbstractStatusReporter.fail(AbstractStatusReporter.java:86) [drill-java-exec-1.0.0-SNAPSHOT-rebuffed.jar:1.0.0-SNAPSHOT]
        at org.apache.drill.exec.work.fragment.FragmentExecutor.sendFinalState(FragmentExecutor.java:290) [drill-java-exec-1.0.0-SNAPSHOT-rebuffed.jar:1.0.0-SNAPSHOT]
        at org.apache.drill.exec.work.fragment.FragmentExecutor.run(FragmentExecutor.java:254) [drill-java-exec-1.0.0-SNAPSHOT-rebuffed.jar:1.0.0-SNAPSHOT]
        at org.apache.drill.common.SelfCleaningRunnable.run(SelfCleaningRunnable.java:38) [drill-common-1.0.0-SNAPSHOT-rebuffed.jar:1.0.0-SNAPSHOT]
        at java.util.concurrent.ThreadPoolExecutor.runWorker(ThreadPoolExecutor.java:1145) [na:1.7.0_45]
        at java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:615) [na:1.7.0_45]
        at java.lang.Thread.run(Thread.java:744) [na:1.7.0_45]
2015-05-13 18:44:27,666 [2aac6473-96a1-bb8f-6a72-b7b174caad01:frag:0:0] INFO  o.a.d.e.w.fragment.FragmentExecutor - 2aac6473-96a1-bb8f-6a72-b7b174caad01:0:0: State change requested from FAILED --> CANCELLATION_REQUESTED for
2015-05-13 18:44:27,666 [2aac6473-96a1-bb8f-6a72-b7b174caad01:frag:0:0] WARN  o.a.d.e.w.fragment.FragmentExecutor - Ignoring unexpected state transition FAILED => CANCELLATION_REQUESTED.
2015-05-13 18:44:27,666 [2aac6473-96a1-bb8f-6a72-b7b174caad01:frag:0:0] INFO  o.a.drill.exec.work.foreman.Foreman - foreman cleaning up.
2015-05-13 18:44:27,679 [2aac6473-96a1-bb8f-6a72-b7b174caad01:frag:0:0] ERROR o.a.d.exec.work.foreman.QueryManager - Failure while storing Query Profile
java.lang.RuntimeException: java.io.IOException: java.lang.InterruptedException
        at org.apache.drill.exec.store.sys.local.FilePStore.put(FilePStore.java:148) ~[drill-java-exec-1.0.0-SNAPSHOT-rebuffed.jar:1.0.0-SNAPSHOT]
        at org.apache.drill.exec.work.foreman.QueryManager.writeFinalProfile(QueryManager.java:286) ~[drill-java-exec-1.0.0-SNAPSHOT-rebuffed.jar:1.0.0-SNAPSHOT]
        at org.apache.drill.exec.work.foreman.Foreman$ForemanResult.close(Foreman.java:731) [drill-java-exec-1.0.0-SNAPSHOT-rebuffed.jar:1.0.0-SNAPSHOT]
        at org.apache.drill.exec.work.foreman.Foreman$StateSwitch.processEvent(Foreman.java:826) [drill-java-exec-1.0.0-SNAPSHOT-rebuffed.jar:1.0.0-SNAPSHOT]
        at org.apache.drill.exec.work.foreman.Foreman$StateSwitch.processEvent(Foreman.java:768) [drill-java-exec-1.0.0-SNAPSHOT-rebuffed.jar:1.0.0-SNAPSHOT]
        at org.apache.drill.common.EventProcessor.sendEvent(EventProcessor.java:73) [drill-common-1.0.0-SNAPSHOT-rebuffed.jar:1.0.0-SNAPSHOT]
        at org.apache.drill.exec.work.foreman.Foreman$StateSwitch.moveToState(Foreman.java:770) [drill-java-exec-1.0.0-SNAPSHOT-rebuffed.jar:1.0.0-SNAPSHOT]
        at org.apache.drill.exec.work.foreman.Foreman.moveToState(Foreman.java:871) [drill-java-exec-1.0.0-SNAPSHOT-rebuffed.jar:1.0.0-SNAPSHOT]
        at org.apache.drill.exec.work.foreman.Foreman.access$2700(Foreman.java:107) [drill-java-exec-1.0.0-SNAPSHOT-rebuffed.jar:1.0.0-SNAPSHOT]
        at org.apache.drill.exec.work.foreman.Foreman$StateListener.moveToState(Foreman.java:1132) [drill-java-exec-1.0.0-SNAPSHOT-rebuffed.jar:1.0.0-SNAPSHOT]
        at org.apache.drill.exec.work.foreman.QueryManager$1.statusUpdate(QueryManager.java:460) [drill-java-exec-1.0.0-SNAPSHOT-rebuffed.jar:1.0.0-SNAPSHOT]
        at org.apache.drill.exec.work.foreman.QueryManager$RootStatusReporter.statusChange(QueryManager.java:440) [drill-java-exec-1.0.0-SNAPSHOT-rebuffed.jar:1.0.0-SNAPSHOT]
        at org.apache.drill.exec.work.fragment.AbstractStatusReporter.fail(AbstractStatusReporter.java:90) [drill-java-exec-1.0.0-SNAPSHOT-rebuffed.jar:1.0.0-SNAPSHOT]
        at org.apache.drill.exec.work.fragment.AbstractStatusReporter.fail(AbstractStatusReporter.java:86) [drill-java-exec-1.0.0-SNAPSHOT-rebuffed.jar:1.0.0-SNAPSHOT]
        at org.apache.drill.exec.work.fragment.FragmentExecutor.sendFinalState(FragmentExecutor.java:290) [drill-java-exec-1.0.0-SNAPSHOT-rebuffed.jar:1.0.0-SNAPSHOT]
        at org.apache.drill.exec.work.fragment.FragmentExecutor.run(FragmentExecutor.java:254) [drill-java-exec-1.0.0-SNAPSHOT-rebuffed.jar:1.0.0-SNAPSHOT]
        at org.apache.drill.common.SelfCleaningRunnable.run(SelfCleaningRunnable.java:38) [drill-common-1.0.0-SNAPSHOT-rebuffed.jar:1.0.0-SNAPSHOT]
        at java.util.concurrent.ThreadPoolExecutor.runWorker(ThreadPoolExecutor.java:1145) [na:1.7.0_45]
        at java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:615) [na:1.7.0_45]
        at java.lang.Thread.run(Thread.java:744) [na:1.7.0_45]

Caused by: java.io.IOException: java.lang.InterruptedException
        at org.apache.hadoop.util.Shell.runCommand(Shell.java:508) ~[hadoop-common-2.4.1-mapr-1408.jar:na]
        at org.apache.hadoop.util.Shell.run(Shell.java:418) ~[hadoop-common-2.4.1-mapr-1408.jar:na]
        at org.apache.hadoop.util.Shell$ShellCommandExecutor.execute(Shell.java:650) ~[hadoop-common-2.4.1-mapr-1408.jar:na]
        at org.apache.hadoop.util.Shell.execCommand(Shell.java:739) ~[hadoop-common-2.4.1-mapr-1408.jar:na]
        at org.apache.hadoop.util.Shell.execCommand(Shell.java:722) ~[hadoop-common-2.4.1-mapr-1408.jar:na]
        at org.apache.hadoop.fs.RawLocalFileSystem.setPermission(RawLocalFileSystem.java:676) ~[hadoop-common-2.4.1-mapr-1408.jar:na]
        at org.apache.hadoop.fs.FilterFileSystem.setPermission(FilterFileSystem.java:467) ~[hadoop-common-2.4.1-mapr-1408.jar:na]
        at org.apache.hadoop.fs.ChecksumFileSystem.create(ChecksumFileSystem.java:456) ~[hadoop-common-2.4.1-mapr-1408.jar:na]
        at org.apache.hadoop.fs.ChecksumFileSystem.create(ChecksumFileSystem.java:424) ~[hadoop-common-2.4.1-mapr-1408.jar:na]
        at org.apache.hadoop.fs.FileSystem.create(FileSystem.java:942) ~[hadoop-common-2.4.1-mapr-1408.jar:na]
        at org.apache.hadoop.fs.FileSystem.create(FileSystem.java:923) ~[hadoop-common-2.4.1-mapr-1408.jar:na]
        at org.apache.hadoop.fs.FileSystem.create(FileSystem.java:820) ~[hadoop-common-2.4.1-mapr-1408.jar:na]
        at org.apache.hadoop.fs.FileSystem.create(FileSystem.java:809) ~[hadoop-common-2.4.1-mapr-1408.jar:na]
        at org.apache.drill.exec.store.dfs.DrillFileSystem.create(DrillFileSystem.java:175) ~[drill-java-exec-1.0.0-SNAPSHOT-rebuffed.jar:1.0.0-SNAPSHOT]
        at org.apache.drill.exec.store.sys.local.FilePStore.put(FilePStore.java:145) ~[drill-java-exec-1.0.0-SNAPSHOT-rebuffed.jar:1.0.0-SNAPSHOT]
        ... 19 common frames omitted

{code}

> Select over corrupt parquet file must return a better message
> -------------------------------------------------------------
>
>                 Key: DRILL-3043
>                 URL: https://issues.apache.org/jira/browse/DRILL-3043
>             Project: Apache Drill
>          Issue Type: Bug
>          Components: Storage - Parquet
>    Affects Versions: 1.0.0
>         Environment: 21cc578b6b8c8f3ca1ebffd3dbb92e35d68bc726 | MD-272: MapRDB plugin needs another update on 2015-05-08 
>            Reporter: Khurram Faraaz
>            Assignee: Steven Phillips
>            Priority: Minor
>
> Select over a corrupt parquet file does not return proper error message to user. Looks like the correct message is getting lost.
> The below CTAS was used to create the original parquet file.
> {code}
> 0: jdbc:drill:schema=dfs.tmp> create table tblJsnToPrq_01 as select key from `oneMlnDblData.json`;
> +------------+---------------------------+
> |  Fragment  | Number of records written |
> +------------+---------------------------+
> | 0_0        | 999999                    |
> +------------+---------------------------+
> 1 row selected (2.381 seconds)
> 0: jdbc:drill:schema=dfs.tmp> select count(key) from  tblJsnToPrq_01;
> +------------+
> |   EXPR$0   |
> +------------+
> | 999999     |
> +------------+
> 1 row selected (0.232 seconds)
> {code}
> {code}
> 0: jdbc:drill:schema=dfs.tmp> select key from `corrupt_Parquet.parquet`;
> Error: SYSTEM ERROR: java.io.IOException: FAILED_TO_UNCOMPRESS(5)
> Fragment 0:0
> [Error Id: 48072728-574f-450f-8370-32dbbb534da6 on centos-01.qa.lab:31010] (state=,code=0)
> {code}
> The way the corrupt parquet file was created from the original parquet file. The first and last 1000000 bytes were written to the new (corrupt) parquet file
> {code}
> [root@centos-01 mal_formed_parquet]# head -c 1000000 0_0_0.parquet >> corrupt_Parquet.parquet
> [root@centos-01 mal_formed_parquet]# tail -c 1000000 0_0_0.parquet >> corrupt_Parquet.parquet
> {code}
> Stack trace from drillbit.log
> {code}
> 2015-05-12 23:05:45,285 [2aad78b6-09c6-0e17-accf-ab279fab39dc:frag:0:0] ERROR o.a.d.c.e.DrillRuntimeException - SYSTEM ERROR: java.io.IOException: FAILED_TO_UNCOMPRESS(5)
> Fragment 0:0
> [Error Id: 48072728-574f-450f-8370-32dbbb534da6 on centos-01.qa.lab:31010]
> org.apache.drill.common.exceptions.UserException: SYSTEM ERROR: java.io.IOException: FAILED_TO_UNCOMPRESS(5)
> Fragment 0:0
> [Error Id: 48072728-574f-450f-8370-32dbbb534da6 on centos-01.qa.lab:31010]
>         at org.apache.drill.common.exceptions.UserException$Builder.build(UserException.java:495) ~[drill-common-1.0.0-SNAPSHOT-rebuffed.jar:1.0.0-SNAPSHOT]
>         at org.apache.drill.exec.work.fragment.FragmentExecutor.sendFinalState(FragmentExecutor.java:286) [drill-java-exec-1.0.0-SNAPSHOT-rebuffed.jar:1.0.0-SNAPSHOT]
>         at org.apache.drill.exec.work.fragment.FragmentExecutor.run(FragmentExecutor.java:254) [drill-java-exec-1.0.0-SNAPSHOT-rebuffed.jar:1.0.0-SNAPSHOT]
>         at org.apache.drill.common.SelfCleaningRunnable.run(SelfCleaningRunnable.java:38) [drill-common-1.0.0-SNAPSHOT-rebuffed.jar:1.0.0-SNAPSHOT]
>         at java.util.concurrent.ThreadPoolExecutor.runWorker(ThreadPoolExecutor.java:1145) [na:1.7.0_45]
>         at java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:615) [na:1.7.0_45]
>         at java.lang.Thread.run(Thread.java:744) [na:1.7.0_45]
> Caused by: org.apache.drill.common.exceptions.DrillRuntimeException: Error in parquet record reader.
> Message:
> Hadoop path: /tmp/corrupt_Parquet.parquet
> Total records read: 0
> Mock records read: 0
> Records to read: 32768
> Row group index: 0
> Records in row group: 999999
> Parquet Metadata: ParquetMetaData{FileMetaData{schema: message root {
>   optional double key;
> }
> , metadata: {}}, blocks: [BlockMetaData{999999, 8000439 [ColumnMetaData{SNAPPY [key] DOUBLE  [PLAIN, BIT_PACKED, RLE], 4}]}]}
>         at org.apache.drill.exec.store.parquet.columnreaders.ParquetRecordReader.handleAndRaise(ParquetRecordReader.java:339) ~[drill-java-exec-1.0.0-SNAPSHOT-rebuffed.jar:1.0.0-SNAPSHOT]
>         at org.apache.drill.exec.store.parquet.columnreaders.ParquetRecordReader.next(ParquetRecordReader.java:441) ~[drill-java-exec-1.0.0-SNAPSHOT-rebuffed.jar:1.0.0-SNAPSHOT]
>         at org.apache.drill.exec.physical.impl.ScanBatch.next(ScanBatch.java:175) ~[drill-java-exec-1.0.0-SNAPSHOT-rebuffed.jar:1.0.0-SNAPSHOT]
>         at org.apache.drill.exec.physical.impl.validate.IteratorValidatorBatchIterator.next(IteratorValidatorBatchIterator.java:118) ~[drill-java-exec-1.0.0-SNAPSHOT-rebuffed.jar:1.0.0-SNAPSHOT]
>         at org.apache.drill.exec.physical.impl.BaseRootExec.next(BaseRootExec.java:83) ~[drill-java-exec-1.0.0-SNAPSHOT-rebuffed.jar:1.0.0-SNAPSHOT]
>         at org.apache.drill.exec.physical.impl.ScreenCreator$ScreenRoot.innerNext(ScreenCreator.java:80) ~[drill-java-exec-1.0.0-SNAPSHOT-rebuffed.jar:1.0.0-SNAPSHOT]
>         at org.apache.drill.exec.physical.impl.BaseRootExec.next(BaseRootExec.java:73) ~[drill-java-exec-1.0.0-SNAPSHOT-rebuffed.jar:1.0.0-SNAPSHOT]
>         at org.apache.drill.exec.work.fragment.FragmentExecutor$1.run(FragmentExecutor.java:223) ~[drill-java-exec-1.0.0-SNAPSHOT-rebuffed.jar:1.0.0-SNAPSHOT]
>         at org.apache.drill.exec.work.fragment.FragmentExecutor$1.run(FragmentExecutor.java:217) ~[drill-java-exec-1.0.0-SNAPSHOT-rebuffed.jar:1.0.0-SNAPSHOT]
>         at java.security.AccessController.doPrivileged(Native Method) ~[na:1.7.0_45]
>         at javax.security.auth.Subject.doAs(Subject.java:415) ~[na:1.7.0_45]
>         at org.apache.hadoop.security.UserGroupInformation.doAs(UserGroupInformation.java:1469) ~[hadoop-common-2.4.1-mapr-1408.jar:na]
>         at org.apache.drill.exec.work.fragment.FragmentExecutor.run(FragmentExecutor.java:217) [drill-java-exec-1.0.0-SNAPSHOT-rebuffed.jar:1.0.0-SNAPSHOT]
>         ... 4 common frames omitted
> Caused by: java.io.IOException: FAILED_TO_UNCOMPRESS(5)
>         at org.xerial.snappy.SnappyNative.throw_error(SnappyNative.java:78) ~[na:na]
>         at org.xerial.snappy.SnappyNative.rawUncompress(Native Method) ~[na:na]
>         at org.xerial.snappy.Snappy.uncompress(Snappy.java:467) ~[snappy-java-1.0.5-M3.jar:na]
>         at parquet.hadoop.codec.SnappyDecompressor$SnappyDirectDecompressor.decompress(SnappyDecompressor.java:163) ~[parquet-hadoop-1.6.0rc3-drill-r0.3.jar:1.6.0rc3-drill-r0.3]
>         at org.apache.drill.exec.store.parquet.DirectCodecFactory$FullDirectDecompressor.decompress(DirectCodecFactory.java:239) ~[drill-java-exec-1.0.0-SNAPSHOT-rebuffed.jar:1.0.0-SNAPSHOT]
>         at org.apache.drill.exec.store.parquet.columnreaders.PageReader.next(PageReader.java:230) ~[drill-java-exec-1.0.0-SNAPSHOT-rebuffed.jar:1.0.0-SNAPSHOT]
>         at org.apache.drill.exec.store.parquet.columnreaders.NullableColumnReader.processPages(NullableColumnReader.java:76) ~[drill-java-exec-1.0.0-SNAPSHOT-rebuffed.jar:1.0.0-SNAPSHOT]
>         at org.apache.drill.exec.store.parquet.columnreaders.ParquetRecordReader.readAllFixedFields(ParquetRecordReader.java:380) ~[drill-java-exec-1.0.0-SNAPSHOT-rebuffed.jar:1.0.0-SNAPSHOT]
>         at org.apache.drill.exec.store.parquet.columnreaders.ParquetRecordReader.next(ParquetRecordReader.java:423) ~[drill-java-exec-1.0.0-SNAPSHOT-rebuffed.jar:1.0.0-SNAPSHOT]
>         ... 15 common frames omitted
> 2015-05-12 23:05:45,287 [2aad78b6-09c6-0e17-accf-ab279fab39dc:frag:0:0] INFO  o.a.drill.exec.work.foreman.Foreman - State change requested.  RUNNING --> FAILED
> org.apache.drill.common.exceptions.UserRemoteException: SYSTEM ERROR: java.io.IOException: FAILED_TO_UNCOMPRESS(5)
> Fragment 0:0
> [Error Id: 48072728-574f-450f-8370-32dbbb534da6 on centos-01.qa.lab:31010]
>         at org.apache.drill.exec.work.foreman.QueryManager$1.statusUpdate(QueryManager.java:460) [drill-java-exec-1.0.0-SNAPSHOT-rebuffed.jar:1.0.0-SNAPSHOT]
>         at org.apache.drill.exec.work.foreman.QueryManager$RootStatusReporter.statusChange(QueryManager.java:440) [drill-java-exec-1.0.0-SNAPSHOT-rebuffed.jar:1.0.0-SNAPSHOT]
>         at org.apache.drill.exec.work.fragment.AbstractStatusReporter.fail(AbstractStatusReporter.java:90) [drill-java-exec-1.0.0-SNAPSHOT-rebuffed.jar:1.0.0-SNAPSHOT]
>         at org.apache.drill.exec.work.fragment.AbstractStatusReporter.fail(AbstractStatusReporter.java:86) [drill-java-exec-1.0.0-SNAPSHOT-rebuffed.jar:1.0.0-SNAPSHOT]
>         at org.apache.drill.exec.work.fragment.FragmentExecutor.sendFinalState(FragmentExecutor.java:290) [drill-java-exec-1.0.0-SNAPSHOT-rebuffed.jar:1.0.0-SNAPSHOT]
>         at org.apache.drill.exec.work.fragment.FragmentExecutor.run(FragmentExecutor.java:254) [drill-java-exec-1.0.0-SNAPSHOT-rebuffed.jar:1.0.0-SNAPSHOT]
>         at org.apache.drill.common.SelfCleaningRunnable.run(SelfCleaningRunnable.java:38) [drill-common-1.0.0-SNAPSHOT-rebuffed.jar:1.0.0-SNAPSHOT]
>         at java.util.concurrent.ThreadPoolExecutor.runWorker(ThreadPoolExecutor.java:1145) [na:1.7.0_45]
>         at java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:615) [na:1.7.0_45]
>         at java.lang.Thread.run(Thread.java:744) [na:1.7.0_45]
> 2015-05-12 23:05:45,298 [2aad78b6-09c6-0e17-accf-ab279fab39dc:frag:0:0] INFO  o.a.d.e.w.fragment.FragmentExecutor - 2aad78b6-09c6-0e17-accf-ab279fab39dc:0:0: State change requested from FAILED --> CANCELLATION_REQUESTED for
> 2015-05-12 23:05:45,298 [2aad78b6-09c6-0e17-accf-ab279fab39dc:frag:0:0] WARN  o.a.d.e.w.fragment.FragmentExecutor - Ignoring unexpected state transition FAILED => CANCELLATION_REQUESTED.
> 2015-05-12 23:05:45,298 [2aad78b6-09c6-0e17-accf-ab279fab39dc:frag:0:0] INFO  o.a.drill.exec.work.foreman.Foreman - foreman cleaning up.
> 2015-05-12 23:05:45,305 [2aad78b6-09c6-0e17-accf-ab279fab39dc:frag:0:0] ERROR o.a.d.exec.work.foreman.QueryManager - Failure while storing Query Profile
> java.lang.RuntimeException: java.io.IOException: java.lang.InterruptedException
>         at org.apache.drill.exec.store.sys.local.FilePStore.put(FilePStore.java:148) ~[drill-java-exec-1.0.0-SNAPSHOT-rebuffed.jar:1.0.0-SNAPSHOT]
>         at org.apache.drill.exec.work.foreman.QueryManager.writeFinalProfile(QueryManager.java:286) ~[drill-java-exec-1.0.0-SNAPSHOT-rebuffed.jar:1.0.0-SNAPSHOT]
>         at org.apache.drill.exec.work.foreman.Foreman$ForemanResult.close(Foreman.java:731) [drill-java-exec-1.0.0-SNAPSHOT-rebuffed.jar:1.0.0-SNAPSHOT]
>         at org.apache.drill.exec.work.foreman.Foreman$StateSwitch.processEvent(Foreman.java:826) [drill-java-exec-1.0.0-SNAPSHOT-rebuffed.jar:1.0.0-SNAPSHOT]
>         at org.apache.drill.exec.work.foreman.Foreman$StateSwitch.processEvent(Foreman.java:768) [drill-java-exec-1.0.0-SNAPSHOT-rebuffed.jar:1.0.0-SNAPSHOT]
>         at org.apache.drill.common.EventProcessor.sendEvent(EventProcessor.java:73) [drill-common-1.0.0-SNAPSHOT-rebuffed.jar:1.0.0-SNAPSHOT]
>         at org.apache.drill.exec.work.foreman.Foreman$StateSwitch.moveToState(Foreman.java:770) [drill-java-exec-1.0.0-SNAPSHOT-rebuffed.jar:1.0.0-SNAPSHOT]
>         at org.apache.drill.exec.work.foreman.Foreman.moveToState(Foreman.java:871) [drill-java-exec-1.0.0-SNAPSHOT-rebuffed.jar:1.0.0-SNAPSHOT]
>         at org.apache.drill.exec.work.foreman.Foreman.access$2700(Foreman.java:107) [drill-java-exec-1.0.0-SNAPSHOT-rebuffed.jar:1.0.0-SNAPSHOT]
>         at org.apache.drill.exec.work.foreman.Foreman$StateListener.moveToState(Foreman.java:1132) [drill-java-exec-1.0.0-SNAPSHOT-rebuffed.jar:1.0.0-SNAPSHOT]
>         at org.apache.drill.exec.work.foreman.QueryManager$1.statusUpdate(QueryManager.java:460) [drill-java-exec-1.0.0-SNAPSHOT-rebuffed.jar:1.0.0-SNAPSHOT]
>         at org.apache.drill.exec.work.foreman.QueryManager$RootStatusReporter.statusChange(QueryManager.java:440) [drill-java-exec-1.0.0-SNAPSHOT-rebuffed.jar:1.0.0-SNAPSHOT]
>         at org.apache.drill.exec.work.fragment.AbstractStatusReporter.fail(AbstractStatusReporter.java:90) [drill-java-exec-1.0.0-SNAPSHOT-rebuffed.jar:1.0.0-SNAPSHOT]
>         at org.apache.drill.exec.work.fragment.AbstractStatusReporter.fail(AbstractStatusReporter.java:86) [drill-java-exec-1.0.0-SNAPSHOT-rebuffed.jar:1.0.0-SNAPSHOT]
>         at org.apache.drill.exec.work.fragment.FragmentExecutor.sendFinalState(FragmentExecutor.java:290) [drill-java-exec-1.0.0-SNAPSHOT-rebuffed.jar:1.0.0-SNAPSHOT]
>         at org.apache.drill.exec.work.fragment.FragmentExecutor.run(FragmentExecutor.java:254) [drill-java-exec-1.0.0-SNAPSHOT-rebuffed.jar:1.0.0-SNAPSHOT]
>         at org.apache.drill.common.SelfCleaningRunnable.run(SelfCleaningRunnable.java:38) [drill-common-1.0.0-SNAPSHOT-rebuffed.jar:1.0.0-SNAPSHOT]
>         at java.util.concurrent.ThreadPoolExecutor.runWorker(ThreadPoolExecutor.java:1145) [na:1.7.0_45]
>         at java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:615) [na:1.7.0_45]
>         at java.lang.Thread.run(Thread.java:744) [na:1.7.0_45]
> Caused by: java.io.IOException: java.lang.InterruptedException
>         at org.apache.hadoop.util.Shell.runCommand(Shell.java:508) ~[hadoop-common-2.4.1-mapr-1408.jar:na]
>         at org.apache.hadoop.util.Shell.run(Shell.java:418) ~[hadoop-common-2.4.1-mapr-1408.jar:na]
>         at org.apache.hadoop.util.Shell$ShellCommandExecutor.execute(Shell.java:650) ~[hadoop-common-2.4.1-mapr-1408.jar:na]
>         at org.apache.hadoop.util.Shell.execCommand(Shell.java:739) ~[hadoop-common-2.4.1-mapr-1408.jar:na]
>         at org.apache.hadoop.util.Shell.execCommand(Shell.java:722) ~[hadoop-common-2.4.1-mapr-1408.jar:na]
>         at org.apache.hadoop.fs.RawLocalFileSystem.setPermission(RawLocalFileSystem.java:676) ~[hadoop-common-2.4.1-mapr-1408.jar:na]
>         at org.apache.hadoop.fs.FilterFileSystem.setPermission(FilterFileSystem.java:467) ~[hadoop-common-2.4.1-mapr-1408.jar:na]
>         at org.apache.hadoop.fs.ChecksumFileSystem.create(ChecksumFileSystem.java:456) ~[hadoop-common-2.4.1-mapr-1408.jar:na]
>         at org.apache.hadoop.fs.ChecksumFileSystem.create(ChecksumFileSystem.java:424) ~[hadoop-common-2.4.1-mapr-1408.jar:na]
>         at org.apache.hadoop.fs.FileSystem.create(FileSystem.java:942) ~[hadoop-common-2.4.1-mapr-1408.jar:na]
>         at org.apache.hadoop.fs.FileSystem.create(FileSystem.java:923) ~[hadoop-common-2.4.1-mapr-1408.jar:na]
>         at org.apache.hadoop.fs.FileSystem.create(FileSystem.java:820) ~[hadoop-common-2.4.1-mapr-1408.jar:na]
>         at org.apache.hadoop.fs.FileSystem.create(FileSystem.java:809) ~[hadoop-common-2.4.1-mapr-1408.jar:na]
>         at org.apache.drill.exec.store.dfs.DrillFileSystem.create(DrillFileSystem.java:175) ~[drill-java-exec-1.0.0-SNAPSHOT-rebuffed.jar:1.0.0-SNAPSHOT]
>         at org.apache.drill.exec.store.sys.local.FilePStore.put(FilePStore.java:145) ~[drill-java-exec-1.0.0-SNAPSHOT-rebuffed.jar:1.0.0-SNAPSHOT]
>         ... 19 common frames omitted
> {code}



--
This message was sent by Atlassian JIRA
(v6.3.4#6332)