For metadata, you can use 'parquet-tools dump' and pipe the output to
more/less.
Parquet dump will print the block (aka row group) and page level metadata.
It will then dump all the data so be prepared to cancel when that happens.

Setting dfs.blocksize == parquet.blocksize is a very good idea and is the
general recommendataion.

Larger block (i.e row group) sizes will increase memory use on write. It
may not have a noticeable impact on read memory use as the current Parquet
reader reads data per page.

There are other potential effects of varying parquet block/row group size.
With filter pushdown to the row group level, a smaller row group will have
better chances of being effectively filtered out. This is still being
worked on, but will become a factor at some time.

Note that  Parquet file can have many row groups and can span many nodes,
but as long as a row group is not split across nodes, reader performance
will not suffer.








On Fri, Jul 1, 2016 at 1:09 PM, John Omernik <j...@omernik.com> wrote:

> I am looking forward to the MapR 1.7 dev preview because of the metadata
> user impersonation JIRA fix.   "Drill always writes one row group per
> file." So is this one parquet block?  "row group" is a new term to this
> email :)
>
> On Fri, Jul 1, 2016 at 2:09 PM, Abdel Hakim Deneche <adene...@maprtech.com
> >
> wrote:
>
> > Just make sure you enable parquet metadata caching, otherwise the more
> > files you have the more time Drill will spend reading the metadata from
> > every single file.
> >
> > On Fri, Jul 1, 2016 at 11:17 AM, John Omernik <j...@omernik.com> wrote:
> >
> > > In addition
> > > 7. Generally speaking, keeping number of files low, will help in
> multiple
> > > phases of planning/execution. True/False
> > >
> > >
> > >
> > > On Fri, Jul 1, 2016 at 12:56 PM, John Omernik <j...@omernik.com>
> wrote:
> > >
> > > > I looked at that, and both the meta and schema options didn't provide
> > me
> > > > block size.
> > > >
> > > > I may be looking at parquet block size wrong, so let me toss out some
> > > > observations, and inferences I am making, and then others who know
> the
> > > > spec/format can confirm or correct.
> > > >
> > > > 1. The block size in parquet is NOT file size. A Parquet file can
> have
> > > > multiple blocks in a single file? (Question: when this occurs, do the
> > > > blocks then line up with DFS block size/chunk size as recommended, or
> > do
> > > we
> > > > get weird issues?) In practice, do writes aim for 1 block per file?
> > > > 2. The block size, when writing is computed prior to compression.
> This
> > is
> > > > an inference based on the parquet-mr library.  A job that has a
> parquet
> > > > block size of 384mb seems to average files of around 256 mb in size.
> > > Thus,
> > > > my theory is that the amount of data in parquet block size is
> computed
> > > > prior to write, and then as the file is written compression is
> applied,
> > > > thus ensuring that the block size (and file size if 1 is not true, or
> > if
> > > > you are just writing a single file) will be under the dfs.block size
> if
> > > you
> > > > make both settings the same.
> > > > 3. Because of 2, setting dfs.blocksize = parquet blocksize is a good
> > > rule,
> > > > because the files will always be under the dfsblock size with
> > > compression,
> > > > ensuring you don't have cross block reads happening.  (You don't have
> > to,
> > > > for example, set the parquet block size to be less then dfs block
> size
> > to
> > > > ensure you don't have any weird issues)
> > > > 4.  Also because of 2, with compression enabled, you don't need any
> > slack
> > > > space for file headers or footers to ensure the files don't cross DFS
> > > > blocks.
> > > > 5. In general larger dfs/parquet block sizes will be good for reader
> > > > performance, however, as you start to get larger, write memory
> demands
> > > > increase.  True/False?  In general does a larger block size also put
> > > > pressures on Reader memory?
> > > > 6. Any other thoughts/challenges on block size?  When talking about
> > > > hundreds/thousands of GB of data, little changes in performance like
> > with
> > > > block size can make a difference.  I am really interested in
> > tips/stories
> > > > to help me understand better.
> > > >
> > > > John
> > > >
> > > >
> > > >
> > > > On Fri, Jul 1, 2016 at 12:26 PM, Parth Chandra <
> pchan...@maprtech.com>
> > > > wrote:
> > > >
> > > >> parquet-tools perhaps?
> > > >>
> > > >> https://github.com/Parquet/parquet-mr/tree/master/parquet-tools
> > > >>
> > > >>
> > > >>
> > > >> On Fri, Jul 1, 2016 at 5:39 AM, John Omernik <j...@omernik.com>
> > wrote:
> > > >>
> > > >> > Is there any way, with Drill or with other tools, given a Parquet
> > > file,
> > > >> to
> > > >> > detect the block size it was written with?  I am copying data from
> > one
> > > >> > cluster to another, and trying to determine the block size.
> > > >> >
> > > >> > While I was able to get the size by asking the devs, I was
> > wondering,
> > > is
> > > >> > there any way to reliably detect it?
> > > >> >
> > > >> > John
> > > >> >
> > > >>
> > > >
> > > >
> > >
> >
> >
> >
> > --
> >
> > Abdelhakim Deneche
> >
> > Software Engineer
> >
> >   <http://www.mapr.com/>
> >
> >
> > Now Available - Free Hadoop On-Demand Training
> > <
> >
> http://www.mapr.com/training?utm_source=Email&utm_medium=Signature&utm_campaign=Free%20available
> > >
> >
>

Reply via email to