Data was written with Spark but I'm using the parquet APIs directly for
reads. I checked the stats in the footer with the following code.

ParquetMetadata metadata = ParquetFileReader.readFooter(conf, path,
ParquetMetadataConverter.NO_FILTER);
ColumnPath deviceId = ColumnPath.get("deviceId");
metadata.getBlocks().forEach(b -> {
    if (b.getTotalByteSize() > 4 * 1024 * 1024L) {
        System.out.println("\nBlockSize = " + b.getTotalByteSize());
        System.out.println("ComprSize = " + b.getCompressedSize());
        System.out.println("Num Rows  = " + b.getRowCount());
        b.getColumns().forEach(c -> {
            if (c.getPath().equals(deviceId)) {
                Comparable max = c.getStatistics().genericGetMax();
                Comparable min = c.getStatistics().genericGetMin();
                System.out.println("\t" + c.getPath() + " [" + min +
", " + max + "]");
            }
        });
    }
});


Thanks,
Pradeep

On Fri, Feb 10, 2017 at 9:08 AM, Lars Volker <l...@cloudera.com> wrote:

> Hi Pradeep,
>
> I don't have any experience with using Parquet APIs through Spark. That
> being said, there are currently several issues around column statistics,
> both in the format and in the parquet-mr implementation (PARQUET-686,
> PARQUET-839, PARQUET-840).
>
> However, in your case and depending on the versions involved, you might
> also hit PARQUET-251, which can cause statistics for some files to be
> ignored. In this context it may be worth to have a look at this file:
> https://github.com/apache/parquet-mr/blob/master/
> parquet-column/src/main/java/org/apache/parquet/CorruptStatistics.java
>
> How did you check that the statistics are not written to the footer? If you
> used parquet-mr, they may be there but be ignored.
>
> Cheers, Lars
>
> On Fri, Feb 10, 2017 at 5:31 PM, Pradeep Gollakota <pradeep...@gmail.com>
> wrote:
>
> > Bumping the thread to see if I get any responses.
> >
> > On Wed, Feb 8, 2017 at 6:49 PM, Pradeep Gollakota <pradeep...@gmail.com>
> > wrote:
> >
> > > Hi folks,
> > >
> > > I generated a bunch of parquet files using spark and
> > > ParquetThriftOutputFormat. The thirft model has a column called
> > "deviceId"
> > > which is a string column. It also has a "timestamp" column of int64.
> > After
> > > the files have been generated, I inspected the file footers and noticed
> > > that only the "timestamp" field has min/max statistics. My primary
> filter
> > > will be deviceId, the data is partitioned and sorted by deviceId, but
> > since
> > > the statistics data is missing, it's not able to prune blocks from
> being
> > > read. Am I missing some configuration setting that allows it to
> generate
> > > the stats data? The following is code is how an RDD[Thrift] is being
> > saved
> > > to parquet. The configuration is default configuration.
> > >
> > > implicit class ThriftRDD[T <: TBase[T, _ <: TFieldIdEnum] :
> > ClassTag](rdd: RDD[T]) {
> > >   def saveAsParquet(output: String,
> > >                     conf: Configuration = rdd.context.
> hadoopConfiguration):
> > Unit = {
> > >     val job = Job.getInstance(conf)
> > >     val clazz: Class[T] = classTag[T].runtimeClass.
> > asInstanceOf[Class[T]]
> > >     ParquetThriftOutputFormat.setThriftClass(job, clazz)
> > >     val r = rdd.map[(Void, T)](x => (null, x))
> > >       .saveAsNewAPIHadoopFile(
> > >         output,
> > >         classOf[Void],
> > >         clazz,
> > >         classOf[ParquetThriftOutputFormat[T]],
> > >         job.getConfiguration)
> > >   }
> > > }
> > >
> > >
> > > Thanks,
> > > Pradeep
> > >
> >
>

Reply via email to