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

aokolnychyi pushed a commit to branch master
in repository https://gitbox.apache.org/repos/asf/iceberg.git


The following commit(s) were added to refs/heads/master by this push:
     new d6ca368  Core: Add DataWriter builders (#2857)
d6ca368 is described below

commit d6ca368eab51e7ec4f1fc546d3ed2d895743efea
Author: Anton Okolnychyi <[email protected]>
AuthorDate: Fri Jul 23 15:03:46 2021 -1000

    Core: Add DataWriter builders (#2857)
---
 .../main/java/org/apache/iceberg/avro/Avro.java    |  93 +++++++++++++++++
 .../apache/iceberg/avro/TestAvroDataWriter.java    | 113 ++++++++++++++++++++
 .../apache/iceberg/data/orc/TestOrcDataWriter.java | 115 ++++++++++++++++++++
 orc/src/main/java/org/apache/iceberg/orc/ORC.java  |  99 ++++++++++++++++++
 .../java/org/apache/iceberg/parquet/Parquet.java   |  93 +++++++++++++++++
 .../iceberg/parquet/TestParquetDataWriter.java     | 116 +++++++++++++++++++++
 6 files changed, 629 insertions(+)

diff --git a/core/src/main/java/org/apache/iceberg/avro/Avro.java 
b/core/src/main/java/org/apache/iceberg/avro/Avro.java
index 28790fa..4f66181 100644
--- a/core/src/main/java/org/apache/iceberg/avro/Avro.java
+++ b/core/src/main/java/org/apache/iceberg/avro/Avro.java
@@ -50,6 +50,7 @@ import org.apache.iceberg.deletes.EqualityDeleteWriter;
 import org.apache.iceberg.deletes.PositionDelete;
 import org.apache.iceberg.deletes.PositionDeleteWriter;
 import org.apache.iceberg.encryption.EncryptionKeyMetadata;
+import org.apache.iceberg.io.DataWriter;
 import org.apache.iceberg.io.DeleteSchemaUtil;
 import org.apache.iceberg.io.FileAppender;
 import org.apache.iceberg.io.InputFile;
@@ -234,6 +235,98 @@ public class Avro {
     }
   }
 
+  public static DataWriteBuilder writeData(OutputFile file) {
+    return new DataWriteBuilder(file);
+  }
+
+  public static class DataWriteBuilder {
+    private final WriteBuilder appenderBuilder;
+    private final String location;
+    private PartitionSpec spec = null;
+    private StructLike partition = null;
+    private EncryptionKeyMetadata keyMetadata = null;
+    private SortOrder sortOrder = null;
+
+    private DataWriteBuilder(OutputFile file) {
+      this.appenderBuilder = write(file);
+      this.location = file.location();
+    }
+
+    public DataWriteBuilder forTable(Table table) {
+      schema(table.schema());
+      withSpec(table.spec());
+      setAll(table.properties());
+      metricsConfig(MetricsConfig.fromProperties(table.properties()));
+      return this;
+    }
+
+    public DataWriteBuilder schema(org.apache.iceberg.Schema newSchema) {
+      appenderBuilder.schema(newSchema);
+      return this;
+    }
+
+    public DataWriteBuilder set(String property, String value) {
+      appenderBuilder.set(property, value);
+      return this;
+    }
+
+    public DataWriteBuilder setAll(Map<String, String> properties) {
+      appenderBuilder.setAll(properties);
+      return this;
+    }
+
+    public DataWriteBuilder meta(String property, String value) {
+      appenderBuilder.meta(property, value);
+      return this;
+    }
+
+    public DataWriteBuilder overwrite() {
+      return overwrite(true);
+    }
+
+    public DataWriteBuilder overwrite(boolean enabled) {
+      appenderBuilder.overwrite(enabled);
+      return this;
+    }
+
+    public DataWriteBuilder metricsConfig(MetricsConfig newMetricsConfig) {
+      appenderBuilder.metricsConfig(newMetricsConfig);
+      return this;
+    }
+
+    public DataWriteBuilder createWriterFunc(Function<Schema, DatumWriter<?>> 
newCreateWriterFunc) {
+      appenderBuilder.createWriterFunc(newCreateWriterFunc);
+      return this;
+    }
+
+    public DataWriteBuilder withSpec(PartitionSpec newSpec) {
+      this.spec = newSpec;
+      return this;
+    }
+
+    public DataWriteBuilder withPartition(StructLike newPartition) {
+      this.partition = newPartition;
+      return this;
+    }
+
+    public DataWriteBuilder withKeyMetadata(EncryptionKeyMetadata metadata) {
+      this.keyMetadata = metadata;
+      return this;
+    }
+
+    public DataWriteBuilder withSortOrder(SortOrder newSortOrder) {
+      this.sortOrder = newSortOrder;
+      return this;
+    }
+
+    public <T> DataWriter<T> build() throws IOException {
+      Preconditions.checkArgument(spec != null, "Cannot create data writer 
without spec");
+
+      FileAppender<T> fileAppender = appenderBuilder.build();
+      return new DataWriter<>(fileAppender, FileFormat.AVRO, location, spec, 
partition, keyMetadata, sortOrder);
+    }
+  }
+
   public static DeleteWriteBuilder writeDeletes(OutputFile file) {
     return new DeleteWriteBuilder(file);
   }
diff --git a/core/src/test/java/org/apache/iceberg/avro/TestAvroDataWriter.java 
b/core/src/test/java/org/apache/iceberg/avro/TestAvroDataWriter.java
new file mode 100644
index 0000000..7680099
--- /dev/null
+++ b/core/src/test/java/org/apache/iceberg/avro/TestAvroDataWriter.java
@@ -0,0 +1,113 @@
+/*
+ * 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.iceberg.avro;
+
+import java.io.IOException;
+import java.util.List;
+import org.apache.iceberg.DataFile;
+import org.apache.iceberg.FileContent;
+import org.apache.iceberg.FileFormat;
+import org.apache.iceberg.Files;
+import org.apache.iceberg.PartitionSpec;
+import org.apache.iceberg.Schema;
+import org.apache.iceberg.SortOrder;
+import org.apache.iceberg.data.GenericRecord;
+import org.apache.iceberg.data.Record;
+import org.apache.iceberg.io.DataWriter;
+import org.apache.iceberg.io.OutputFile;
+import org.apache.iceberg.relocated.com.google.common.collect.ImmutableList;
+import org.apache.iceberg.relocated.com.google.common.collect.ImmutableMap;
+import org.apache.iceberg.relocated.com.google.common.collect.Lists;
+import org.apache.iceberg.types.Types;
+import org.junit.Assert;
+import org.junit.Before;
+import org.junit.Rule;
+import org.junit.Test;
+import org.junit.rules.TemporaryFolder;
+
+public class TestAvroDataWriter {
+  private static final Schema SCHEMA = new Schema(
+      Types.NestedField.required(1, "id", Types.LongType.get()),
+      Types.NestedField.optional(2, "data", Types.StringType.get()));
+
+  private List<Record> records;
+
+  @Rule
+  public TemporaryFolder temp = new TemporaryFolder();
+
+  @Before
+  public void createRecords() {
+    GenericRecord record = GenericRecord.create(SCHEMA);
+
+    ImmutableList.Builder<Record> builder = ImmutableList.builder();
+    builder.add(record.copy(ImmutableMap.of("id", 1L, "data", "a")));
+    builder.add(record.copy(ImmutableMap.of("id", 2L, "data", "b")));
+    builder.add(record.copy(ImmutableMap.of("id", 3L, "data", "c")));
+    builder.add(record.copy(ImmutableMap.of("id", 4L, "data", "d")));
+    builder.add(record.copy(ImmutableMap.of("id", 5L, "data", "e")));
+
+    this.records = builder.build();
+  }
+
+  @Test
+  public void testDataWriter() throws IOException {
+    OutputFile file = Files.localOutput(temp.newFile());
+
+    SortOrder sortOrder = SortOrder.builderFor(SCHEMA)
+        .withOrderId(10)
+        .asc("id")
+        .build();
+
+    DataWriter<Record> dataWriter = Avro.writeData(file)
+        .schema(SCHEMA)
+        .createWriterFunc(org.apache.iceberg.data.avro.DataWriter::create)
+        .overwrite()
+        .withSpec(PartitionSpec.unpartitioned())
+        .withSortOrder(sortOrder)
+        .build();
+
+    try {
+      for (Record record : records) {
+        dataWriter.add(record);
+      }
+    } finally {
+      dataWriter.close();
+    }
+
+    DataFile dataFile = dataWriter.toDataFile();
+
+    Assert.assertEquals("Format should be Avro", FileFormat.AVRO, 
dataFile.format());
+    Assert.assertEquals("Should be data file", FileContent.DATA, 
dataFile.content());
+    Assert.assertEquals("Record count should match", records.size(), 
dataFile.recordCount());
+    Assert.assertEquals("Partition should be empty", 0, 
dataFile.partition().size());
+    Assert.assertEquals("Sort order should match", sortOrder.orderId(), (int) 
dataFile.sortOrderId());
+    Assert.assertNull("Key metadata should be null", dataFile.keyMetadata());
+
+    List<Record> writtenRecords;
+    try (AvroIterable<Record> reader = Avro.read(file.toInputFile())
+        .project(SCHEMA)
+        .createReaderFunc(org.apache.iceberg.data.avro.DataReader::create)
+        .build()) {
+      writtenRecords = Lists.newArrayList(reader);
+    }
+
+    Assert.assertEquals("Written records should match", records, 
writtenRecords);
+  }
+}
diff --git 
a/data/src/test/java/org/apache/iceberg/data/orc/TestOrcDataWriter.java 
b/data/src/test/java/org/apache/iceberg/data/orc/TestOrcDataWriter.java
new file mode 100644
index 0000000..af759d3
--- /dev/null
+++ b/data/src/test/java/org/apache/iceberg/data/orc/TestOrcDataWriter.java
@@ -0,0 +1,115 @@
+/*
+ * 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.iceberg.data.orc;
+
+import java.io.IOException;
+import java.util.List;
+import org.apache.iceberg.DataFile;
+import org.apache.iceberg.FileContent;
+import org.apache.iceberg.FileFormat;
+import org.apache.iceberg.Files;
+import org.apache.iceberg.PartitionSpec;
+import org.apache.iceberg.Schema;
+import org.apache.iceberg.SortOrder;
+import org.apache.iceberg.data.GenericRecord;
+import org.apache.iceberg.data.Record;
+import org.apache.iceberg.io.CloseableIterable;
+import org.apache.iceberg.io.DataWriter;
+import org.apache.iceberg.io.OutputFile;
+import org.apache.iceberg.orc.ORC;
+import org.apache.iceberg.relocated.com.google.common.collect.ImmutableList;
+import org.apache.iceberg.relocated.com.google.common.collect.ImmutableMap;
+import org.apache.iceberg.relocated.com.google.common.collect.Lists;
+import org.apache.iceberg.types.Types;
+import org.junit.Assert;
+import org.junit.Before;
+import org.junit.Rule;
+import org.junit.Test;
+import org.junit.rules.TemporaryFolder;
+
+public class TestOrcDataWriter {
+  private static final Schema SCHEMA = new Schema(
+      Types.NestedField.required(1, "id", Types.LongType.get()),
+      Types.NestedField.optional(2, "data", Types.StringType.get()));
+
+  private List<Record> records;
+
+  @Rule
+  public TemporaryFolder temp = new TemporaryFolder();
+
+  @Before
+  public void createRecords() {
+    GenericRecord record = GenericRecord.create(SCHEMA);
+
+    ImmutableList.Builder<Record> builder = ImmutableList.builder();
+    builder.add(record.copy(ImmutableMap.of("id", 1L, "data", "a")));
+    builder.add(record.copy(ImmutableMap.of("id", 2L, "data", "b")));
+    builder.add(record.copy(ImmutableMap.of("id", 3L, "data", "c")));
+    builder.add(record.copy(ImmutableMap.of("id", 4L, "data", "d")));
+    builder.add(record.copy(ImmutableMap.of("id", 5L, "data", "e")));
+
+    this.records = builder.build();
+  }
+
+  @Test
+  public void testDataWriter() throws IOException {
+    OutputFile file = Files.localOutput(temp.newFile());
+
+    SortOrder sortOrder = SortOrder.builderFor(SCHEMA)
+        .withOrderId(10)
+        .asc("id")
+        .build();
+
+    DataWriter<Record> dataWriter = ORC.writeData(file)
+        .schema(SCHEMA)
+        .createWriterFunc(GenericOrcWriter::buildWriter)
+        .overwrite()
+        .withSpec(PartitionSpec.unpartitioned())
+        .withSortOrder(sortOrder)
+        .build();
+
+    try {
+      for (Record record : records) {
+        dataWriter.add(record);
+      }
+    } finally {
+      dataWriter.close();
+    }
+
+    DataFile dataFile = dataWriter.toDataFile();
+
+    Assert.assertEquals("Format should be ORC", FileFormat.ORC, 
dataFile.format());
+    Assert.assertEquals("Should be data file", FileContent.DATA, 
dataFile.content());
+    Assert.assertEquals("Record count should match", records.size(), 
dataFile.recordCount());
+    Assert.assertEquals("Partition should be empty", 0, 
dataFile.partition().size());
+    Assert.assertEquals("Sort order should match", sortOrder.orderId(), (int) 
dataFile.sortOrderId());
+    Assert.assertNull("Key metadata should be null", dataFile.keyMetadata());
+
+    List<Record> writtenRecords;
+    try (CloseableIterable<Record> reader = ORC.read(file.toInputFile())
+        .project(SCHEMA)
+        .createReaderFunc(fileSchema -> GenericOrcReader.buildReader(SCHEMA, 
fileSchema))
+        .build()) {
+      writtenRecords = Lists.newArrayList(reader);
+    }
+
+    Assert.assertEquals("Written records should match", records, 
writtenRecords);
+  }
+}
diff --git a/orc/src/main/java/org/apache/iceberg/orc/ORC.java 
b/orc/src/main/java/org/apache/iceberg/orc/ORC.java
index aff66e6..1ab5c70 100644
--- a/orc/src/main/java/org/apache/iceberg/orc/ORC.java
+++ b/orc/src/main/java/org/apache/iceberg/orc/ORC.java
@@ -27,13 +27,20 @@ import java.util.function.BiFunction;
 import java.util.function.Function;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.fs.Path;
+import org.apache.iceberg.FileFormat;
 import org.apache.iceberg.MetricsConfig;
+import org.apache.iceberg.PartitionSpec;
 import org.apache.iceberg.Schema;
+import org.apache.iceberg.SortOrder;
+import org.apache.iceberg.StructLike;
+import org.apache.iceberg.Table;
+import org.apache.iceberg.encryption.EncryptionKeyMetadata;
 import org.apache.iceberg.exceptions.RuntimeIOException;
 import org.apache.iceberg.expressions.Expression;
 import org.apache.iceberg.hadoop.HadoopInputFile;
 import org.apache.iceberg.hadoop.HadoopOutputFile;
 import org.apache.iceberg.io.CloseableIterable;
+import org.apache.iceberg.io.DataWriter;
 import org.apache.iceberg.io.FileAppender;
 import org.apache.iceberg.io.InputFile;
 import org.apache.iceberg.io.OutputFile;
@@ -122,6 +129,98 @@ public class ORC {
     }
   }
 
+  public static DataWriteBuilder writeData(OutputFile file) {
+    return new DataWriteBuilder(file);
+  }
+
+  public static class DataWriteBuilder {
+    private final WriteBuilder appenderBuilder;
+    private final String location;
+    private PartitionSpec spec = null;
+    private StructLike partition = null;
+    private EncryptionKeyMetadata keyMetadata = null;
+    private SortOrder sortOrder = null;
+
+    private DataWriteBuilder(OutputFile file) {
+      this.appenderBuilder = write(file);
+      this.location = file.location();
+    }
+
+    public DataWriteBuilder forTable(Table table) {
+      schema(table.schema());
+      withSpec(table.spec());
+      setAll(table.properties());
+      metricsConfig(MetricsConfig.fromProperties(table.properties()));
+      return this;
+    }
+
+    public DataWriteBuilder schema(Schema newSchema) {
+      appenderBuilder.schema(newSchema);
+      return this;
+    }
+
+    public DataWriteBuilder set(String property, String value) {
+      appenderBuilder.config(property, value);
+      return this;
+    }
+
+    public DataWriteBuilder setAll(Map<String, String> properties) {
+      appenderBuilder.setAll(properties);
+      return this;
+    }
+
+    public DataWriteBuilder meta(String property, String value) {
+      appenderBuilder.metadata(property, value);
+      return this;
+    }
+
+    public DataWriteBuilder overwrite() {
+      return overwrite(true);
+    }
+
+    public DataWriteBuilder overwrite(boolean enabled) {
+      appenderBuilder.overwrite(enabled);
+      return this;
+    }
+
+    public DataWriteBuilder metricsConfig(MetricsConfig newMetricsConfig) {
+      appenderBuilder.metricsConfig(newMetricsConfig);
+      return this;
+    }
+
+    public DataWriteBuilder createWriterFunc(BiFunction<Schema, 
TypeDescription, OrcRowWriter<?>> writerFunction) {
+      appenderBuilder.createWriterFunc(writerFunction);
+      return this;
+    }
+
+    public DataWriteBuilder withSpec(PartitionSpec newSpec) {
+      this.spec = newSpec;
+      return this;
+    }
+
+    public DataWriteBuilder withPartition(StructLike newPartition) {
+      this.partition = newPartition;
+      return this;
+    }
+
+    public DataWriteBuilder withKeyMetadata(EncryptionKeyMetadata metadata) {
+      this.keyMetadata = metadata;
+      return this;
+    }
+
+    public DataWriteBuilder withSortOrder(SortOrder newSortOrder) {
+      this.sortOrder = newSortOrder;
+      return this;
+    }
+
+    public <T> DataWriter<T> build() {
+      Preconditions.checkArgument(spec != null, "Cannot create data writer 
without spec");
+
+      FileAppender<T> fileAppender = appenderBuilder.build();
+      return new DataWriter<>(fileAppender, FileFormat.ORC, location, spec, 
partition, keyMetadata, sortOrder);
+    }
+  }
+
   public static ReadBuilder read(InputFile file) {
     return new ReadBuilder(file);
   }
diff --git a/parquet/src/main/java/org/apache/iceberg/parquet/Parquet.java 
b/parquet/src/main/java/org/apache/iceberg/parquet/Parquet.java
index 69ae1cf..fef3209 100644
--- a/parquet/src/main/java/org/apache/iceberg/parquet/Parquet.java
+++ b/parquet/src/main/java/org/apache/iceberg/parquet/Parquet.java
@@ -49,6 +49,7 @@ import org.apache.iceberg.expressions.Expression;
 import org.apache.iceberg.hadoop.HadoopInputFile;
 import org.apache.iceberg.hadoop.HadoopOutputFile;
 import org.apache.iceberg.io.CloseableIterable;
+import org.apache.iceberg.io.DataWriter;
 import org.apache.iceberg.io.DeleteSchemaUtil;
 import org.apache.iceberg.io.FileAppender;
 import org.apache.iceberg.io.InputFile;
@@ -369,6 +370,98 @@ public class Parquet {
     }
   }
 
+  public static DataWriteBuilder writeData(OutputFile file) {
+    return new DataWriteBuilder(file);
+  }
+
+  public static class DataWriteBuilder {
+    private final WriteBuilder appenderBuilder;
+    private final String location;
+    private PartitionSpec spec = null;
+    private StructLike partition = null;
+    private EncryptionKeyMetadata keyMetadata = null;
+    private SortOrder sortOrder = null;
+
+    private DataWriteBuilder(OutputFile file) {
+      this.appenderBuilder = write(file);
+      this.location = file.location();
+    }
+
+    public DataWriteBuilder forTable(Table table) {
+      schema(table.schema());
+      withSpec(table.spec());
+      setAll(table.properties());
+      metricsConfig(MetricsConfig.fromProperties(table.properties()));
+      return this;
+    }
+
+    public DataWriteBuilder schema(Schema newSchema) {
+      appenderBuilder.schema(newSchema);
+      return this;
+    }
+
+    public DataWriteBuilder set(String property, String value) {
+      appenderBuilder.set(property, value);
+      return this;
+    }
+
+    public DataWriteBuilder setAll(Map<String, String> properties) {
+      appenderBuilder.setAll(properties);
+      return this;
+    }
+
+    public DataWriteBuilder meta(String property, String value) {
+      appenderBuilder.meta(property, value);
+      return this;
+    }
+
+    public DataWriteBuilder overwrite() {
+      return overwrite(true);
+    }
+
+    public DataWriteBuilder overwrite(boolean enabled) {
+      appenderBuilder.overwrite(enabled);
+      return this;
+    }
+
+    public DataWriteBuilder metricsConfig(MetricsConfig newMetricsConfig) {
+      appenderBuilder.metricsConfig(newMetricsConfig);
+      return this;
+    }
+
+    public DataWriteBuilder createWriterFunc(Function<MessageType, 
ParquetValueWriter<?>> newCreateWriterFunc) {
+      appenderBuilder.createWriterFunc(newCreateWriterFunc);
+      return this;
+    }
+
+    public DataWriteBuilder withSpec(PartitionSpec newSpec) {
+      this.spec = newSpec;
+      return this;
+    }
+
+    public DataWriteBuilder withPartition(StructLike newPartition) {
+      this.partition = newPartition;
+      return this;
+    }
+
+    public DataWriteBuilder withKeyMetadata(EncryptionKeyMetadata metadata) {
+      this.keyMetadata = metadata;
+      return this;
+    }
+
+    public DataWriteBuilder withSortOrder(SortOrder newSortOrder) {
+      this.sortOrder = newSortOrder;
+      return this;
+    }
+
+    public <T> DataWriter<T> build() throws IOException {
+      Preconditions.checkArgument(spec != null, "Cannot create data writer 
without spec");
+
+      FileAppender<T> fileAppender = appenderBuilder.build();
+      return new DataWriter<>(fileAppender, FileFormat.PARQUET, location, 
spec, partition, keyMetadata, sortOrder);
+    }
+  }
+
   public static DeleteWriteBuilder writeDeletes(OutputFile file) {
     return new DeleteWriteBuilder(file);
   }
diff --git 
a/parquet/src/test/java/org/apache/iceberg/parquet/TestParquetDataWriter.java 
b/parquet/src/test/java/org/apache/iceberg/parquet/TestParquetDataWriter.java
new file mode 100644
index 0000000..b29a627
--- /dev/null
+++ 
b/parquet/src/test/java/org/apache/iceberg/parquet/TestParquetDataWriter.java
@@ -0,0 +1,116 @@
+/*
+ * 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.iceberg.parquet;
+
+import java.io.IOException;
+import java.util.List;
+import org.apache.iceberg.DataFile;
+import org.apache.iceberg.FileContent;
+import org.apache.iceberg.FileFormat;
+import org.apache.iceberg.Files;
+import org.apache.iceberg.PartitionSpec;
+import org.apache.iceberg.Schema;
+import org.apache.iceberg.SortOrder;
+import org.apache.iceberg.data.GenericRecord;
+import org.apache.iceberg.data.Record;
+import org.apache.iceberg.data.parquet.GenericParquetReaders;
+import org.apache.iceberg.data.parquet.GenericParquetWriter;
+import org.apache.iceberg.io.CloseableIterable;
+import org.apache.iceberg.io.DataWriter;
+import org.apache.iceberg.io.OutputFile;
+import org.apache.iceberg.relocated.com.google.common.collect.ImmutableList;
+import org.apache.iceberg.relocated.com.google.common.collect.ImmutableMap;
+import org.apache.iceberg.relocated.com.google.common.collect.Lists;
+import org.apache.iceberg.types.Types;
+import org.junit.Assert;
+import org.junit.Before;
+import org.junit.Rule;
+import org.junit.Test;
+import org.junit.rules.TemporaryFolder;
+
+public class TestParquetDataWriter {
+  private static final Schema SCHEMA = new Schema(
+      Types.NestedField.required(1, "id", Types.LongType.get()),
+      Types.NestedField.optional(2, "data", Types.StringType.get()));
+
+  private List<Record> records;
+
+  @Rule
+  public TemporaryFolder temp = new TemporaryFolder();
+
+  @Before
+  public void createRecords() {
+    GenericRecord record = GenericRecord.create(SCHEMA);
+
+    ImmutableList.Builder<Record> builder = ImmutableList.builder();
+    builder.add(record.copy(ImmutableMap.of("id", 1L, "data", "a")));
+    builder.add(record.copy(ImmutableMap.of("id", 2L, "data", "b")));
+    builder.add(record.copy(ImmutableMap.of("id", 3L, "data", "c")));
+    builder.add(record.copy(ImmutableMap.of("id", 4L, "data", "d")));
+    builder.add(record.copy(ImmutableMap.of("id", 5L, "data", "e")));
+
+    this.records = builder.build();
+  }
+
+  @Test
+  public void testDataWriter() throws IOException {
+    OutputFile file = Files.localOutput(temp.newFile());
+
+    SortOrder sortOrder = SortOrder.builderFor(SCHEMA)
+        .withOrderId(10)
+        .asc("id")
+        .build();
+
+    DataWriter<Record> dataWriter = Parquet.writeData(file)
+        .schema(SCHEMA)
+        .createWriterFunc(GenericParquetWriter::buildWriter)
+        .overwrite()
+        .withSpec(PartitionSpec.unpartitioned())
+        .withSortOrder(sortOrder)
+        .build();
+
+    try {
+      for (Record record : records) {
+        dataWriter.add(record);
+      }
+    } finally {
+      dataWriter.close();
+    }
+
+    DataFile dataFile = dataWriter.toDataFile();
+
+    Assert.assertEquals("Format should be Parquet", FileFormat.PARQUET, 
dataFile.format());
+    Assert.assertEquals("Should be data file", FileContent.DATA, 
dataFile.content());
+    Assert.assertEquals("Record count should match", records.size(), 
dataFile.recordCount());
+    Assert.assertEquals("Partition should be empty", 0, 
dataFile.partition().size());
+    Assert.assertEquals("Sort order should match", sortOrder.orderId(), (int) 
dataFile.sortOrderId());
+    Assert.assertNull("Key metadata should be null", dataFile.keyMetadata());
+
+    List<Record> writtenRecords;
+    try (CloseableIterable<Record> reader = Parquet.read(file.toInputFile())
+        .project(SCHEMA)
+        .createReaderFunc(fileSchema -> 
GenericParquetReaders.buildReader(SCHEMA, fileSchema))
+        .build()) {
+      writtenRecords = Lists.newArrayList(reader);
+    }
+
+    Assert.assertEquals("Written records should match", records, 
writtenRecords);
+  }
+}

Reply via email to