-
Notifications
You must be signed in to change notification settings - Fork 513
Commit
This commit does not belong to any branch on this repository, and may belong to a fork outside of the repository.
Propagate ParquetOutputFormat options to ParquetWriter (#4980)
* Propagate ParquetOutputFormat options to ParquetWriter * fixup * fixup * cleanup test * Apply configuration in scio-smb Parquet as well * Add missing runtime dep to scio-smb * add page size, writer version * Add page properties * javafmt * remove unused import * scope to test * Fix build issues --------- Co-authored-by: Michel Davit <[email protected]>
- Loading branch information
1 parent
6fe7a29
commit 110f795
Showing
6 changed files
with
279 additions
and
22 deletions.
There are no files selected for viewing
This file contains bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
This file contains bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
130 changes: 130 additions & 0 deletions
130
scio-parquet/src/test/scala/com/spotify/scio/parquet/TestWriterUtils.scala
This file contains bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
Original file line number | Diff line number | Diff line change |
---|---|---|
@@ -0,0 +1,130 @@ | ||
/* | ||
* Copyright 2023 Spotify AB. | ||
* | ||
* Licensed under the Apache License, Version 2.0 (the "License"); | ||
* you may not use this file except in compliance with the License. | ||
* You may obtain a copy of the License at | ||
* | ||
* http://www.apache.org/licenses/LICENSE-2.0 | ||
* | ||
* Unless required by applicable law or agreed to in writing, | ||
* software distributed under the License is distributed on an | ||
* "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY | ||
* KIND, either express or implied. See the License for the | ||
* specific language governing permissions and limitations | ||
* under the License. | ||
*/ | ||
|
||
package com.spotify.scio.parquet | ||
|
||
import com.spotify.scio._ | ||
import com.spotify.scio.parquet.avro._ | ||
import com.spotify.scio.avro.{Account, AccountStatus} | ||
import org.apache.commons.io.FileUtils | ||
import org.apache.parquet.HadoopReadOptions | ||
import org.apache.parquet.column.Encoding | ||
import org.apache.parquet.hadoop.ParquetFileReader | ||
import org.apache.parquet.hadoop.metadata.ColumnChunkMetaData | ||
import org.scalatest.BeforeAndAfterAll | ||
import org.scalatest.flatspec.AnyFlatSpec | ||
import org.scalatest.matchers.should.Matchers | ||
|
||
import java.nio.file.{Files, Path} | ||
import scala.jdk.CollectionConverters._ | ||
|
||
class TestWriterUtils extends AnyFlatSpec with Matchers with BeforeAndAfterAll { | ||
private lazy val testDir = Files.createTempDirectory("scio-parquet-writer-utils-test-").toFile | ||
|
||
override protected def afterAll(): Unit = FileUtils.deleteDirectory(testDir) | ||
|
||
"WriterUtils" should "set configuration values correctly" in { | ||
val path = testDir.toPath.resolve("avro-files") | ||
|
||
val sc = ScioContext() | ||
sc.parallelize(1 to 10) | ||
.map { r => | ||
Account | ||
.newBuilder() | ||
.setId(r) | ||
.setType("checking") | ||
.setName(r.toString) | ||
.setAmount(r.doubleValue) | ||
.setAccountStatus(AccountStatus.Active) | ||
.build() | ||
} | ||
.saveAsParquetAvroFile( // WriterUtils has too many protected classes to invoke directly; test via Scio write | ||
path.toFile.getAbsolutePath, | ||
numShards = 1, | ||
conf = ParquetConfiguration.of( | ||
"parquet.enable.dictionary" -> false, | ||
"parquet.enable.dictionary#account_status" -> true, | ||
"parquet.bloom.filter.enabled" -> true, | ||
"parquet.bloom.filter.enabled#id" -> false | ||
) | ||
) | ||
|
||
sc.run() | ||
|
||
val columnEncodings = getColumnEncodings(path.resolve("part-00000-of-00001.parquet")) | ||
|
||
assertColumn( | ||
columnEncodings(0), | ||
"id", | ||
hasBloomFilter = false, | ||
Seq(Encoding.BIT_PACKED, Encoding.PLAIN) | ||
) | ||
assertColumn( | ||
columnEncodings(1), | ||
"type", | ||
hasBloomFilter = true, | ||
Seq(Encoding.BIT_PACKED, Encoding.PLAIN) | ||
) | ||
assertColumn( | ||
columnEncodings(2), | ||
"name", | ||
hasBloomFilter = true, | ||
Seq( | ||
Encoding.BIT_PACKED, | ||
Encoding.RLE, // RLE encoding is used for optional fields | ||
Encoding.PLAIN | ||
) | ||
) | ||
assertColumn( | ||
columnEncodings(3), | ||
"amount", | ||
hasBloomFilter = true, | ||
Seq(Encoding.BIT_PACKED, Encoding.PLAIN) | ||
) | ||
assertColumn( | ||
columnEncodings(4), | ||
"account_status", | ||
hasBloomFilter = true, | ||
Seq(Encoding.BIT_PACKED, Encoding.RLE, Encoding.PLAIN_DICTIONARY) | ||
) | ||
} | ||
|
||
private def getColumnEncodings(path: Path): List[ColumnChunkMetaData] = { | ||
val options = HadoopReadOptions.builder(ParquetConfiguration.empty()).build | ||
val r = ParquetFileReader.open(BeamInputFile.of(path.toFile.getAbsolutePath), options) | ||
assert(r.getRowGroups.size() == 1) | ||
|
||
val columns = r.getRowGroups.get(0).getColumns.asScala.toList | ||
r.close() | ||
columns | ||
} | ||
|
||
private def assertColumn( | ||
column: ColumnChunkMetaData, | ||
name: String, | ||
hasBloomFilter: Boolean, | ||
encodings: Iterable[Encoding] | ||
): Unit = { | ||
column.getPath.asScala should contain only name | ||
column.getEncodings.asScala should contain theSameElementsAs encodings | ||
if (hasBloomFilter) { | ||
column.getBloomFilterOffset should be > -1L | ||
} else { | ||
column.getBloomFilterOffset shouldBe -1L | ||
} | ||
} | ||
} |
This file contains bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
86 changes: 86 additions & 0 deletions
86
scio-smb/src/main/java/org/apache/beam/sdk/extensions/smb/ParquetUtils.java
This file contains bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
Original file line number | Diff line number | Diff line change |
---|---|---|
@@ -0,0 +1,86 @@ | ||
/* | ||
* Copyright 2023 Spotify AB. | ||
* | ||
* Licensed under the Apache License, Version 2.0 (the "License"); | ||
* you may not use this file except in compliance with the License. | ||
* You may obtain a copy of the License at | ||
* | ||
* http://www.apache.org/licenses/LICENSE-2.0 | ||
* | ||
* Unless required by applicable law or agreed to in writing, | ||
* software distributed under the License is distributed on an | ||
* "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY | ||
* KIND, either express or implied. See the License for the | ||
* specific language governing permissions and limitations | ||
* under the License. | ||
*/ | ||
|
||
package org.apache.beam.sdk.extensions.smb; | ||
|
||
import java.io.IOException; | ||
import java.util.Map; | ||
import java.util.function.Function; | ||
import java.util.stream.Collectors; | ||
import org.apache.hadoop.conf.Configuration; | ||
import org.apache.parquet.column.ParquetProperties; | ||
import org.apache.parquet.hadoop.ParquetOutputFormat; | ||
import org.apache.parquet.hadoop.ParquetWriter; | ||
import org.apache.parquet.hadoop.metadata.CompressionCodecName; | ||
|
||
class ParquetUtils { | ||
static <T, SELF extends ParquetWriter.Builder<T, SELF>> ParquetWriter<T> buildWriter( | ||
ParquetWriter.Builder<T, SELF> builder, Configuration conf, CompressionCodecName compression) | ||
throws IOException { | ||
// https://github.com/apache/parquet-mr/tree/master/parquet-hadoop#class-parquetoutputformat | ||
long rowGroupSize = | ||
conf.getLong(ParquetOutputFormat.BLOCK_SIZE, ParquetWriter.DEFAULT_BLOCK_SIZE); | ||
|
||
for (Map.Entry<String, Boolean> entry : | ||
getColumnarConfig(conf, ParquetOutputFormat.BLOOM_FILTER_ENABLED, Boolean::parseBoolean) | ||
.entrySet()) { | ||
builder = builder.withBloomFilterEnabled(entry.getKey(), entry.getValue()); | ||
} | ||
|
||
for (Map.Entry<String, Boolean> entry : | ||
getColumnarConfig(conf, ParquetOutputFormat.ENABLE_DICTIONARY, Boolean::parseBoolean) | ||
.entrySet()) { | ||
builder = builder.withDictionaryEncoding(entry.getKey(), entry.getValue()); | ||
} | ||
|
||
for (Map.Entry<String, Long> entry : | ||
getColumnarConfig(conf, ParquetOutputFormat.BLOOM_FILTER_EXPECTED_NDV, Long::parseLong) | ||
.entrySet()) { | ||
builder = builder.withBloomFilterNDV(entry.getKey(), entry.getValue()); | ||
} | ||
|
||
return builder | ||
.withConf(conf) | ||
.withCompressionCodec(compression) | ||
.withPageSize(ParquetOutputFormat.getPageSize(conf)) | ||
.withPageRowCountLimit( | ||
conf.getInt( | ||
ParquetOutputFormat.PAGE_ROW_COUNT_LIMIT, | ||
ParquetProperties.DEFAULT_PAGE_ROW_COUNT_LIMIT)) | ||
.withPageWriteChecksumEnabled(ParquetOutputFormat.getPageWriteChecksumEnabled(conf)) | ||
.withWriterVersion(ParquetOutputFormat.getWriterVersion(conf)) | ||
.withBloomFilterEnabled(ParquetOutputFormat.getBloomFilterEnabled(conf)) | ||
.withDictionaryEncoding(ParquetOutputFormat.getEnableDictionary(conf)) | ||
.withDictionaryPageSize(ParquetOutputFormat.getDictionaryPageSize(conf)) | ||
.withMaxRowCountForPageSizeCheck(ParquetOutputFormat.getMaxRowCountForPageSizeCheck(conf)) | ||
.withMinRowCountForPageSizeCheck(ParquetOutputFormat.getMinRowCountForPageSizeCheck(conf)) | ||
.withValidation(ParquetOutputFormat.getValidation(conf)) | ||
.withRowGroupSize(rowGroupSize) | ||
.withRowGroupSize(rowGroupSize) | ||
.build(); | ||
} | ||
|
||
private static <T> Map<String, T> getColumnarConfig( | ||
Configuration conf, String key, Function<String, T> toT) { | ||
final String keyPrefix = key + "#"; | ||
return conf.getPropsWithPrefix(keyPrefix).entrySet().stream() | ||
.collect( | ||
Collectors.toMap( | ||
entry -> entry.getKey().replaceFirst(keyPrefix, ""), | ||
entry -> toT.apply(entry.getValue()))); | ||
} | ||
} |
This file contains bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters