" *Release 24.3 of Dremio will continue to write Parquet V1, since an average performance degradation of 1.5% was observed in writes and 6.5% was observed in queries when TPC-DS data was written using Parquet V2 instead of Parquet V1. The aforementioned query performance tests utilized the C3 cache to store data.*" (...) "*Users can enable Parquet V2 on write using the following configuration key.*
ALTER SYSTEM SET "store.parquet.writer.version" = 'v2' " https://www.dremio.com/blog/vectorized-reading-of-parquet-v2-improves-performance-up-to-75/ "*Java Vector API support* *The feature is experimental and is currently not part of the parquet distribution. Parquet-MR has supported Java Vector API to speed up reading, to enable this feature:Java 17+, 64-bitRequiring the CPU to support instruction sets:avx512vbmiavx512_vbmi2To build the jars: mvn clean package -P vector-pluginsFor Apache Spark to enable this feature:Build parquet and replace the parquet-encoding-{VERSION}.jar on the spark jars folderBuild parquet-encoding-vector and copy parquet-encoding-vector-{VERSION}.jar to the spark jars folderEdit spark class#VectorizedRleValuesReader, function#readNextGroup refer to parquet class#ParquetReadRouter, function#readBatchUsing512VectorBuild spark with maven and replace spark-sql_2.12-{VERSION}.jar on the spark jars folder*" https://github.com/apache/parquet-mr?tab=readme-ov-file#java-vector-api-support You are using spark 3.2.0 spark version 3.2.4 was released April 13, 2023 https://spark.apache.org/releases/spark-release-3-2-4.html You are using a spark version that is EOL. tor. 18. apr. 2024 kl. 00:25 skrev Prem Sahoo <prem.re...@gmail.com>: > Hello Ryan, > May I know how you can write Parquet V2 encoding from spark 3.2.0 ? As > per my knowledge Dremio is creating and reading Parquet V2. > "Apache Parquet-MR Writer version PARQUET_2_0, which is widely adopted by > engines that write Parquet data, supports delta encodings. However, these > encodings were not previously supported by Dremio's vectorized Parquet > reader, resulting in decreased speed. Now, in version 24.3 and Dremio > Cloud, when you use the Dremio SQL query engine on Parquet datasets, you’ll > receive best-in-class performance." > > Could you let me know where Parquet Community is not recommending Parquet > V2 ? > > > > On Wed, Apr 17, 2024 at 2:44 PM Ryan Blue <b...@tabular.io> wrote: > >> Prem, as I said earlier, v2 is not a finalized spec so you should not use >> it. That's why it is not the default. You can get Spark to write v2 files, >> but it isn't recommended by the Parquet community. >> >> On Wed, Apr 17, 2024 at 11:05 AM Prem Sahoo <prem.re...@gmail.com> wrote: >> >>> Hello Community, >>> Could anyone shed more light on this (Spark Supporting Parquet V2)? >>> >>> On Tue, Apr 16, 2024 at 3:42 PM Mich Talebzadeh < >>> mich.talebza...@gmail.com> wrote: >>> >>>> Hi Prem, >>>> >>>> Regrettably this is not my area of speciality. I trust >>>> another colleague will have a more informed idea. Alternatively you may >>>> raise an SPIP for it. >>>> >>>> Spark Project Improvement Proposals (SPIP) | Apache Spark >>>> <https://spark.apache.org/improvement-proposals.html> >>>> >>>> HTH >>>> >>>> Mich Talebzadeh, >>>> Technologist | Solutions Architect | Data Engineer | Generative AI >>>> London >>>> United Kingdom >>>> >>>> >>>> view my Linkedin profile >>>> <https://www.linkedin.com/in/mich-talebzadeh-ph-d-5205b2/> >>>> >>>> >>>> https://en.everybodywiki.com/Mich_Talebzadeh >>>> >>>> >>>> >>>> *Disclaimer:* The information provided is correct to the best of my >>>> knowledge but of course cannot be guaranteed . It is essential to note >>>> that, as with any advice, quote "one test result is worth one-thousand >>>> expert opinions (Werner >>>> <https://en.wikipedia.org/wiki/Wernher_von_Braun>Von Braun >>>> <https://en.wikipedia.org/wiki/Wernher_von_Braun>)". >>>> >>>> >>>> On Tue, 16 Apr 2024 at 18:17, Prem Sahoo <prem.re...@gmail.com> wrote: >>>> >>>>> Hello Mich, >>>>> Thanks for example. >>>>> I have the same parquet-mr version which creates Parquet version 1. We >>>>> need to create V2 as it is more optimized. We have Dremio where if we use >>>>> Parquet V2 it is 75% better than Parquet V1 in case of read and 25 % >>>>> better >>>>> in case of write . so we are inclined towards this way. Please let us >>>>> know >>>>> why Spark is not going towards Parquet V2 ? >>>>> Sent from my iPhone >>>>> >>>>> On Apr 16, 2024, at 1:04 PM, Mich Talebzadeh < >>>>> mich.talebza...@gmail.com> wrote: >>>>> >>>>> >>>>> Well let us do a test in PySpark. >>>>> >>>>> Take this code and create a default parquet file. My spark is 3.4 >>>>> >>>>> cat parquet_checxk.py >>>>> from pyspark.sql import SparkSession >>>>> >>>>> spark = >>>>> SparkSession.builder.appName("ParquetVersionExample").getOrCreate() >>>>> >>>>> data = [("London", 8974432), ("New York City", 8804348), ("Beijing", >>>>> 21893000)] >>>>> df = spark.createDataFrame(data, ["city", "population"]) >>>>> >>>>> df.write.mode("overwrite").parquet("parquet_example") # it create >>>>> file in hdfs directory >>>>> >>>>> Use a tool called parquet-tools (downloadable using pip from >>>>> https://pypi.org/project/parquet-tools/) >>>>> >>>>> Get the parquet files from hdfs to the current directory say >>>>> >>>>> hdfs dfs -get /user/hduser/parquet_example . >>>>> cd ./parquet_example >>>>> do an ls and pickup file 3 like below to inspect >>>>> parquet-tools inspect >>>>> part-00003-c33854c8-a8b6-4315-bf51-20198ce0ba62-c000.snappy.parquet >>>>> >>>>> Now this is the output >>>>> >>>>> ############ file meta data ############ >>>>> created_by: parquet-mr version 1.12.3 (build >>>>> f8dced182c4c1fbdec6ccb3185537b5a01e6ed6b) >>>>> num_columns: 2 >>>>> num_rows: 1 >>>>> num_row_groups: 1 >>>>> format_version: 1.0 >>>>> serialized_size: 563 >>>>> >>>>> >>>>> ############ Columns ############ >>>>> name >>>>> age >>>>> >>>>> ############ Column(name) ############ >>>>> name: name >>>>> path: name >>>>> max_definition_level: 1 >>>>> max_repetition_level: 0 >>>>> physical_type: BYTE_ARRAY >>>>> logical_type: String >>>>> converted_type (legacy): UTF8 >>>>> compression: SNAPPY (space_saved: -5%) >>>>> >>>>> ############ Column(age) ############ >>>>> name: age >>>>> path: age >>>>> max_definition_level: 1 >>>>> max_repetition_level: 0 >>>>> physical_type: INT64 >>>>> logical_type: None >>>>> converted_type (legacy): NONE >>>>> compression: SNAPPY (space_saved: -5%) >>>>> >>>>> File Information: >>>>> >>>>> - format_version: 1.0: This line explicitly states that the format >>>>> version of the Parquet file is 1.0, which corresponds to Parquet >>>>> version 1. >>>>> - created_by: parquet-mr version 1.12.3: While this doesn't >>>>> directly specify the format version, itt is accepted that older >>>>> versions of >>>>> parquet-mr like 1.12.3 typically write Parquet version 1 files. >>>>> >>>>> Since in this case Spark 3.4 is capable of reading both versions (1 >>>>> and 2), you don't necessarily need to modify your Spark code to access >>>>> this file. However, if you want to create Parquet files in version 2 using >>>>> Spark, you might need to consider additional changes like excluding >>>>> parquet-mr or upgrading Parquet libraries and do a custom build.of Spark. >>>>> However, taking klaws of diminishing returns, I would not advise that >>>>> either.. You can ofcourse usse gzip for compression that may be more >>>>> suitable for your needs. >>>>> >>>>> HTH >>>>> >>>>> Mich Talebzadeh, >>>>> Technologist | Solutions Architect | Data Engineer | Generative AI >>>>> London >>>>> United Kingdom >>>>> >>>>> >>>>> view my Linkedin profile >>>>> <https://www.linkedin.com/in/mich-talebzadeh-ph-d-5205b2/> >>>>> >>>>> >>>>> https://en.everybodywiki.com/Mich_Talebzadeh >>>>> >>>>> >>>>> >>>>> *Disclaimer:* The information provided is correct to the best of my >>>>> knowledge but of course cannot be guaranteed . It is essential to note >>>>> that, as with any advice, quote "one test result is worth one-thousand >>>>> expert opinions (Werner >>>>> <https://en.wikipedia.org/wiki/Wernher_von_Braun>Von Braun >>>>> <https://en.wikipedia.org/wiki/Wernher_von_Braun>)". >>>>> >>>>> >>>>> On Tue, 16 Apr 2024 at 15:00, Prem Sahoo <prem.re...@gmail.com> wrote: >>>>> >>>>>> Hello Community, >>>>>> Could any of you shed some light on below questions please ? >>>>>> Sent from my iPhone >>>>>> >>>>>> On Apr 15, 2024, at 9:02 PM, Prem Sahoo <prem.re...@gmail.com> wrote: >>>>>> >>>>>> >>>>>> Any specific reason spark does not support or community doesn't want >>>>>> to go to Parquet V2 , which is more optimized and read and write is too >>>>>> much faster (form other component which I am using) >>>>>> >>>>>> On Mon, Apr 15, 2024 at 7:55 PM Ryan Blue <b...@tabular.io> wrote: >>>>>> >>>>>>> Spark will read data written with v2 encodings just fine. You just >>>>>>> don't need to worry about making Spark produce v2. And you should >>>>>>> probably >>>>>>> also not produce v2 encodings from other systems. >>>>>>> >>>>>>> On Mon, Apr 15, 2024 at 4:37 PM Prem Sahoo <prem.re...@gmail.com> >>>>>>> wrote: >>>>>>> >>>>>>>> oops but so spark does not support parquet V2 atm ?, as We have a >>>>>>>> use case where we need parquet V2 as one of our components uses >>>>>>>> Parquet V2 >>>>>>>> . >>>>>>>> >>>>>>>> On Mon, Apr 15, 2024 at 7:09 PM Ryan Blue <b...@tabular.io> wrote: >>>>>>>> >>>>>>>>> Hi Prem, >>>>>>>>> >>>>>>>>> Parquet v1 is the default because v2 has not been finalized and >>>>>>>>> adopted by the community. I highly recommend not using v2 encodings >>>>>>>>> at this >>>>>>>>> time. >>>>>>>>> >>>>>>>>> Ryan >>>>>>>>> >>>>>>>>> On Mon, Apr 15, 2024 at 3:05 PM Prem Sahoo <prem.re...@gmail.com> >>>>>>>>> wrote: >>>>>>>>> >>>>>>>>>> I am using spark 3.2.0 . but my spark package comes with >>>>>>>>>> parquet-mr 1.2.1 which writes in parquet version 1 not version >>>>>>>>>> version 2:(. >>>>>>>>>> so I was looking how to write in Parquet version2 ? >>>>>>>>>> >>>>>>>>>> On Mon, Apr 15, 2024 at 5:05 PM Mich Talebzadeh < >>>>>>>>>> mich.talebza...@gmail.com> wrote: >>>>>>>>>> >>>>>>>>>>> Sorry you have a point there. It was released in version 3.00. >>>>>>>>>>> What version of spark are you using? >>>>>>>>>>> >>>>>>>>>>> Technologist | Solutions Architect | Data Engineer | Generative >>>>>>>>>>> AI >>>>>>>>>>> London >>>>>>>>>>> United Kingdom >>>>>>>>>>> >>>>>>>>>>> >>>>>>>>>>> view my Linkedin profile >>>>>>>>>>> <https://www.linkedin.com/in/mich-talebzadeh-ph-d-5205b2/> >>>>>>>>>>> >>>>>>>>>>> >>>>>>>>>>> https://en.everybodywiki.com/Mich_Talebzadeh >>>>>>>>>>> >>>>>>>>>>> >>>>>>>>>>> >>>>>>>>>>> *Disclaimer:* The information provided is correct to the best >>>>>>>>>>> of my knowledge but of course cannot be guaranteed . It is >>>>>>>>>>> essential to >>>>>>>>>>> note that, as with any advice, quote "one test result is worth >>>>>>>>>>> one-thousand expert opinions (Werner >>>>>>>>>>> <https://en.wikipedia.org/wiki/Wernher_von_Braun>Von Braun >>>>>>>>>>> <https://en.wikipedia.org/wiki/Wernher_von_Braun>)". >>>>>>>>>>> >>>>>>>>>>> >>>>>>>>>>> On Mon, 15 Apr 2024 at 21:33, Prem Sahoo <prem.re...@gmail.com> >>>>>>>>>>> wrote: >>>>>>>>>>> >>>>>>>>>>>> Thank you so much for the info! But do we have any release >>>>>>>>>>>> notes where it says spark2.4.0 onwards supports parquet version 2. >>>>>>>>>>>> I was >>>>>>>>>>>> under the impression Spark3.0 onwards it started supporting . >>>>>>>>>>>> >>>>>>>>>>>> >>>>>>>>>>>> >>>>>>>>>>>> >>>>>>>>>>>> On Mon, Apr 15, 2024 at 4:28 PM Mich Talebzadeh < >>>>>>>>>>>> mich.talebza...@gmail.com> wrote: >>>>>>>>>>>> >>>>>>>>>>>>> Well if I am correct, Parquet version 2 support was introduced >>>>>>>>>>>>> in Spark version 2.4.0. Therefore, any version of Spark starting >>>>>>>>>>>>> from 2.4.0 >>>>>>>>>>>>> supports Parquet version 2. Assuming that you are using Spark >>>>>>>>>>>>> version >>>>>>>>>>>>> 2.4.0 or later, you should be able to take advantage of Parquet >>>>>>>>>>>>> version 2 >>>>>>>>>>>>> features. >>>>>>>>>>>>> >>>>>>>>>>>>> HTH >>>>>>>>>>>>> >>>>>>>>>>>>> Mich Talebzadeh, >>>>>>>>>>>>> Technologist | Solutions Architect | Data Engineer | >>>>>>>>>>>>> Generative AI >>>>>>>>>>>>> London >>>>>>>>>>>>> United Kingdom >>>>>>>>>>>>> >>>>>>>>>>>>> >>>>>>>>>>>>> view my Linkedin profile >>>>>>>>>>>>> <https://www.linkedin.com/in/mich-talebzadeh-ph-d-5205b2/> >>>>>>>>>>>>> >>>>>>>>>>>>> >>>>>>>>>>>>> https://en.everybodywiki.com/Mich_Talebzadeh >>>>>>>>>>>>> >>>>>>>>>>>>> >>>>>>>>>>>>> >>>>>>>>>>>>> *Disclaimer:* The information provided is correct to the best >>>>>>>>>>>>> of my knowledge but of course cannot be guaranteed . It is >>>>>>>>>>>>> essential to >>>>>>>>>>>>> note that, as with any advice, quote "one test result is >>>>>>>>>>>>> worth one-thousand expert opinions (Werner >>>>>>>>>>>>> <https://en.wikipedia.org/wiki/Wernher_von_Braun>Von Braun >>>>>>>>>>>>> <https://en.wikipedia.org/wiki/Wernher_von_Braun>)". >>>>>>>>>>>>> >>>>>>>>>>>>> >>>>>>>>>>>>> On Mon, 15 Apr 2024 at 20:53, Prem Sahoo <prem.re...@gmail.com> >>>>>>>>>>>>> wrote: >>>>>>>>>>>>> >>>>>>>>>>>>>> Thank you for the information! >>>>>>>>>>>>>> I can use any version of parquet-mr to produce parquet file. >>>>>>>>>>>>>> >>>>>>>>>>>>>> regarding 2nd question . >>>>>>>>>>>>>> Which version of spark is supporting parquet version 2? >>>>>>>>>>>>>> May I get the release notes where parquet versions are >>>>>>>>>>>>>> mentioned ? >>>>>>>>>>>>>> >>>>>>>>>>>>>> >>>>>>>>>>>>>> On Mon, Apr 15, 2024 at 2:34 PM Mich Talebzadeh < >>>>>>>>>>>>>> mich.talebza...@gmail.com> wrote: >>>>>>>>>>>>>> >>>>>>>>>>>>>>> Parquet-mr is a Java library that provides functionality >>>>>>>>>>>>>>> for working with Parquet files with hadoop. It is therefore >>>>>>>>>>>>>>> more geared >>>>>>>>>>>>>>> towards working with Parquet files within the Hadoop ecosystem, >>>>>>>>>>>>>>> particularly using MapReduce jobs. There is no definitive way >>>>>>>>>>>>>>> to check >>>>>>>>>>>>>>> exact compatible versions within the library itself. However, >>>>>>>>>>>>>>> you can have >>>>>>>>>>>>>>> a look at this >>>>>>>>>>>>>>> >>>>>>>>>>>>>>> https://github.com/apache/parquet-mr/blob/master/CHANGES.md >>>>>>>>>>>>>>> >>>>>>>>>>>>>>> HTH >>>>>>>>>>>>>>> >>>>>>>>>>>>>>> Mich Talebzadeh, >>>>>>>>>>>>>>> Technologist | Solutions Architect | Data Engineer | >>>>>>>>>>>>>>> Generative AI >>>>>>>>>>>>>>> London >>>>>>>>>>>>>>> United Kingdom >>>>>>>>>>>>>>> >>>>>>>>>>>>>>> >>>>>>>>>>>>>>> view my Linkedin profile >>>>>>>>>>>>>>> <https://www.linkedin.com/in/mich-talebzadeh-ph-d-5205b2/> >>>>>>>>>>>>>>> >>>>>>>>>>>>>>> >>>>>>>>>>>>>>> https://en.everybodywiki.com/Mich_Talebzadeh >>>>>>>>>>>>>>> >>>>>>>>>>>>>>> >>>>>>>>>>>>>>> >>>>>>>>>>>>>>> *Disclaimer:* The information provided is correct to the >>>>>>>>>>>>>>> best of my knowledge but of course cannot be guaranteed . It is >>>>>>>>>>>>>>> essential >>>>>>>>>>>>>>> to note that, as with any advice, quote "one test result is >>>>>>>>>>>>>>> worth one-thousand expert opinions (Werner >>>>>>>>>>>>>>> <https://en.wikipedia.org/wiki/Wernher_von_Braun>Von Braun >>>>>>>>>>>>>>> <https://en.wikipedia.org/wiki/Wernher_von_Braun>)". >>>>>>>>>>>>>>> >>>>>>>>>>>>>>> >>>>>>>>>>>>>>> On Mon, 15 Apr 2024 at 18:59, Prem Sahoo < >>>>>>>>>>>>>>> prem.re...@gmail.com> wrote: >>>>>>>>>>>>>>> >>>>>>>>>>>>>>>> Hello Team, >>>>>>>>>>>>>>>> May I know how to check which version of parquet is >>>>>>>>>>>>>>>> supported by parquet-mr 1.2.1 ? >>>>>>>>>>>>>>>> >>>>>>>>>>>>>>>> Which version of parquet-mr is supporting parquet version 2 >>>>>>>>>>>>>>>> (V2) ? >>>>>>>>>>>>>>>> >>>>>>>>>>>>>>>> Which version of spark is supporting parquet version 2? >>>>>>>>>>>>>>>> May I get the release notes where parquet versions are >>>>>>>>>>>>>>>> mentioned ? >>>>>>>>>>>>>>>> >>>>>>>>>>>>>>> >>>>>>>>> >>>>>>>>> -- >>>>>>>>> Ryan Blue >>>>>>>>> Tabular >>>>>>>>> >>>>>>>> >>>>>>> >>>>>>> -- >>>>>>> Ryan Blue >>>>>>> Tabular >>>>>>> >>>>>> >> >> -- >> Ryan Blue >> Tabular >> > -- Bjørn Jørgensen Vestre Aspehaug 4, 6010 Ålesund Norge +47 480 94 297