Skip to content
Merged
Show file tree
Hide file tree
Changes from all commits
Commits
File filter

Filter by extension

Filter by extension

Conversations
Failed to load comments.
Loading
Jump to
Jump to file
Failed to load files.
Loading
Diff view
Diff view
Original file line number Diff line number Diff line change
@@ -0,0 +1,98 @@
/*
* 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;

import java.io.IOException;
import java.io.UncheckedIOException;
import java.util.Map;
import org.apache.iceberg.FileFormat;
import org.apache.iceberg.MetricsConfig;
import org.apache.iceberg.Schema;
import org.apache.iceberg.avro.Avro;
import org.apache.iceberg.data.avro.DataWriter;
import org.apache.iceberg.data.orc.GenericOrcWriter;
import org.apache.iceberg.data.parquet.GenericParquetWriter;
import org.apache.iceberg.io.FileAppender;
import org.apache.iceberg.io.FileAppenderFactory;
import org.apache.iceberg.io.OutputFile;
import org.apache.iceberg.orc.ORC;
import org.apache.iceberg.parquet.Parquet;
import org.apache.iceberg.relocated.com.google.common.collect.Maps;

/**
* Factory to create a new {@link FileAppender} to write {@link Record}s.
*/
public class GenericAppenderFactory implements FileAppenderFactory<Record> {

private final Schema schema;
private final Map<String, String> config = Maps.newHashMap();

public GenericAppenderFactory(Schema schema) {
this.schema = schema;
}

public GenericAppenderFactory set(String property, String value) {
config.put(property, value);
return this;
}

public GenericAppenderFactory setAll(Map<String, String> properties) {
config.putAll(properties);
return this;
}

@Override
public FileAppender<Record> newAppender(OutputFile outputFile, FileFormat fileFormat) {
MetricsConfig metricsConfig = MetricsConfig.fromProperties(config);
try {
switch (fileFormat) {
case AVRO:
return Avro.write(outputFile)
.schema(schema)
.createWriterFunc(DataWriter::create)
.setAll(config)
.overwrite()
.build();

case PARQUET:
return Parquet.write(outputFile)
.schema(schema)
.createWriterFunc(GenericParquetWriter::buildWriter)
.setAll(config)
.metricsConfig(metricsConfig)
.overwrite()
.build();

case ORC:
return ORC.write(outputFile)
.schema(schema)
.createWriterFunc(GenericOrcWriter::buildWriter)
.setAll(config)
.overwrite()
.build();

default:
throw new UnsupportedOperationException("Cannot write format: " + fileFormat);
}
} catch (IOException e) {
throw new UncheckedIOException(e);
}
}
}
31 changes: 5 additions & 26 deletions data/src/test/java/org/apache/iceberg/TestSplitScan.java
Original file line number Diff line number Diff line change
Expand Up @@ -24,15 +24,12 @@
import java.util.List;
import java.util.Locale;
import org.apache.hadoop.conf.Configuration;
import org.apache.iceberg.avro.Avro;
import org.apache.iceberg.data.GenericAppenderFactory;
import org.apache.iceberg.data.IcebergGenerics;
import org.apache.iceberg.data.RandomGenericData;
import org.apache.iceberg.data.Record;
import org.apache.iceberg.data.avro.DataWriter;
import org.apache.iceberg.data.parquet.GenericParquetWriter;
import org.apache.iceberg.hadoop.HadoopTables;
import org.apache.iceberg.io.FileAppender;
import org.apache.iceberg.parquet.Parquet;
import org.apache.iceberg.relocated.com.google.common.collect.Lists;
import org.apache.iceberg.types.Types;
import org.junit.Assert;
Expand Down Expand Up @@ -122,28 +119,10 @@ private File writeToFile(List<Record> records, FileFormat fileFormat) throws IOE
File file = temp.newFile();
Assert.assertTrue(file.delete());

