Bhavana, Could you send me (omal...@apache.org) the incorrect ORC file? Which file system were you using? hdfs? Which version of Hadoop and Hive?
Thanks, Owen On Fri, May 22, 2015 at 9:37 AM, Grant Overby (groverby) <grove...@cisco.com > wrote: > I’m getting the following exception when Hive executes a query on an > external table. It seems the postscript isn’t written even though .close() > is called and returns normally. Any thoughts? > > > java.io.IOException: Malformed ORC file > hdfs://twig06.twigs:8020/warehouse/completed/events/connection_events/dt=1432229400/1432229419251-bb46892c-939f-45ca-b867-da3675d0ca72.orc. > Invalid postscript length 0 > > at > org.apache.hadoop.hive.ql.io.orc.ReaderImpl.ensureOrcFooter(ReaderImpl.java:230) > > at > org.apache.hadoop.hive.ql.io.orc.ReaderImpl.extractMetaInfoFromFooter(ReaderImpl.java:370) > > at org.apache.hadoop.hive.ql.io.orc.ReaderImpl.<init>(ReaderImpl.java:311) > > at org.apache.hadoop.hive.ql.io.orc.OrcFile.createReader(OrcFile.java:228) > > at > org.apache.hadoop.hive.ql.io.orc.OrcInputFormat.getReader(OrcInputFormat.java:1130) > > at > org.apache.hadoop.hive.ql.io.orc.OrcInputFormat.getRecordReader(OrcInputFormat.java:1039) > > at > org.apache.hadoop.hive.ql.io.HiveInputFormat.getRecordReader(HiveInputFormat.java:246) > > These orc files are written manually using an orc writer: > > Path tmpPath = new Path(tmpPathName); > Configuration writerConf = new Configuration(); > OrcFile.WriterOptions writerOptions = OrcFile.writerOptions(writerConf); > writerOptions.bufferSize(256 * 1024); > writerOptions.compress(SNAPPY); > writerOptions.fileSystem(fileSystem); > writerOptions.inspector(new FlatTableObjectInspector(dbName + "." + > tableName, fields)); > writerOptions.rowIndexStride(10_000); > writerOptions.blockPadding(true); > writerOptions.stripeSize(122 * 1024 * 1024); > writerOptions.version(V_0_12); > writer = OrcFile.createWriter(tmpPath, writerOptions); > > > The writer.close() is executed and only if writer.close() returns > normally is the orc file moved from a tmp dir to the external table > partition’s dir. > > private void closeWriter() { > if (writer != null) { > try { > writer.close(); > Path tmpPath = new Path(tmpPathName); > if (fileSystem.exists(tmpPath) && > fileSystem.getFileStatus(tmpPath).getLen() > 0) { > Path completedPath = new Path(completedPathName); > fileSystem.setPermission(tmpPath, PERMISSION_664); > fileSystem.rename(tmpPath, completedPath); > > HiveOperations.getInstance().registerExternalizedPartition(dbName, tableName, > partition); > } else if (fileSystem.exists(tmpPath)) { > fileSystem.delete(tmpPath, false); > } > } catch (IOException e) { > Throwables.propagate(e); > } finally { > writer = null; > } > } > } > > > I expect writer.close() to write the postscript, but it seems not to > have. > > > http://grepcode.com/file/repo1.maven.org/maven2/org.apache.hive/hive-exec/0.14.0/org/apache/hadoop/hive/ql/io/orc/WriterImpl.java#WriterImpl.close%28%29 > > > Thoughts? > Am I doing something wrong? Bug? > Fix? > > *Grant Overby* > Software Engineer > Cisco.com <http://www.cisco.com/> > grove...@cisco.com > Mobile: *865 724 4910 <865%20724%204910>* > > > > Think before you print. > > This email may contain confidential and privileged material for the sole > use of the intended recipient. Any review, use, distribution or disclosure > by others is strictly prohibited. If you are not the intended recipient (or > authorized to receive for the recipient), please contact the sender by > reply email and delete all copies of this message. > > Please click here > <http://www.cisco.com/web/about/doing_business/legal/cri/index.html> for > Company Registration Information. > > > >