Cool. At some point we are interested in having simple compressed
(e.g. with LZ4 or ZSTD) record batches natively in the Arrow protocol,
see

https://issues.apache.org/jira/browse/ARROW-300

On Thu, May 23, 2019 at 10:21 AM Joris Peeters
<joris.mg.peet...@gmail.com> wrote:
>
> Cool, thanks. I think we'll just go with reading LZ4 compressed Arrow
> directly from disk then, and by-pass Parquet altogether.
> The compressed Arrow files are about 20% larger than the PQ files, but
> getting it into some useful form in memory is almost on par with pandas.
>
> At the moment, I don't need the additional benefits parquet would give
> (various forms of filtering), so this is perfectly OK. And now I don't have
> half of Hadoop in my dependencies. \o/
> But yes, definitely +1 on a self-contained vectorised Parquet reader.
>
> -J
>
>
> On Thu, May 23, 2019 at 2:58 PM Wes McKinney <wesmck...@gmail.com> wrote:
>
> > hi Joris,
> >
> > The Apache Parquet mailing list is
> >
> > d...@parquet.apache.org
> >
> > I'm copying the list here
> >
> > AFAIK parquet-mr doesn't feature vectorized reading (for Arrow or
> > otherwise). There are some vectorized Java-based readers in the wild:
> > in Dremio [1] and Apache Spark, at least. I'm interested to see a
> > reusable library that supports vectorized Arrow reads in Java.
> >
> > - Wes
> >
> > [1]: https://github.com/dremio/dremio-oss
> >
> > On Thu, May 23, 2019 at 8:54 AM Joris Peeters
> > <joris.mg.peet...@gmail.com> wrote:
> > >
> > > Hello,
> > >
> > > I'm trying to read a Parquet file from disk into Arrow in memory, in
> > Scala.
> > > I'm wondering what the most efficient approach is, especially for the
> > > reading part. I'm aware that Parquet reading is perhaps beyond the scope
> > of
> > > this mailing list but,
> > >
> > > - I believe Arrow and Parquet are closely intertwined these days?
> > > - I can't find an appropriate Parquet mailing list.
> > >
> > > Any pointers would be appreciated!
> > >
> > > Below is the code I currently have. My concern is that this alone already
> > > takes about 2s, whereas "pq.read_pandas(the_file_path).to_pandas()" takes
> > > ~=100ms in Python. So I suspect I'm not doing this in the most efficient
> > > way possible ... The Parquet data holds 1570150 rows, with 14 columns of
> > > various types, and takes 15MB on disk.
> > >
> > > import org.apache.hadoop.conf.Configuration
> > > import org.apache.parquet.column.ColumnDescriptor
> > > import
> > org.apache.parquet.example.data.simple.convert.GroupRecordConverter
> > > import org.apache.parquet.format.converter.ParquetMetadataConverter
> > > import org.apache.parquet.hadoop.{ParquetFileReader}
> > > import org.apache.parquet.io.ColumnIOFactory
> > >
> > > ...
> > >
> > > val path: Path = Paths.get("C:\\item.pq")
> > > val jpath = new org.apache.hadoop.fs.Path(path.toFile.getAbsolutePath)
> > > val conf = new Configuration()
> > >
> > > val readFooter = ParquetFileReader.readFooter(conf, jpath,
> > > ParquetMetadataConverter.NO_FILTER)
> > > val schema = readFooter.getFileMetaData.getSchema
> > > val r = ParquetFileReader.open(conf, jpath)
> > >
> > > val pages = r.readNextRowGroup()
> > > val rows = pages.getRowCount
> > >
> > > val columnIO = new ColumnIOFactory().getColumnIO(schema)
> > > val recordReader = columnIO.getRecordReader(pages, new
> > > GroupRecordConverter(schema))
> > >
> > > // This takes about 2s
> > > (1 to rows.toInt).map { i =>
> > >   val group = recordReader.read
> > >   // Just read first column for now ...
> > >   val x = group.getLong(0,0)
> > > }
> > >
> > > ...
> > >
> > > As this will be in the hot path of my code, I'm quite keen to make it
> > > as fast as possible. Note that the eventual objective is to build
> > > Arrow data. I was assuming there would be a way to quickly load the
> > > columns. I suspect the loop over the rows, building row-based records,
> > > is causing a lot of overhead, but can't seem to find another way.
> > >
> > >
> > > Thanks,
> > >
> > > -J
> >

Reply via email to