switch (fileFormat) {
case AVRO:
try (FileAppender<Record> appender = Avro.write(Files.localOutput(file))
.schema(SCHEMA)
.createWriterFunc(DataWriter::create)
.named(fileFormat.name())
.build()) {
appender.addAll(records);
}
break;
case PARQUET:
try (FileAppender<Record> appender = Parquet.write(Files.localOutput(file))
.schema(SCHEMA)
.createWriterFunc(GenericParquetWriter::buildWriter)
.named(fileFormat.name())
.set(TableProperties.PARQUET_ROW_GROUP_SIZE_BYTES, String.valueOf(SPLIT_SIZE))
.build()) {
appender.addAll(records);
}
break;
default:
throw new UnsupportedOperationException("Cannot write format: " + fileFormat);
GenericAppenderFactory factory = new GenericAppenderFactory(SCHEMA).set(
TableProperties.PARQUET_ROW_GROUP_SIZE_BYTES, String.valueOf(SPLIT_SIZE));
try (FileAppender<Record> appender = factory.newAppender(Files.localOutput(file), fileFormat)) {
appender.addAll(records);
}
return file;
}
Expand Down
Original file line number Diff line number Diff line change
@@ -0,0 +1,97 @@
/*
* 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;

import java.io.File;
import java.io.IOException;
import java.util.List;
import org.apache.iceberg.AppendFiles;
import org.apache.iceberg.DataFile;
import org.apache.iceberg.DataFiles;
import org.apache.iceberg.FileFormat;
import org.apache.iceberg.Files;
import org.apache.iceberg.StructLike;
import org.apache.iceberg.Table;
import org.apache.iceberg.io.FileAppender;
import org.apache.iceberg.relocated.com.google.common.base.Preconditions;
import org.junit.Assert;
import org.junit.rules.TemporaryFolder;

/**
* Helper for appending {@link DataFile} to a table or appending {@link Record}s to a table.
*/
public class GenericAppenderHelper {

private final Table table;
private final FileFormat fileFormat;
private final TemporaryFolder tmp;

public GenericAppenderHelper(Table table, FileFormat fileFormat, TemporaryFolder tmp) {
this.table = table;
this.fileFormat = fileFormat;
this.tmp = tmp;
}

public void appendToTable(DataFile... dataFiles) {
Preconditions.checkNotNull(table, "table not set");

AppendFiles append = table.newAppend();

for (DataFile dataFile : dataFiles) {
append = append.appendFile(dataFile);
}

append.commit();
}

public void appendToTable(List<Record> records) throws IOException {
appendToTable(null, records);
}

public void appendToTable(StructLike partition, List<Record> records) throws IOException {
appendToTable(writeFile(partition, records));
}

public DataFile writeFile(StructLike partition, List<Record> records) throws IOException {
Preconditions.checkNotNull(table, "table not set");
File file = tmp.newFile();
Assert.assertTrue(file.delete());
return appendToLocalFile(table, file, fileFormat, partition, records);
}

private static DataFile appendToLocalFile(
Table table, File file, FileFormat format, StructLike partition, List<Record> records)
throws IOException {
FileAppender<Record> appender = new GenericAppenderFactory(table.schema()).newAppender(
Files.localOutput(file), format);
try (FileAppender<Record> fileAppender = appender) {
fileAppender.addAll(records);
}

return DataFiles.builder(table.spec())
.withRecordCount(records.size())
.withFileSizeInBytes(file.length())
.withPath(file.toURI().toString())
.withMetrics(appender.metrics())
.withFormat(format)
.withPartition(partition)
.build();
}
}
68 changes: 10 additions & 58 deletions data/src/test/java/org/apache/iceberg/data/TestLocalScan.java
Original file line number Diff line number Diff line change
Expand Up @@ -39,16 +39,10 @@
import org.apache.iceberg.Table;
import org.apache.iceberg.TableProperties;
import org.apache.iceberg.Tables;
import org.apache.iceberg.avro.Avro;
import org.apache.iceberg.data.avro.DataWriter;
import org.apache.iceberg.data.orc.GenericOrcWriter;
import org.apache.iceberg.data.parquet.GenericParquetWriter;
import org.apache.iceberg.expressions.Expressions;
import org.apache.iceberg.hadoop.HadoopInputFile;
import org.apache.iceberg.hadoop.HadoopTables;
import org.apache.iceberg.io.FileAppender;
import org.apache.iceberg.orc.ORC;
import org.apache.iceberg.parquet.Parquet;
import org.apache.iceberg.relocated.com.google.common.base.Preconditions;
import org.apache.iceberg.relocated.com.google.common.collect.ImmutableList;
import org.apache.iceberg.relocated.com.google.common.collect.ImmutableMap;
Expand Down Expand Up @@ -458,59 +452,17 @@ private DataFile writeFile(String location, String filename, Schema schema, List
Path path = new Path(location, filename);
FileFormat fileFormat = FileFormat.fromFileName(filename);
Preconditions.checkNotNull(fileFormat, "Cannot determine format for file: %s", filename);
switch (fileFormat) {
case AVRO:
FileAppender<Record> avroAppender = Avro.write(fromPath(path, CONF))
.schema(schema)
.createWriterFunc(DataWriter::create)
.named(fileFormat.name())
.build();
try {
avroAppender.addAll(records);
} finally {
avroAppender.close();
}

return DataFiles.builder(PartitionSpec.unpartitioned())
.withInputFile(HadoopInputFile.fromPath(path, CONF))
.withMetrics(avroAppender.metrics())
.build();

case PARQUET:
FileAppender<Record> parquetAppender = Parquet.write(fromPath(path, CONF))
.schema(schema)
.createWriterFunc(GenericParquetWriter::buildWriter)
.build();
try {
parquetAppender.addAll(records);
} finally {
parquetAppender.close();
}

return DataFiles.builder(PartitionSpec.unpartitioned())
.withInputFile(HadoopInputFile.fromPath(path, CONF))
.withMetrics(parquetAppender.metrics())
.build();

case ORC:
FileAppender<Record> orcAppender = ORC.write(fromPath(path, CONF))
.schema(schema)
.createWriterFunc(GenericOrcWriter::buildWriter)
.build();
try {
orcAppender.addAll(records);
} finally {
orcAppender.close();
}

return DataFiles.builder(PartitionSpec.unpartitioned())
.withInputFile(HadoopInputFile.fromPath(path, CONF))
.withMetrics(orcAppender.metrics())
.build();

default:
throw new UnsupportedOperationException("Cannot write format: " + fileFormat);

FileAppender<Record> fileAppender = new GenericAppenderFactory(schema).newAppender(
fromPath(path, CONF), fileFormat);
try (FileAppender<Record> appender = fileAppender) {
appender.addAll(records);
}

return DataFiles.builder(PartitionSpec.unpartitioned())
.withInputFile(HadoopInputFile.fromPath(path, CONF))
.withMetrics(fileAppender.metrics())
.build();
}

@Test
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -202,11 +202,6 @@ public Type primitive(Type.PrimitiveType primitive) {
"Cannot project decimal with incompatible precision: %s < %s",
requestedDecimal.precision(), decimal.precision());
break;
case TIMESTAMP:
Types.TimestampType timestamp = (Types.TimestampType) primitive;
Preconditions.checkArgument(timestamp.shouldAdjustToUTC(),
"Cannot project timestamp (without time zone) as timestamptz (with time zone)");
break;
default:
}

Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -104,12 +104,7 @@ public DataType primitive(Type.PrimitiveType primitive) {
throw new UnsupportedOperationException(
"Spark does not support time fields");
case TIMESTAMP:
Types.TimestampType timestamp = (Types.TimestampType) primitive;
if (timestamp.shouldAdjustToUTC()) {
return TimestampType$.MODULE$;
}
throw new UnsupportedOperationException(
"Spark does not support timestamp without time zone fields");
return TimestampType$.MODULE$;
case STRING:
return StringType$.MODULE$;
case UUID:
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -104,6 +104,7 @@ public OrcValueReader<?> primitive(Type.PrimitiveType iPrimitive, TypeDescriptio
return OrcValueReaders.floats();
case DOUBLE:
return OrcValueReaders.doubles();
case TIMESTAMP:
case TIMESTAMP_INSTANT:
return SparkOrcValueReaders.timestampTzs();
case DECIMAL:
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -127,6 +127,7 @@ public Converter primitive(Type.PrimitiveType iPrimitive, TypeDescription primit
case DOUBLE:
primitiveValueReader = OrcValueReaders.doubles();
break;
case TIMESTAMP:
case TIMESTAMP_INSTANT:
primitiveValueReader = SparkOrcValueReaders.timestampTzs();
break;
Expand Down
Loading