Skip to content
New issue

Have a question about this project? Sign up for a free GitHub account to open an issue and contact its maintainers and the community.

By clicking “Sign up for GitHub”, you agree to our terms of service and privacy statement. We’ll occasionally send you account related emails.

Already on GitHub? Sign in to your account

Re-add iceberg bounded source; test splitting #30805

Merged
merged 3 commits into from
Apr 12, 2024
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.
Jump to
Jump to file
Failed to load files.
Diff view
Diff view
15 changes: 4 additions & 11 deletions sdks/java/io/iceberg/build.gradle
Original file line number Diff line number Diff line change
Expand Up @@ -43,27 +43,20 @@ def hive_version = "3.1.3"
dependencies {
implementation library.java.vendored_guava_32_1_2_jre
implementation project(path: ":sdks:java:core", configuration: "shadow")
implementation project(":sdks:java:io:hadoop-common")
implementation library.java.slf4j_api
implementation library.java.joda_time
implementation "org.apache.parquet:parquet-column:$parquet_version"
implementation "org.apache.parquet:parquet-common:$parquet_version"
implementation "org.apache.orc:orc-core:$orc_version"
implementation "org.apache.iceberg:iceberg-core:$iceberg_version"
implementation "org.apache.iceberg:iceberg-api:$iceberg_version"
implementation "org.apache.iceberg:iceberg-parquet:$iceberg_version"
implementation "org.apache.iceberg:iceberg-orc:$iceberg_version"
implementation "org.apache.iceberg:iceberg-arrow:$iceberg_version"
implementation "org.apache.iceberg:iceberg-data:$iceberg_version"
implementation library.java.avro
implementation library.java.hadoop_common



provided library.java.avro
provided library.java.hadoop_client
permitUnusedDeclared library.java.hadoop_client
provided library.java.hadoop_common
testImplementation library.java.hadoop_client

testImplementation "org.apache.iceberg:iceberg-gcp:$iceberg_version"
testImplementation "org.apache.iceberg:iceberg-data:$iceberg_version"
testImplementation project(path: ":sdks:java:core", configuration: "shadowTest")
testImplementation library.java.junit
testRuntimeOnly library.java.slf4j_jdk14
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -17,9 +17,16 @@
*/
package org.apache.beam.io.iceberg;

import static org.apache.beam.sdk.util.Preconditions.checkStateNotNull;

import org.apache.beam.sdk.io.Read;
import org.apache.beam.sdk.transforms.PTransform;
import org.apache.beam.sdk.values.PBegin;
import org.apache.beam.sdk.values.PCollection;
import org.apache.beam.sdk.values.Row;
import org.apache.iceberg.Table;
import org.apache.iceberg.catalog.TableIdentifier;
import org.checkerframework.checker.nullness.qual.Nullable;

public class IcebergIO {
Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Seems like we are missing the Read transform ?

Copy link
Member Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Will do soon. I realize all the tests are a vanilla Read.from(IcebergBoundedSource)

Copy link
Member Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

done


Expand All @@ -28,6 +35,10 @@ public static WriteRows writeToDynamicDestinations(
return new WriteRows(catalog, dynamicDestinations);
}

public static ReadTable readTable(IcebergCatalogConfig catalogConfig, TableIdentifier tableId) {
return new ReadTable(catalogConfig, tableId);
}

static class WriteRows extends PTransform<PCollection<Row>, IcebergWriteResult> {

private final IcebergCatalogConfig catalog;
Expand All @@ -47,4 +58,36 @@ public IcebergWriteResult expand(PCollection<Row> input) {
"Write Rows to Destinations", new WriteToDestinations(catalog, dynamicDestinations));
}
}

public static class ReadTable extends PTransform<PBegin, PCollection<Row>> {

private final IcebergCatalogConfig catalogConfig;
private final transient @Nullable TableIdentifier tableId;

private TableIdentifier getTableId() {
return checkStateNotNull(
tableId, "Transient field tableId null; it should not be accessed after serialization");
}

private ReadTable(IcebergCatalogConfig catalogConfig, TableIdentifier tableId) {
this.catalogConfig = catalogConfig;
this.tableId = tableId;
}

@Override
public PCollection<Row> expand(PBegin input) {

Table table = catalogConfig.catalog().loadTable(getTableId());

return input.apply(
Read.from(
new ScanSource(
IcebergScanConfig.builder()
.setCatalogConfig(catalogConfig)
.setScanType(IcebergScanConfig.ScanType.TABLE)
.setTableIdentifier(getTableId())
.setSchema(SchemaAndRowConversions.icebergSchemaToBeamSchema(table.schema()))
.build())));
}
}
}
Original file line number Diff line number Diff line change
@@ -0,0 +1,166 @@
/*
* 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.io.iceberg;

import com.google.auto.value.AutoValue;
import java.io.Serializable;
import org.apache.beam.sdk.schemas.Schema;
import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.collect.ImmutableMap;
import org.apache.iceberg.Table;
import org.apache.iceberg.catalog.TableIdentifier;
import org.apache.iceberg.expressions.Expression;
import org.checkerframework.checker.nullness.qual.MonotonicNonNull;
import org.checkerframework.checker.nullness.qual.Nullable;
import org.checkerframework.dataflow.qual.Pure;

@AutoValue
public abstract class IcebergScanConfig implements Serializable {

private transient @MonotonicNonNull Table cachedTable;

public enum ScanType {
TABLE,
BATCH
}

@Pure
public abstract ScanType getScanType();

@Pure
public abstract IcebergCatalogConfig getCatalogConfig();

@Pure
public abstract String getTableIdentifier();

@Pure
public Table getTable() {
if (cachedTable == null) {
cachedTable =
getCatalogConfig().catalog().loadTable(TableIdentifier.parse(getTableIdentifier()));
}
return cachedTable;
}

@Pure
public abstract Schema getSchema();

@Pure
public abstract @Nullable Expression getFilter();

@Pure
public abstract @Nullable Boolean getCaseSensitive();

@Pure
public abstract ImmutableMap<String, String> getOptions();

@Pure
public abstract @Nullable Long getSnapshot();

@Pure
public abstract @Nullable Long getTimestamp();

@Pure
public abstract @Nullable Long getFromSnapshotInclusive();

@Pure
public abstract @Nullable String getFromSnapshotRefInclusive();

@Pure
public abstract @Nullable Long getFromSnapshotExclusive();

@Pure
public abstract @Nullable String getFromSnapshotRefExclusive();

@Pure
public abstract @Nullable Long getToSnapshot();

@Pure
public abstract @Nullable String getToSnapshotRef();

@Pure
public abstract @Nullable String getTag();

@Pure
public abstract @Nullable String getBranch();

@Pure
public static Builder builder() {
return new AutoValue_IcebergScanConfig.Builder()
.setScanType(ScanType.TABLE)
.setFilter(null)
.setCaseSensitive(null)
.setOptions(ImmutableMap.of())
.setSnapshot(null)
.setTimestamp(null)
.setFromSnapshotInclusive(null)
.setFromSnapshotRefInclusive(null)
.setFromSnapshotExclusive(null)
.setFromSnapshotRefExclusive(null)
.setToSnapshot(null)
.setToSnapshotRef(null)
.setTag(null)
.setBranch(null);
}

@AutoValue.Builder
public abstract static class Builder {
public abstract Builder setScanType(ScanType type);

public abstract Builder setCatalogConfig(IcebergCatalogConfig catalog);

public abstract Builder setTableIdentifier(String tableIdentifier);

public Builder setTableIdentifier(TableIdentifier tableIdentifier) {
return this.setTableIdentifier(tableIdentifier.toString());
}

public Builder setTableIdentifier(String... names) {
return setTableIdentifier(TableIdentifier.of(names));
}

public abstract Builder setSchema(Schema schema);

public abstract Builder setFilter(@Nullable Expression filter);

public abstract Builder setCaseSensitive(@Nullable Boolean caseSensitive);

public abstract Builder setOptions(ImmutableMap<String, String> options);

public abstract Builder setSnapshot(@Nullable Long snapshot);

public abstract Builder setTimestamp(@Nullable Long timestamp);

public abstract Builder setFromSnapshotInclusive(@Nullable Long fromInclusive);

public abstract Builder setFromSnapshotRefInclusive(@Nullable String ref);

public abstract Builder setFromSnapshotExclusive(@Nullable Long fromExclusive);

public abstract Builder setFromSnapshotRefExclusive(@Nullable String ref);

public abstract Builder setToSnapshot(@Nullable Long snapshot);

public abstract Builder setToSnapshotRef(@Nullable String ref);

public abstract Builder setTag(@Nullable String tag);

public abstract Builder setBranch(@Nullable String branch);

public abstract IcebergScanConfig build();
}
}
Original file line number Diff line number Diff line change
Expand Up @@ -17,7 +17,7 @@
*/
package org.apache.beam.io.iceberg;

import static org.apache.beam.io.iceberg.RowHelper.rowToRecord;
import static org.apache.beam.io.iceberg.SchemaAndRowConversions.rowToRecord;

import java.io.IOException;
import org.apache.beam.sdk.values.Row;
Expand All @@ -30,7 +30,6 @@
import org.apache.iceberg.data.parquet.GenericParquetWriter;
import org.apache.iceberg.io.DataWriter;
import org.apache.iceberg.io.OutputFile;
import org.apache.iceberg.orc.ORC;
import org.apache.iceberg.parquet.Parquet;

class RecordWriter {
Expand Down

This file was deleted.