Thanks for the feedback, just filed https://issues.apache.org/jira/browse/SPARK-10803 to track this issue.

Cheng

On 9/24/15 4:25 AM, Borisa Zivkovic wrote:
Hi,

your suggestion works nicely.. I was able to attach metadata to columns and read that metadata from spark and by using ParquetFileReader It would be nice if we had a way to manipulate parquet metadata directly from DataFrames though.

regards

On Wed, 23 Sep 2015 at 09:25 Borisa Zivkovic <borisha.zivko...@gmail.com <mailto:borisha.zivko...@gmail.com>> wrote:

    Hi,

    thanks a lot for this! I will try it out to see if this works ok.

    I am planning to use "stable" metadata - so those will be same
    across all parquet files inside directory hierarchy...



    On Tue, 22 Sep 2015 at 18:54 Cheng Lian <lian.cs....@gmail.com
    <mailto:lian.cs....@gmail.com>> wrote:

        Michael reminded me that although we don't support direct
        manipulation over Parquet metadata, you can still save/query
        metadata to/from Parquet via DataFrame per-column metadata.
        For example:

        import sqlContext.implicits._
        import org.apache.spark.sql.types.MetadataBuilder

        val path = "file:///tmp/parquet/meta"

        // Saving metadata
        val meta = new MetadataBuilder().putString("appVersion",
        "1.0.2").build()
        sqlContext.range(10).select($"id".as("id",
        meta)).coalesce(1).write.mode("overwrite").parquet(path)

        // Querying metadata
        
sqlContext.read.parquet(path).schema("id").metadata.getString("appVersion")

        The metadata is saved together with Spark SQL schema as a JSON
        string. For example, the above code generates the following
        Parquet metadata (inspected with parquet-meta):

        file:
        
file:/private/tmp/parquet/meta/part-r-00000-77cb2237-e6a8-4cb6-a452-ae205ba7b660.gz.parquet
        creator:     parquet-mr version 1.6.0
        extra: org.apache.spark.sql.parquet.row.metadata =
        
{"type":"struct","fields":[{"name":"id","type":"long","nullable":true,*"metadata":{"appVersion":"1.0.2"}*}]}


        Cheng


        On 9/22/15 9:37 AM, Cheng Lian wrote:
        I see, this makes sense. We should probably add this in Spark
        SQL.

        However, there's one corner case to note about user-defined
        Parquet metadata. When committing a write job,
        ParquetOutputCommitter writes Parquet summary files
        (_metadata and _common_metadata), and user-defined key-value
        metadata written in all Parquet part-files get merged here.
        The problem is that, if a single key is associated with
        multiple values, Parquet doesn't know how to reconcile this
        situation, and simply gives up writing summary files. This
        can be particular annoying for appending. In general, users
        should avoid storing "unstable" values like timestamps as
        Parquet metadata.

        Cheng

        On 9/22/15 1:58 AM, Borisa Zivkovic wrote:
        thanks for answer.

        I need this in order to be able to track schema metadata.

        basically when I create parquet files from Spark I want to
        be able to "tag" them in some way (giving the schema
        appropriate name or attaching some key/values) and then it
        is fairly easy to get basic metadata about parquet files
        when processing and discovering those later on.

        On Mon, 21 Sep 2015 at 18:17 Cheng Lian
        <lian.cs....@gmail.com <mailto:lian.cs....@gmail.com>> wrote:

            Currently Spark SQL doesn't support customizing schema
            name and
            metadata. May I know why these two matters in your use
            case? Some
            Parquet data models, like parquet-avro, do support it,
            while some others
            don't (e.g. parquet-hive).

            Cheng

            On 9/21/15 7:13 AM, Borisa Zivkovic wrote:
            > Hi,
            >
            > I am trying to figure out how to write parquet
            metadata when
            > persisting DataFrames to parquet using Spark (1.4.1)
            >
            > I could not find a way to change schema name (which
            seems to be
            > hardcoded to root) and also how to add data to
            key/value metadata in
            > parquet footer.
            >
            >
            org.apache.parquet.hadoop.metadata.FileMetaData#getKeyValueMetaData
            >
            > org.apache.parquet.schema.Type#getName
            >
            > thanks
            >
            >




Reply via email to