This is an automated email from the ASF dual-hosted git repository.

gangwu pushed a commit to branch master
in repository https://gitbox.apache.org/repos/asf/parquet-java.git


The following commit(s) were added to refs/heads/master by this push:
     new 3b5fb4b15 GH-2988: Supports disabling statistics for specific columns 
(#2989)
3b5fb4b15 is described below

commit 3b5fb4b15e25853dd14c68a8fd7e60c559dcf32c
Author: Xianyang Liu <[email protected]>
AuthorDate: Wed Aug 28 22:59:20 2024 +0800

    GH-2988: Supports disabling statistics for specific columns (#2989)
---
 .../apache/parquet/column/ParquetProperties.java   |  22 ++++
 .../parquet/column/impl/ColumnValueCollector.java  |   6 +-
 .../parquet/column/statistics/NoopStatistics.java  | 130 +++++++++++++++++++++
 .../parquet/column/statistics/Statistics.java      |   9 ++
 .../parquet/column/statistics/TestStatistics.java  |  27 +++++
 .../org/apache/parquet/hadoop/ParquetWriter.java   |  12 ++
 .../parquet/statistics/DataGenerationContext.java  |  43 ++++---
 .../apache/parquet/statistics/TestStatistics.java  | 122 ++++++++++++++++++-
 8 files changed, 353 insertions(+), 18 deletions(-)

diff --git 
a/parquet-column/src/main/java/org/apache/parquet/column/ParquetProperties.java 
b/parquet-column/src/main/java/org/apache/parquet/column/ParquetProperties.java
index 0645b2b53..a9f80406a 100644
--- 
a/parquet-column/src/main/java/org/apache/parquet/column/ParquetProperties.java
+++ 
b/parquet-column/src/main/java/org/apache/parquet/column/ParquetProperties.java
@@ -63,6 +63,7 @@ public class ParquetProperties {
   public static final double DEFAULT_BLOOM_FILTER_FPP = 0.01;
   public static final boolean DEFAULT_ADAPTIVE_BLOOM_FILTER_ENABLED = false;
   public static final int DEFAULT_BLOOM_FILTER_CANDIDATES_NUMBER = 5;
+  public static final boolean DEFAULT_STATISTICS_ENABLED = true;
 
   public static final boolean DEFAULT_PAGE_WRITE_CHECKSUM_ENABLED = true;
 
@@ -122,6 +123,7 @@ public class ParquetProperties {
   private final boolean pageWriteChecksumEnabled;
   private final ColumnProperty<ByteStreamSplitMode> byteStreamSplitEnabled;
   private final Map<String, String> extraMetaData;
+  private final ColumnProperty<Boolean> statistics;
 
   private ParquetProperties(Builder builder) {
     this.pageSizeThreshold = builder.pageSize;
@@ -149,6 +151,7 @@ public class ParquetProperties {
     this.pageWriteChecksumEnabled = builder.pageWriteChecksumEnabled;
     this.byteStreamSplitEnabled = builder.byteStreamSplitEnabled.build();
     this.extraMetaData = builder.extraMetaData;
+    this.statistics = builder.statistics.build();
   }
 
   public static Builder builder() {
@@ -330,6 +333,10 @@ public class ParquetProperties {
     return extraMetaData;
   }
 
+  public boolean getStatisticsEnabled(ColumnDescriptor column) {
+    return statistics.getValue(column);
+  }
+
   @Override
   public String toString() {
     return "Parquet page size to " + getPageSizeThreshold() + '\n'
@@ -372,6 +379,7 @@ public class ParquetProperties {
     private boolean pageWriteChecksumEnabled = 
DEFAULT_PAGE_WRITE_CHECKSUM_ENABLED;
     private final ColumnProperty.Builder<ByteStreamSplitMode> 
byteStreamSplitEnabled;
     private Map<String, String> extraMetaData = new HashMap<>();
+    private final ColumnProperty.Builder<Boolean> statistics;
 
     private Builder() {
       enableDict = 
ColumnProperty.<Boolean>builder().withDefaultValue(DEFAULT_IS_DICTIONARY_ENABLED);
@@ -387,6 +395,7 @@ public class ParquetProperties {
           
ColumnProperty.<Boolean>builder().withDefaultValue(DEFAULT_ADAPTIVE_BLOOM_FILTER_ENABLED);
       numBloomFilterCandidates =
           
ColumnProperty.<Integer>builder().withDefaultValue(DEFAULT_BLOOM_FILTER_CANDIDATES_NUMBER);
+      statistics = 
ColumnProperty.<Boolean>builder().withDefaultValue(DEFAULT_STATISTICS_ENABLED);
     }
 
     private Builder(ParquetProperties toCopy) {
@@ -409,6 +418,7 @@ public class ParquetProperties {
       this.maxBloomFilterBytes = toCopy.maxBloomFilterBytes;
       this.byteStreamSplitEnabled = 
ColumnProperty.builder(toCopy.byteStreamSplitEnabled);
       this.extraMetaData = toCopy.extraMetaData;
+      this.statistics = ColumnProperty.builder(toCopy.statistics);
     }
 
     /**
@@ -657,6 +667,18 @@ public class ParquetProperties {
       return this;
     }
 
+    /**
+     * Enable or disable the statistics for given column. All column 
statistics are enabled by default.
+     *
+     * @param columnPath the given column
+     * @param enabled enable or disable
+     * @return this builder for method chaining
+     */
+    public Builder withStatisticsEnabled(String columnPath, boolean enabled) {
+      this.statistics.withValue(columnPath, enabled);
+      return this;
+    }
+
     public ParquetProperties build() {
       ParquetProperties properties = new ParquetProperties(this);
       // we pass a constructed but uninitialized factory to ParquetProperties 
above as currently
diff --git 
a/parquet-column/src/main/java/org/apache/parquet/column/impl/ColumnValueCollector.java
 
b/parquet-column/src/main/java/org/apache/parquet/column/impl/ColumnValueCollector.java
index bb3135a18..164287df2 100644
--- 
a/parquet-column/src/main/java/org/apache/parquet/column/impl/ColumnValueCollector.java
+++ 
b/parquet-column/src/main/java/org/apache/parquet/column/impl/ColumnValueCollector.java
@@ -36,6 +36,7 @@ import org.apache.parquet.io.api.Binary;
 class ColumnValueCollector {
 
   private final ColumnDescriptor path;
+  private final boolean statisticsEnabled;
   private BloomFilterWriter bloomFilterWriter;
   private BloomFilter bloomFilter;
   private Statistics<?> statistics;
@@ -43,12 +44,15 @@ class ColumnValueCollector {
 
   ColumnValueCollector(ColumnDescriptor path, BloomFilterWriter 
bloomFilterWriter, ParquetProperties props) {
     this.path = path;
+    this.statisticsEnabled = props.getStatisticsEnabled(path);
     resetPageStatistics();
     initBloomFilter(bloomFilterWriter, props);
   }
 
   void resetPageStatistics() {
-    this.statistics = Statistics.createStats(path.getPrimitiveType());
+    this.statistics = statisticsEnabled
+        ? Statistics.createStats(path.getPrimitiveType())
+        : Statistics.noopStats(path.getPrimitiveType());
     this.sizeStatisticsBuilder = SizeStatistics.newBuilder(
         path.getPrimitiveType(), path.getMaxRepetitionLevel(), 
path.getMaxDefinitionLevel());
   }
diff --git 
a/parquet-column/src/main/java/org/apache/parquet/column/statistics/NoopStatistics.java
 
b/parquet-column/src/main/java/org/apache/parquet/column/statistics/NoopStatistics.java
new file mode 100644
index 000000000..2834568f5
--- /dev/null
+++ 
b/parquet-column/src/main/java/org/apache/parquet/column/statistics/NoopStatistics.java
@@ -0,0 +1,130 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you 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.parquet.column.statistics;
+
+import org.apache.parquet.io.api.Binary;
+import org.apache.parquet.schema.PrimitiveType;
+
+/**
+ * A noop statistics which always return empty.
+ */
+class NoopStatistics<T extends Comparable<T>> extends Statistics<T> {
+
+  NoopStatistics(PrimitiveType type) {
+    super(type);
+  }
+
+  @Override
+  public void updateStats(int value) {}
+
+  @Override
+  public void updateStats(long value) {}
+
+  @Override
+  public void updateStats(float value) {}
+
+  @Override
+  public void updateStats(double value) {}
+
+  @Override
+  public void updateStats(boolean value) {}
+
+  @Override
+  public void updateStats(Binary value) {}
+
+  @Override
+  public boolean equals(Object other) {
+    if (other == this) return true;
+    if (!(other instanceof Statistics)) return false;
+    Statistics stats = (Statistics) other;
+    return type().equals(stats.type());
+  }
+
+  @Override
+  public int hashCode() {
+    return 31 * type().hashCode();
+  }
+
+  @Override
+  protected void mergeStatisticsMinMax(Statistics stats) {}
+
+  @Override
+  public void setMinMaxFromBytes(byte[] minBytes, byte[] maxBytes) {}
+
+  @Override
+  public T genericGetMin() {
+    throw new UnsupportedOperationException(
+        "genericGetMin is not supported by " + getClass().getName());
+  }
+
+  @Override
+  public T genericGetMax() {
+    throw new UnsupportedOperationException(
+        "genericGetMax is not supported by " + getClass().getName());
+  }
+
+  @Override
+  public byte[] getMaxBytes() {
+    throw new UnsupportedOperationException(
+        "getMaxBytes is not supported by " + getClass().getName());
+  }
+
+  @Override
+  public byte[] getMinBytes() {
+    throw new UnsupportedOperationException(
+        "getMinBytes is not supported by " + getClass().getName());
+  }
+
+  @Override
+  String stringify(T value) {
+    throw new UnsupportedOperationException(
+        "stringify is not supported by " + getClass().getName());
+  }
+
+  @Override
+  public boolean isSmallerThan(long size) {
+    throw new UnsupportedOperationException(
+        "isSmallerThan is not supported by " + getClass().getName());
+  }
+
+  @Override
+  public long getNumNulls() {
+    return -1;
+  }
+
+  @Override
+  public boolean isEmpty() {
+    return true;
+  }
+
+  @Override
+  public boolean hasNonNullValue() {
+    return false;
+  }
+
+  @Override
+  public boolean isNumNullsSet() {
+    return false;
+  }
+
+  @Override
+  public Statistics<T> copy() {
+    return new NoopStatistics<>(this.type());
+  }
+}
diff --git 
a/parquet-column/src/main/java/org/apache/parquet/column/statistics/Statistics.java
 
b/parquet-column/src/main/java/org/apache/parquet/column/statistics/Statistics.java
index 83070d49f..206ddadad 100644
--- 
a/parquet-column/src/main/java/org/apache/parquet/column/statistics/Statistics.java
+++ 
b/parquet-column/src/main/java/org/apache/parquet/column/statistics/Statistics.java
@@ -252,6 +252,15 @@ public abstract class Statistics<T extends Comparable<T>> {
     }
   }
 
+  /**
+   * Creates a noop {@code NoopStatistics} statistics instance. This is only 
used when the user disables statistics for the specified column.
+   * @param type type of the column
+   * @return a noop statistics
+   */
+  public static Statistics<?> noopStats(Type type) {
+    return new NoopStatistics<>(type.asPrimitiveType());
+  }
+
   /**
    * Returns a builder to create new statistics object. Used to read the 
statistics from the parquet file.
    *
diff --git 
a/parquet-column/src/test/java/org/apache/parquet/column/statistics/TestStatistics.java
 
b/parquet-column/src/test/java/org/apache/parquet/column/statistics/TestStatistics.java
index d60ad7299..dec244f62 100644
--- 
a/parquet-column/src/test/java/org/apache/parquet/column/statistics/TestStatistics.java
+++ 
b/parquet-column/src/test/java/org/apache/parquet/column/statistics/TestStatistics.java
@@ -30,9 +30,11 @@ import static 
org.apache.parquet.schema.PrimitiveType.PrimitiveTypeName.FLOAT;
 import static org.apache.parquet.schema.PrimitiveType.PrimitiveTypeName.INT32;
 import static org.apache.parquet.schema.PrimitiveType.PrimitiveTypeName.INT64;
 import static org.apache.parquet.schema.PrimitiveType.PrimitiveTypeName.INT96;
+import static org.apache.parquet.schema.Type.Repetition.REQUIRED;
 import static org.junit.Assert.assertArrayEquals;
 import static org.junit.Assert.assertEquals;
 import static org.junit.Assert.assertFalse;
+import static org.junit.Assert.assertThrows;
 import static org.junit.Assert.assertTrue;
 
 import java.nio.ByteBuffer;
@@ -900,4 +902,29 @@ public class TestStatistics {
     assertEquals(0, Double.compare(-0.0, (Double) stats.genericGetMin()));
     assertEquals(0, Double.compare(0.0, (Double) stats.genericGetMax()));
   }
+
+  @Test
+  public void testNoopStatistics() {
+    // Test basic max/min
+    integerArray = new int[] {1, 3, 14, 54, 66, 8, 0, 23, 54};
+    Statistics<?> stats = Statistics.noopStats(new PrimitiveType(REQUIRED, 
INT32, "int32"));
+    assertTrue(stats.isEmpty());
+
+    for (int i : integerArray) {
+      stats.updateStats(i);
+    }
+
+    assertEquals(stats.getNumNulls(), -1);
+    assertFalse(stats.hasNonNullValue());
+    assertFalse(stats.isNumNullsSet());
+    assertTrue(stats.isEmpty());
+
+    assertThrows(UnsupportedOperationException.class, stats::genericGetMax);
+    assertThrows(UnsupportedOperationException.class, stats::genericGetMin);
+    assertThrows(UnsupportedOperationException.class, stats::getMaxBytes);
+    assertThrows(UnsupportedOperationException.class, stats::getMinBytes);
+    assertThrows(UnsupportedOperationException.class, stats::maxAsString);
+    assertThrows(UnsupportedOperationException.class, stats::minAsString);
+    assertThrows(UnsupportedOperationException.class, () -> 
stats.isSmallerThan(0));
+  }
 }
diff --git 
a/parquet-hadoop/src/main/java/org/apache/parquet/hadoop/ParquetWriter.java 
b/parquet-hadoop/src/main/java/org/apache/parquet/hadoop/ParquetWriter.java
index ac868df8b..60867aa93 100644
--- a/parquet-hadoop/src/main/java/org/apache/parquet/hadoop/ParquetWriter.java
+++ b/parquet-hadoop/src/main/java/org/apache/parquet/hadoop/ParquetWriter.java
@@ -897,6 +897,18 @@ public class ParquetWriter<T> implements Closeable {
       return self();
     }
 
+    /**
+     * Sets the statistics enabled/disabled for the specified column. All 
column statistics are enabled by default.
+     *
+     * @param columnPath the path of the column (dot-string)
+     * @param enabled    whether to write calculate statistics for the column
+     * @return this builder for method chaining
+     */
+    public SELF withStatisticsEnabled(String columnPath, boolean enabled) {
+      encodingPropsBuilder.withStatisticsEnabled(columnPath, enabled);
+      return self();
+    }
+
     /**
      * Build a {@link ParquetWriter} with the accumulated configuration.
      *
diff --git 
a/parquet-hadoop/src/test/java/org/apache/parquet/statistics/DataGenerationContext.java
 
b/parquet-hadoop/src/test/java/org/apache/parquet/statistics/DataGenerationContext.java
index 6bb64e6ee..4e6b2a1b6 100644
--- 
a/parquet-hadoop/src/test/java/org/apache/parquet/statistics/DataGenerationContext.java
+++ 
b/parquet-hadoop/src/test/java/org/apache/parquet/statistics/DataGenerationContext.java
@@ -18,14 +18,16 @@
  */
 package org.apache.parquet.statistics;
 
+import com.google.common.collect.ImmutableSet;
 import java.io.File;
 import java.io.IOException;
+import java.util.Set;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.fs.Path;
 import org.apache.parquet.column.ParquetProperties;
 import org.apache.parquet.example.data.Group;
 import org.apache.parquet.hadoop.ParquetWriter;
-import org.apache.parquet.hadoop.example.GroupWriteSupport;
+import org.apache.parquet.hadoop.example.ExampleParquetWriter;
 import org.apache.parquet.hadoop.metadata.CompressionCodecName;
 import org.apache.parquet.schema.MessageType;
 
@@ -39,6 +41,7 @@ public class DataGenerationContext {
     protected final boolean enableDictionary;
     protected final boolean enableValidation;
     protected final ParquetProperties.WriterVersion version;
+    protected final Set<String> disableColumnStatistics;
 
     public WriteContext(
         File path,
@@ -49,6 +52,19 @@ public class DataGenerationContext {
         boolean enableValidation,
         ParquetProperties.WriterVersion version)
         throws IOException {
+      this(path, schema, blockSize, pageSize, enableDictionary, 
enableValidation, version, ImmutableSet.of());
+    }
+
+    public WriteContext(
+        File path,
+        MessageType schema,
+        int blockSize,
+        int pageSize,
+        boolean enableDictionary,
+        boolean enableValidation,
+        ParquetProperties.WriterVersion version,
+        Set<String> disableColumnStatistics)
+        throws IOException {
       this.path = path;
       this.fsPath = new Path(path.toString());
       this.schema = schema;
@@ -57,6 +73,7 @@ public class DataGenerationContext {
       this.enableDictionary = enableDictionary;
       this.enableValidation = enableValidation;
       this.version = version;
+      this.disableColumnStatistics = disableColumnStatistics;
     }
 
     public abstract void write(ParquetWriter<Group> writer) throws IOException;
@@ -67,8 +84,6 @@ public class DataGenerationContext {
   public static void writeAndTest(WriteContext context) throws IOException {
     // Create the configuration, and then apply the schema to our 
configuration.
     Configuration configuration = new Configuration();
-    GroupWriteSupport.setSchema(context.schema, configuration);
-    GroupWriteSupport groupWriteSupport = new GroupWriteSupport();
 
     // Create the writer properties
     final int blockSize = context.blockSize;
@@ -79,17 +94,17 @@ public class DataGenerationContext {
     ParquetProperties.WriterVersion writerVersion = context.version;
     CompressionCodecName codec = CompressionCodecName.UNCOMPRESSED;
 
-    try (ParquetWriter<Group> writer = new ParquetWriter<Group>(
-        context.fsPath,
-        groupWriteSupport,
-        codec,
-        blockSize,
-        pageSize,
-        dictionaryPageSize,
-        enableDictionary,
-        enableValidation,
-        writerVersion,
-        configuration)) {
+    try (ParquetWriter<Group> writer = 
ExampleParquetWriter.builder(context.fsPath)
+        .withType(context.schema)
+        .withCompressionCodec(codec)
+        .withRowGroupSize(blockSize)
+        .withPageSize(pageSize)
+        .withDictionaryPageSize(dictionaryPageSize)
+        .withDictionaryEncoding(enableDictionary)
+        .withValidation(enableValidation)
+        .withWriterVersion(writerVersion)
+        .withConf(configuration)
+        .build()) {
       context.write(writer);
     }
 
diff --git 
a/parquet-hadoop/src/test/java/org/apache/parquet/statistics/TestStatistics.java
 
b/parquet-hadoop/src/test/java/org/apache/parquet/statistics/TestStatistics.java
index 0915b4644..c3aecb658 100644
--- 
a/parquet-hadoop/src/test/java/org/apache/parquet/statistics/TestStatistics.java
+++ 
b/parquet-hadoop/src/test/java/org/apache/parquet/statistics/TestStatistics.java
@@ -31,6 +31,7 @@ import static 
org.apache.parquet.schema.Type.Repetition.REQUIRED;
 import static org.junit.Assert.assertEquals;
 import static org.junit.Assert.assertTrue;
 
+import com.google.common.collect.ImmutableSet;
 import java.io.File;
 import java.io.IOException;
 import java.math.BigInteger;
@@ -38,6 +39,7 @@ import java.util.Arrays;
 import java.util.Comparator;
 import java.util.List;
 import java.util.Random;
+import java.util.Set;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.parquet.column.ColumnDescriptor;
 import org.apache.parquet.column.ParquetProperties;
@@ -231,17 +233,28 @@ public class TestStatistics {
 
   public static class PageStatsValidator {
     public void validate(MessageType schema, PageReadStore store) {
+      validate(schema, store, null);
+    }
+
+    public void validate(MessageType schema, PageReadStore store, Set<String> 
statisticsDisabledSet) {
       for (ColumnDescriptor desc : schema.getColumns()) {
+        boolean statisticsDisabled = false;
+        if (statisticsDisabledSet != null) {
+          String dotPath = String.join(".", desc.getPath());
+          statisticsDisabled = statisticsDisabledSet.contains(dotPath);
+        }
+
         PageReader reader = store.getPageReader(desc);
         DictionaryPage dict = reader.readDictionaryPage();
         DataPage page;
         while ((page = reader.readPage()) != null) {
-          validateStatsForPage(page, dict, desc);
+          validateStatsForPage(page, dict, desc, statisticsDisabled);
         }
       }
     }
 
-    private void validateStatsForPage(DataPage page, DictionaryPage dict, 
ColumnDescriptor desc) {
+    private void validateStatsForPage(
+        DataPage page, DictionaryPage dict, ColumnDescriptor desc, boolean 
statisticsDisabled) {
       SingletonPageReader reader = new SingletonPageReader(dict, page);
       PrimitiveConverter converter = getValidatingConverter(page, 
desc.getType());
       Statistics<?> stats = getStatisticsFromPageHeader(page);
@@ -251,6 +264,10 @@ public class TestStatistics {
           desc.getPrimitiveType().comparator().getClass(),
           stats.comparator().getClass());
 
+      if (statisticsDisabled) {
+        Assert.assertTrue(stats.isEmpty());
+      }
+
       if (stats.isEmpty()) {
         // stats are empty if num nulls = 0 and there are no non-null values
         // this happens if stats are not written (e.g., when stats are too big)
@@ -288,6 +305,18 @@ public class TestStatistics {
         boolean enableDictionary,
         ParquetProperties.WriterVersion version)
         throws IOException {
+      this(seed, path, blockSize, pageSize, enableDictionary, version, 
ImmutableSet.of());
+    }
+
+    public DataContext(
+        long seed,
+        File path,
+        int blockSize,
+        int pageSize,
+        boolean enableDictionary,
+        ParquetProperties.WriterVersion version,
+        Set<String> disableColumnStatistics)
+        throws IOException {
       super(path, buildSchema(seed), blockSize, pageSize, enableDictionary, 
true, version);
 
       this.random = new Random(seed);
@@ -472,7 +501,8 @@ public class TestStatistics {
 
         PageReadStore pageReadStore;
         while ((pageReadStore = reader.readNextRowGroup()) != null) {
-          validator.validate(metadata.getFileMetaData().getSchema(), 
pageReadStore);
+          validator.validate(
+              metadata.getFileMetaData().getSchema(), pageReadStore, 
this.disableColumnStatistics);
         }
       }
     }
@@ -527,4 +557,90 @@ public class TestStatistics {
       DataGenerationContext.writeAndTest(test);
     }
   }
+
+  @Test
+  public void testDisableStatistics() throws IOException {
+    File file = folder.newFile("test_file.parquet");
+    file.delete();
+
+    LOG.info(String.format("RANDOM SEED: %s", RANDOM_SEED));
+
+    Random random = new Random(RANDOM_SEED);
+
+    int blockSize = (random.nextInt(54) + 10) * MEGABYTE;
+    int pageSize = (random.nextInt(10) + 1) * MEGABYTE;
+
+    List<DataContext> contexts = Arrays.asList(
+        new DataContext(
+            random.nextLong(),
+            file,
+            blockSize,
+            pageSize,
+            false,
+            ParquetProperties.WriterVersion.PARQUET_1_0,
+            ImmutableSet.of(
+                "i32",
+                "i64",
+                "i96",
+                "sngl",
+                "dbl",
+                "strings",
+                "binary",
+                "fixed-binary",
+                "unconstrained-i32")),
+        new DataContext(
+            random.nextLong(),
+            file,
+            blockSize,
+            pageSize,
+            true,
+            ParquetProperties.WriterVersion.PARQUET_1_0,
+            ImmutableSet.of(
+                "unconstrained-i64",
+                "unconstrained-sngl",
+                "unconstrained-dbl",
+                "int8",
+                "uint8",
+                "int16",
+                "uint16",
+                "int32",
+                "uint32")),
+        new DataContext(
+            random.nextLong(),
+            file,
+            blockSize,
+            pageSize,
+            false,
+            ParquetProperties.WriterVersion.PARQUET_2_0,
+            ImmutableSet.of(
+                "int64",
+                "uint64",
+                "decimal-int32",
+                "decimal-int64",
+                "decimal-fixed",
+                "decimal-binary",
+                "utf8",
+                "enum",
+                "json")),
+        new DataContext(
+            random.nextLong(),
+            file,
+            blockSize,
+            pageSize,
+            true,
+            ParquetProperties.WriterVersion.PARQUET_2_0,
+            ImmutableSet.of(
+                "bson",
+                "date",
+                "time-millis",
+                "time-micros",
+                "timestamp-millis",
+                "timestamp-micros",
+                "interval",
+                "float16")));
+
+    for (DataContext test : contexts) {
+      DataGenerationContext.writeAndTest(test);
+    }
+  }
 }

Reply via email to