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
2 changes: 1 addition & 1 deletion .github/trigger_files/IO_Iceberg_Integration_Tests.json
Original file line number Diff line number Diff line change
@@ -1,4 +1,4 @@
{
"comment": "Modify this file in a trivial way to cause this test suite to run.",
"modification": 4
"modification": 1
}
7 changes: 6 additions & 1 deletion sdks/java/io/iceberg/build.gradle
Original file line number Diff line number Diff line change
Expand Up @@ -51,6 +51,8 @@ dependencies {
implementation library.java.joda_time
implementation "org.apache.parquet:parquet-column:$parquet_version"
implementation "org.apache.parquet:parquet-hadoop:$parquet_version"
implementation "org.apache.parquet:parquet-common:$parquet_version"
implementation project(":sdks:java:io:parquet")
implementation "org.apache.orc:orc-core:$orc_version"
implementation "org.apache.iceberg:iceberg-core:$iceberg_version"
implementation "org.apache.iceberg:iceberg-api:$iceberg_version"
Expand All @@ -74,11 +76,13 @@ dependencies {
testImplementation library.java.bigdataoss_gcsio
testImplementation library.java.bigdataoss_util_hadoop
testImplementation "org.apache.parquet:parquet-avro:$parquet_version"
testImplementation "org.apache.parquet:parquet-common:$parquet_version"
testImplementation "org.apache.iceberg:iceberg-data:$iceberg_version"
testImplementation project(path: ":sdks:java:core", configuration: "shadowTest")
testImplementation project(":sdks:java:extensions:google-cloud-platform-core")
testImplementation library.java.junit
testImplementation library.java.hamcrest
testImplementation project(path: ":sdks:java:extensions:avro")
testImplementation 'org.awaitility:awaitility:4.2.0'

// Hive catalog test dependencies
testImplementation project(path: ":sdks:java:io:iceberg:hive")
Expand All @@ -95,6 +99,7 @@ dependencies {
testImplementation project(path: ":sdks:java:io:iceberg:bqms", configuration: "shadow")
testImplementation project(":sdks:java:io:google-cloud-platform")
testImplementation library.java.google_api_services_bigquery
testImplementation 'com.google.cloud:google-cloud-storage'

testImplementation library.java.google_auth_library_oauth2_http
testRuntimeOnly library.java.slf4j_jdk14
Expand Down

Large diffs are not rendered by default.

Original file line number Diff line number Diff line change
@@ -0,0 +1,190 @@
/*
* 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.beam.sdk.io.iceberg;

import static org.apache.beam.sdk.io.iceberg.AddFiles.ERROR_TAG;
import static org.apache.beam.sdk.io.iceberg.AddFiles.OUTPUT_TAG;
import static org.apache.beam.sdk.io.iceberg.AddFilesSchemaTransformProvider.Configuration;
import static org.apache.beam.sdk.util.Preconditions.checkStateNotNull;

import com.google.auto.service.AutoService;
import com.google.auto.value.AutoValue;
import java.util.List;
import java.util.Map;
import org.apache.beam.sdk.schemas.AutoValueSchema;
import org.apache.beam.sdk.schemas.Schema;
import org.apache.beam.sdk.schemas.annotations.DefaultSchema;
import org.apache.beam.sdk.schemas.annotations.SchemaFieldDescription;
import org.apache.beam.sdk.schemas.transforms.SchemaTransform;
import org.apache.beam.sdk.schemas.transforms.SchemaTransformProvider;
import org.apache.beam.sdk.schemas.transforms.TypedSchemaTransformProvider;
import org.apache.beam.sdk.schemas.transforms.providers.ErrorHandling;
import org.apache.beam.sdk.transforms.Filter;
import org.apache.beam.sdk.transforms.MapElements;
import org.apache.beam.sdk.values.PCollectionRowTuple;
import org.apache.beam.sdk.values.TypeDescriptors;
import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.base.Preconditions;
import org.checkerframework.checker.nullness.qual.Nullable;
import org.joda.time.Duration;

@AutoService(SchemaTransformProvider.class)
public class AddFilesSchemaTransformProvider extends TypedSchemaTransformProvider<Configuration> {
@Override
public AddFilesSchemaTransform from(Configuration configuration) {
return new AddFilesSchemaTransform(configuration);
}

@Override
public String identifier() {
return "beam:schematransform:iceberg_add_files:v1";
}

@DefaultSchema(AutoValueSchema.class)
@AutoValue
public abstract static class Configuration {
public static Builder builder() {
return new AutoValue_AddFilesSchemaTransformProvider_Configuration.Builder();
}

@SchemaFieldDescription("A fully-qualified table identifier.")
public abstract String getTable();

@SchemaFieldDescription("Properties used to set up the Iceberg catalog.")
public abstract @Nullable Map<String, String> getCatalogProperties();

@SchemaFieldDescription("Properties passed to the Hadoop ")
public abstract @Nullable Map<String, String> getConfigProperties();

@SchemaFieldDescription(
"For a streaming pipeline, sets the frequency at which incoming files are appended. Defaults to 600 (10 minutes). "
+ "A commit is triggered when either this or append batch size is reached.")
public abstract @Nullable Integer getTriggeringFrequencySeconds();

@SchemaFieldDescription(
"For a streaming pipeline, sets the desired number of appended files per commit. Defaults to 100,000 files. "
+ "A commit is triggered when either this or append triggering interval is reached.")
public abstract @Nullable Integer getAppendBatchSize();

@SchemaFieldDescription(
"The prefix shared among all partitions. For example, a data file may have the following"
+ " location:%n"
+ "'gs://bucket/namespace/table/data/id=13/name=beam/data_file.parquet'%n%n"
+ "The provided prefix should go up until the partition information:%n"
+ "'gs://bucket/namespace/table/data/'.%n"
+ "If not provided, will try determining each DataFile's partition from its metrics metadata.")
public abstract @Nullable String getLocationPrefix();

@SchemaFieldDescription(
"Fields used to create a partition spec that is applied when tables are created. For a field 'foo', "
+ "the available partition transforms are:\n\n"
+ "- `foo`\n"
+ "- `truncate(foo, N)`\n"
+ "- `bucket(foo, N)`\n"
+ "- `hour(foo)`\n"
+ "- `day(foo)`\n"
+ "- `month(foo)`\n"
+ "- `year(foo)`\n"
+ "- `void(foo)`\n\n"
+ "For more information on partition transforms, please visit https://iceberg.apache.org/spec/#partition-transforms.")
public abstract @Nullable List<String> getPartitionFields();

@SchemaFieldDescription(
"Iceberg table properties to be set on the table when it is created.\n"
+ "For more information on table properties,"
+ " please visit https://iceberg.apache.org/docs/latest/configuration/#table-properties.")
public abstract @Nullable Map<String, String> getTableProperties();

@SchemaFieldDescription("This option specifies whether and where to output unwritable rows.")
public abstract @Nullable ErrorHandling getErrorHandling();

@AutoValue.Builder
public abstract static class Builder {
public abstract Builder setTable(String table);

public abstract Builder setCatalogProperties(Map<String, String> catalogProperties);

public abstract Builder setConfigProperties(Map<String, String> confProperties);

public abstract Builder setTriggeringFrequencySeconds(Integer triggeringFrequencySeconds);

public abstract Builder setAppendBatchSize(Integer size);

public abstract Builder setLocationPrefix(String prefix);

public abstract Builder setPartitionFields(List<String> fields);

public abstract Builder setTableProperties(Map<String, String> props);

public abstract Builder setErrorHandling(ErrorHandling errorHandling);

public abstract Configuration build();
}

public IcebergCatalogConfig getIcebergCatalog() {
return IcebergCatalogConfig.builder()
.setCatalogProperties(getCatalogProperties())
.setConfigProperties(getConfigProperties())
.build();
}
}

public static class AddFilesSchemaTransform extends SchemaTransform {
private final Configuration configuration;

public AddFilesSchemaTransform(Configuration configuration) {
this.configuration = configuration;
}

@Override
public PCollectionRowTuple expand(PCollectionRowTuple input) {
Schema inputSchema = input.getSinglePCollection().getSchema();
Preconditions.checkState(
inputSchema.getFieldCount() == 1
&& inputSchema.getField(0).getType().getTypeName().equals(Schema.TypeName.STRING),
"Incoming Row Schema must contain only one field of type String. Instead, got schema: %s",
inputSchema);

@Nullable Integer frequency = configuration.getTriggeringFrequencySeconds();

PCollectionRowTuple result =
input
.getSinglePCollection()
.apply("Filter empty paths", Filter.by(row -> row.getString(0) != null))
.apply(
"ExtractPaths",
MapElements.into(TypeDescriptors.strings())
.via(row -> checkStateNotNull(row.getString(0))))
.apply(
new AddFiles(
configuration.getIcebergCatalog(),
configuration.getTable(),
configuration.getLocationPrefix(),
configuration.getPartitionFields(),
configuration.getTableProperties(),
configuration.getAppendBatchSize(),
frequency != null ? Duration.standardSeconds(frequency) : null));

PCollectionRowTuple output = PCollectionRowTuple.of("snapshots", result.get(OUTPUT_TAG));
ErrorHandling errorHandling = configuration.getErrorHandling();
if (errorHandling != null) {
output = output.and(errorHandling.getOutput(), result.get(ERROR_TAG));
}
return output;
}
}
}
Original file line number Diff line number Diff line change
Expand Up @@ -68,10 +68,13 @@ class PartitionUtils {

static PartitionSpec toPartitionSpec(
@Nullable List<String> fields, org.apache.beam.sdk.schemas.Schema beamSchema) {
return toPartitionSpec(fields, IcebergUtils.beamSchemaToIcebergSchema(beamSchema));
}

static PartitionSpec toPartitionSpec(@Nullable List<String> fields, Schema schema) {
if (fields == null) {
return PartitionSpec.unpartitioned();
}
Schema schema = IcebergUtils.beamSchemaToIcebergSchema(beamSchema);
PartitionSpec.Builder builder = PartitionSpec.builderFor(schema);

for (String field : fields) {
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -46,9 +46,11 @@
import org.apache.iceberg.encryption.EncryptedInputFile;
import org.apache.iceberg.expressions.Evaluator;
import org.apache.iceberg.expressions.Expression;
import org.apache.iceberg.expressions.Expressions;
import org.apache.iceberg.hadoop.HadoopInputFile;
import org.apache.iceberg.io.CloseableIterable;
import org.apache.iceberg.io.InputFile;
import org.apache.iceberg.mapping.MappingUtil;
import org.apache.iceberg.mapping.NameMapping;
import org.apache.iceberg.mapping.NameMappingParser;
import org.apache.iceberg.parquet.ParquetReader;
Expand Down Expand Up @@ -112,6 +114,34 @@ static ParquetReader<Record> createReader(FileScanTask task, Table table, Schema
true);
}

static ParquetReader<Record> createReader(InputFile inputFile, Schema schema) {
ParquetReadOptions.Builder optionsBuilder;
if (inputFile instanceof HadoopInputFile) {
// remove read properties already set that may conflict with this read
Configuration conf = new Configuration(((HadoopInputFile) inputFile).getConf());
for (String property : READ_PROPERTIES_TO_REMOVE) {
conf.unset(property);
}
optionsBuilder = HadoopReadOptions.builder(conf);
} else {
optionsBuilder = ParquetReadOptions.builder();
}
optionsBuilder =
optionsBuilder
.withRange(0, inputFile.getLength())
.withMaxAllocationInBytes(MAX_FILE_BUFFER_SIZE);

return new ParquetReader<>(
inputFile,
schema,
optionsBuilder.build(),
fileSchema -> GenericParquetReaders.buildReader(schema, fileSchema),
MappingUtil.create(schema),
Expressions.alwaysTrue(),
false,
true);
}

static Map<Integer, ?> constantsMap(
FileScanTask task,
BiFunction<Type, Object, Object> converter,
Expand Down
Loading
Loading