|
| 1 | +/* |
| 2 | + * Licensed to the Apache Software Foundation (ASF) under one |
| 3 | + * or more contributor license agreements. See the NOTICE file |
| 4 | + * distributed with this work for additional information |
| 5 | + * regarding copyright ownership. The ASF licenses this file |
| 6 | + * to you under the Apache License, Version 2.0 (the |
| 7 | + * "License"); you may not use this file except in compliance |
| 8 | + * with the License. You may obtain a copy of the License at |
| 9 | + * |
| 10 | + * http://www.apache.org/licenses/LICENSE-2.0 |
| 11 | + * |
| 12 | + * Unless required by applicable law or agreed to in writing, |
| 13 | + * software distributed under the License is distributed on an |
| 14 | + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY |
| 15 | + * KIND, either express or implied. See the License for the |
| 16 | + * specific language governing permissions and limitations |
| 17 | + * under the License. |
| 18 | + */ |
| 19 | +package org.apache.parquet.benchmarks; |
| 20 | + |
| 21 | +import java.io.File; |
| 22 | +import java.io.IOException; |
| 23 | +import java.util.concurrent.TimeUnit; |
| 24 | +import org.apache.parquet.column.ParquetProperties.WriterVersion; |
| 25 | +import org.apache.parquet.example.data.Group; |
| 26 | +import org.apache.parquet.hadoop.ParquetFileWriter; |
| 27 | +import org.apache.parquet.hadoop.ParquetReader; |
| 28 | +import org.apache.parquet.hadoop.ParquetWriter; |
| 29 | +import org.apache.parquet.hadoop.api.ReadSupport; |
| 30 | +import org.apache.parquet.hadoop.example.ExampleParquetWriter; |
| 31 | +import org.apache.parquet.hadoop.example.GroupReadSupport; |
| 32 | +import org.apache.parquet.hadoop.metadata.CompressionCodecName; |
| 33 | +import org.apache.parquet.io.InputFile; |
| 34 | +import org.apache.parquet.io.LocalInputFile; |
| 35 | +import org.apache.parquet.io.LocalOutputFile; |
| 36 | +import org.openjdk.jmh.annotations.Benchmark; |
| 37 | +import org.openjdk.jmh.annotations.BenchmarkMode; |
| 38 | +import org.openjdk.jmh.annotations.Fork; |
| 39 | +import org.openjdk.jmh.annotations.Level; |
| 40 | +import org.openjdk.jmh.annotations.Measurement; |
| 41 | +import org.openjdk.jmh.annotations.Mode; |
| 42 | +import org.openjdk.jmh.annotations.OutputTimeUnit; |
| 43 | +import org.openjdk.jmh.annotations.Param; |
| 44 | +import org.openjdk.jmh.annotations.Scope; |
| 45 | +import org.openjdk.jmh.annotations.Setup; |
| 46 | +import org.openjdk.jmh.annotations.State; |
| 47 | +import org.openjdk.jmh.annotations.TearDown; |
| 48 | +import org.openjdk.jmh.annotations.Warmup; |
| 49 | +import org.openjdk.jmh.infra.Blackhole; |
| 50 | + |
| 51 | +/** |
| 52 | + * File-level read benchmarks measuring end-to-end Parquet read throughput through the |
| 53 | + * example {@link Group} API. A temporary file is generated once during setup from |
| 54 | + * pre-generated rows using {@link LocalOutputFile}, then read repeatedly during the |
| 55 | + * benchmark. |
| 56 | + * |
| 57 | + * <p>Parameterized across compression codec and writer version. The footer parse |
| 58 | + * (via {@link LocalInputFile} open) is included in the timed section so the result |
| 59 | + * reflects the full open-and-read cost a typical caller would observe. |
| 60 | + * |
| 61 | + * <p>{@link Mode#SingleShotTime} is used because each invocation does enough work |
| 62 | + * (a full read of {@value TestDataFactory#DEFAULT_ROW_COUNT} rows) that JIT |
| 63 | + * amortization across invocations is unnecessary. Ten measurement iterations |
| 64 | + * provide stable statistics for SS mode. |
| 65 | + */ |
| 66 | +@BenchmarkMode(Mode.SingleShotTime) |
| 67 | +@Fork(1) |
| 68 | +@Warmup(iterations = 5, batchSize = 1) |
| 69 | +@Measurement(iterations = 10, batchSize = 1) |
| 70 | +@OutputTimeUnit(TimeUnit.MILLISECONDS) |
| 71 | +@State(Scope.Benchmark) |
| 72 | +public class FileReadBenchmark { |
| 73 | + |
| 74 | + @Param({"UNCOMPRESSED", "SNAPPY", "ZSTD", "GZIP"}) |
| 75 | + public String codec; |
| 76 | + |
| 77 | + @Param({"PARQUET_1_0", "PARQUET_2_0"}) |
| 78 | + public String writerVersion; |
| 79 | + |
| 80 | + private File tempFile; |
| 81 | + |
| 82 | + @Setup(Level.Trial) |
| 83 | + public void setup() throws IOException { |
| 84 | + tempFile = File.createTempFile("parquet-read-bench-", ".parquet"); |
| 85 | + tempFile.deleteOnExit(); |
| 86 | + tempFile.delete(); // remove so the writer can create it |
| 87 | + |
| 88 | + Group[] rows = TestDataFactory.generateRows( |
| 89 | + TestDataFactory.newGroupFactory(), TestDataFactory.DEFAULT_ROW_COUNT, TestDataFactory.DEFAULT_SEED); |
| 90 | + try (ParquetWriter<Group> writer = ExampleParquetWriter.builder(new LocalOutputFile(tempFile.toPath())) |
| 91 | + .withWriteMode(ParquetFileWriter.Mode.OVERWRITE) |
| 92 | + .withType(TestDataFactory.FILE_BENCHMARK_SCHEMA) |
| 93 | + .withCompressionCodec(CompressionCodecName.valueOf(codec)) |
| 94 | + .withWriterVersion(WriterVersion.valueOf(writerVersion)) |
| 95 | + .withDictionaryEncoding(true) |
| 96 | + .build()) { |
| 97 | + for (Group row : rows) { |
| 98 | + writer.write(row); |
| 99 | + } |
| 100 | + } |
| 101 | + } |
| 102 | + |
| 103 | + @TearDown(Level.Trial) |
| 104 | + public void tearDown() { |
| 105 | + if (tempFile != null && tempFile.exists()) { |
| 106 | + tempFile.delete(); |
| 107 | + } |
| 108 | + } |
| 109 | + |
| 110 | + @Benchmark |
| 111 | + public void readFile(Blackhole bh) throws IOException { |
| 112 | + InputFile inputFile = new LocalInputFile(tempFile.toPath()); |
| 113 | + try (ParquetReader<Group> reader = new ParquetReader.Builder<Group>(inputFile) { |
| 114 | + @Override |
| 115 | + protected ReadSupport<Group> getReadSupport() { |
| 116 | + return new GroupReadSupport(); |
| 117 | + } |
| 118 | + }.build()) { |
| 119 | + Group group; |
| 120 | + while ((group = reader.read()) != null) { |
| 121 | + bh.consume(group); |
| 122 | + } |
| 123 | + } |
| 124 | + } |
| 125 | +} |
0 commit comments