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

Managed Transform protos & translation; Iceberg SchemaTransforms & translation #30910

Merged
merged 66 commits into from
Apr 22, 2024
Merged
Show file tree
Hide file tree
Changes from 2 commits
Commits
Show all changes
66 commits
Select commit Hold shift + click to select a range
42611e0
iceberg write schematransform and test
ahmedabu98 Apr 9, 2024
16e6235
cleanup
ahmedabu98 Apr 9, 2024
ed72898
IcebergIO translation and tests
ahmedabu98 Apr 11, 2024
1738345
add sanity check for building with Row; add documentation about outpu…
ahmedabu98 Apr 11, 2024
364ebbe
spotless
ahmedabu98 Apr 11, 2024
79d2c94
spotless
ahmedabu98 Apr 11, 2024
30de265
permitUnusedDeclared iceberg
ahmedabu98 Apr 11, 2024
905d590
Change ManagedSchemaTransformProvider to take a Row config instead of…
ahmedabu98 Apr 11, 2024
553281f
Merge branch 'master' of https://github.com/ahmedabu98/beam into iceb…
ahmedabu98 Apr 11, 2024
2c733ec
Merge branch 'master' of https://github.com/ahmedabu98/beam into iceb…
ahmedabu98 Apr 11, 2024
1067d84
don't auto generate external wrapper for this just yet
ahmedabu98 Apr 11, 2024
6db699a
spotless
ahmedabu98 Apr 11, 2024
301e388
spotless
ahmedabu98 Apr 11, 2024
f1576e3
Merge branch 'master' of https://github.com/ahmedabu98/beam into iceb…
ahmedabu98 Apr 14, 2024
04cc2db
Merge branch 'master' of https://github.com/ahmedabu98/beam into iceb…
ahmedabu98 Apr 15, 2024
27e5fb0
Read schematransform and tests
ahmedabu98 Apr 15, 2024
e2cb93b
Merge branch 'iceberg_translation' of https://github.com/ahmedabu98/b…
ahmedabu98 Apr 15, 2024
aa8b1ed
pulling in IcebergIO changes; spotless
ahmedabu98 Apr 15, 2024
6823524
Merge branch 'managed_row_config' of https://github.com/ahmedabu98/be…
ahmedabu98 Apr 15, 2024
0674069
icebergio translation; managed translation; protos
ahmedabu98 Apr 16, 2024
9034cee
spotless
ahmedabu98 Apr 16, 2024
0bc3779
spotless; use underscore instead of camel case field names when trans…
ahmedabu98 Apr 16, 2024
27b5e6a
add grpc dependency
ahmedabu98 Apr 16, 2024
05e3a1a
updated proto description; fix gen xlang command
ahmedabu98 Apr 17, 2024
2b49c37
ManagedTransform explicit input/output types; move iceberg package to…
ahmedabu98 Apr 17, 2024
7de01bb
externalizable IcebergCatalogConfig
ahmedabu98 Apr 17, 2024
6f739f4
externalizable IcebergCatalogConfig supports all properties; address …
ahmedabu98 Apr 17, 2024
1df6aa7
unify iceberg urns and identifiers; update some comments
ahmedabu98 Apr 18, 2024
6d7a353
one source for all supported managed transform identifiers
ahmedabu98 Apr 18, 2024
ea5a34d
add documentation
ahmedabu98 Apr 18, 2024
642da92
custom serialization for OneTableDynamicDestinations
ahmedabu98 Apr 18, 2024
5a2fd87
add iceberg via managed API tests; update proto doc
ahmedabu98 Apr 18, 2024
804fdac
rename config; change test schematransform location
ahmedabu98 Apr 18, 2024
c301cfc
spotless
ahmedabu98 Apr 18, 2024
841fd43
add missing package-info file
ahmedabu98 Apr 18, 2024
928eead
spotless
ahmedabu98 Apr 18, 2024
4f12cbd
replace icebergIO translation with iceberg schematransform translatio…
ahmedabu98 Apr 18, 2024
9b26b18
remove ExternalizableIcebergCatalogConfig (no longer needed)
ahmedabu98 Apr 18, 2024
d912a22
pull identifiers from generated proto
ahmedabu98 Apr 18, 2024
523fc69
remove unused hadoop dependency
ahmedabu98 Apr 18, 2024
cb89d16
update generate sequence wrapper after Schema sorting
ahmedabu98 Apr 18, 2024
17ecec6
managed transform translation uses default schema
ahmedabu98 Apr 19, 2024
108a12c
yaml returns null row; cleanup
ahmedabu98 Apr 19, 2024
5923a97
spotless
ahmedabu98 Apr 19, 2024
8ab4c0c
remove SchemaAwareTransformPayload and use SchemaTransformPayload ins…
ahmedabu98 Apr 19, 2024
3426360
create a beam-schema-compatible class for Snapshot info
ahmedabu98 Apr 19, 2024
49a1596
removed new proto file and moved Managed URNs to beam_runner_api.prot…
ahmedabu98 Apr 19, 2024
52a308c
Row and Schema snake_case <-> camelCase conversion logic
ahmedabu98 Apr 19, 2024
d9fcb06
Row sorted() util
ahmedabu98 Apr 19, 2024
11354c8
use Row::sorted to fetch Managed & Iceberg row configs
ahmedabu98 Apr 19, 2024
2992192
use snake_case convention when translating transforms to spec; remove…
ahmedabu98 Apr 19, 2024
b311068
spotless
ahmedabu98 Apr 19, 2024
2461b44
cleanup
ahmedabu98 Apr 20, 2024
ecb4dbb
DefaultSchemaProvider can now provide the underlying SchemaProvider
ahmedabu98 Apr 20, 2024
68895a7
perform snake_case <-> camelCase conversions directly in TypedSchemaT…
ahmedabu98 Apr 20, 2024
d2135b8
update icebergIO and managed translations to reflect field name conve…
ahmedabu98 Apr 20, 2024
64863ce
sorted SnapshotInfo
ahmedabu98 Apr 20, 2024
5afb633
update manual Python wrappers to use snake_case convention; remove ca…
ahmedabu98 Apr 20, 2024
2ddd5bb
Row utils allow nullable
ahmedabu98 Apr 20, 2024
d5a4d66
add FileWriteResult test for version number; fix existing Java and YA…
ahmedabu98 Apr 22, 2024
3b74f77
add schema-aware transform urn to transform annotations during transl…
ahmedabu98 Apr 22, 2024
af65032
add comments why we sort and snake_case configuration schemas
ahmedabu98 Apr 22, 2024
7130e56
add SchemaTransformTranslation abstraction. when encountering a SCHEM…
ahmedabu98 Apr 22, 2024
de81e60
add documentation
ahmedabu98 Apr 22, 2024
34dc371
prioritize registered providers; remove snake_case <-> camelCase conv…
ahmedabu98 Apr 22, 2024
82b481d
cleanup
ahmedabu98 Apr 22, 2024
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,216 @@
/*
* 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 static org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.base.Preconditions.checkArgument;

import com.google.auto.service.AutoService;
import com.google.auto.value.AutoValue;
import java.util.*;
import org.apache.beam.io.iceberg.IcebergWriteSchemaTransformProvider.Config;
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.transforms.SchemaTransform;
import org.apache.beam.sdk.schemas.transforms.SchemaTransformProvider;
import org.apache.beam.sdk.schemas.transforms.TypedSchemaTransformProvider;
import org.apache.beam.sdk.transforms.MapElements;
import org.apache.beam.sdk.transforms.SimpleFunction;
import org.apache.beam.sdk.util.Preconditions;
import org.apache.beam.sdk.values.KV;
import org.apache.beam.sdk.values.PCollection;
import org.apache.beam.sdk.values.PCollectionRowTuple;
import org.apache.beam.sdk.values.Row;
import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.annotations.VisibleForTesting;
import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.base.Strings;
import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.collect.Sets;
import org.apache.iceberg.CatalogUtil;
import org.apache.iceberg.Snapshot;
import org.apache.iceberg.catalog.TableIdentifier;
import org.checkerframework.checker.nullness.qual.Nullable;

@AutoService(SchemaTransformProvider.class)
public class IcebergWriteSchemaTransformProvider extends TypedSchemaTransformProvider<Config> {

static final String INPUT_TAG = "input";
static final String OUTPUT_TAG = "output";

@Override
protected SchemaTransform from(Config configuration) {
ahmedabu98 marked this conversation as resolved.
Show resolved Hide resolved
configuration.validate();
return new IcebergWriteSchemaTransform(configuration);
}

@Override
public List<String> inputCollectionNames() {
return Collections.singletonList(INPUT_TAG);
}

@Override
public List<String> outputCollectionNames() {
return Collections.singletonList(OUTPUT_TAG);
}

@Override
public String identifier() {
return "beam:schematransform:org.apache.beam:iceberg_write:v1";
chamikaramj marked this conversation as resolved.
Show resolved Hide resolved
}

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

public abstract String getTable();

public abstract CatalogConfig getCatalogConfig();

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

public abstract Builder setCatalogConfig(CatalogConfig catalogConfig);

public abstract Config build();
}

public void validate() {
getCatalogConfig().validate();
}
}

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

public abstract String getCatalogName();

public abstract @Nullable String getCatalogType();

public abstract @Nullable String getCatalogImplementation();

public abstract @Nullable String getWarehouseLocation();

@AutoValue.Builder
public abstract static class Builder {

public abstract Builder setCatalogName(String catalogName);

public abstract Builder setCatalogType(String catalogType);

public abstract Builder setCatalogImplementation(String catalogImplementation);

public abstract Builder setWarehouseLocation(String warehouseLocation);

public abstract CatalogConfig build();
}

Set<String> validTypes =
Sets.newHashSet(
CatalogUtil.ICEBERG_CATALOG_TYPE_HADOOP,
CatalogUtil.ICEBERG_CATALOG_TYPE_HIVE,
CatalogUtil.ICEBERG_CATALOG_TYPE_REST);

public void validate() {
if (Strings.isNullOrEmpty(getCatalogType())) {
checkArgument(
validTypes.contains(Preconditions.checkArgumentNotNull(getCatalogType())),
"Invalid catalog type. Please pick one of %s",
validTypes);
}
}
}

@VisibleForTesting
static class IcebergWriteSchemaTransform extends SchemaTransform {
private final Config configuration;

IcebergWriteSchemaTransform(Config configuration) {
this.configuration = configuration;
}

@Override
public PCollectionRowTuple expand(PCollectionRowTuple input) {

PCollection<Row> rows = input.get(INPUT_TAG);

CatalogConfig catalogConfig = configuration.getCatalogConfig();

IcebergCatalogConfig.Builder catalogBuilder =
IcebergCatalogConfig.builder()
.setName(catalogConfig.getCatalogName())
.setIcebergCatalogType(catalogConfig.getCatalogType())
.setWarehouseLocation(catalogConfig.getWarehouseLocation());

if (!Strings.isNullOrEmpty(catalogConfig.getCatalogType())) {
catalogBuilder = catalogBuilder.setIcebergCatalogType(catalogConfig.getCatalogType());
}
if (!Strings.isNullOrEmpty(catalogConfig.getWarehouseLocation())) {
catalogBuilder = catalogBuilder.setWarehouseLocation(catalogConfig.getWarehouseLocation());
}

// TODO: support dynamic destinations
DynamicDestinations dynamicDestinations =
DynamicDestinations.singleTable(TableIdentifier.parse(configuration.getTable()));

IcebergWriteResult result =
rows.apply(
IcebergIO.writeToDynamicDestinations(catalogBuilder.build(), dynamicDestinations));

PCollection<Row> snapshots =
result
.getSnapshots()
.apply(MapElements.via(new SnapshotToRow()))
.setRowSchema(SnapshotToRow.SNAPSHOT_SCHEMA);

return PCollectionRowTuple.of(OUTPUT_TAG, snapshots);
}

@VisibleForTesting
static class SnapshotToRow extends SimpleFunction<KV<String, Snapshot>, Row> {
static final Schema SNAPSHOT_SCHEMA =
ahmedabu98 marked this conversation as resolved.
Show resolved Hide resolved
Schema.builder()
.addStringField("table")
.addStringField("operation")
.addMapField("summary", Schema.FieldType.STRING, Schema.FieldType.STRING)
.addStringField("manifestListLocation")
.build();

@Override
public Row apply(KV<String, Snapshot> input) {
Snapshot snapshot = input.getValue();
Row row =
Row.withSchema(SNAPSHOT_SCHEMA)
.addValues(
input.getKey(),
snapshot.operation(),
snapshot.summary(),
snapshot.manifestListLocation())
.build();
System.out.println("SNAPSHOT: " + snapshot);
ahmedabu98 marked this conversation as resolved.
Show resolved Hide resolved
System.out.println("ROW: " + row);
return row;
}
}
}
}
Original file line number Diff line number Diff line change
@@ -0,0 +1,121 @@
/*
* 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 static org.apache.beam.io.iceberg.IcebergWriteSchemaTransformProvider.CatalogConfig;
import static org.apache.beam.io.iceberg.IcebergWriteSchemaTransformProvider.Config;
import static org.apache.beam.io.iceberg.IcebergWriteSchemaTransformProvider.INPUT_TAG;
import static org.apache.beam.io.iceberg.IcebergWriteSchemaTransformProvider.OUTPUT_TAG;
import static org.hamcrest.MatcherAssert.assertThat;
import static org.junit.Assert.assertEquals;

import java.util.List;
import java.util.UUID;
import org.apache.beam.sdk.testing.PAssert;
import org.apache.beam.sdk.testing.TestPipeline;
import org.apache.beam.sdk.transforms.Create;
import org.apache.beam.sdk.transforms.SerializableFunction;
import org.apache.beam.sdk.values.PCollection;
import org.apache.beam.sdk.values.PCollectionRowTuple;
import org.apache.beam.sdk.values.Row;
import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.collect.ImmutableList;
import org.apache.iceberg.CatalogUtil;
import org.apache.iceberg.Table;
import org.apache.iceberg.catalog.TableIdentifier;
import org.apache.iceberg.data.IcebergGenerics;
import org.apache.iceberg.data.Record;
import org.hamcrest.Matchers;
import org.junit.ClassRule;
import org.junit.Rule;
import org.junit.Test;
import org.junit.rules.TemporaryFolder;
import org.junit.runner.RunWith;
import org.junit.runners.JUnit4;

@RunWith(JUnit4.class)
public class IcebergWriteSchemaTransformProviderTest {

@ClassRule public static final TemporaryFolder TEMPORARY_FOLDER = new TemporaryFolder();

@Rule
public transient TestDataWarehouse warehouse = new TestDataWarehouse(TEMPORARY_FOLDER, "default");

@Rule public transient TestPipeline testPipeline = TestPipeline.create();

@Test
public void testSimpleAppend() {
String identifier = "default.table_" + Long.toString(UUID.randomUUID().hashCode(), 16);

TableIdentifier tableId = TableIdentifier.parse(identifier);
System.out.println(tableId);

// Create a table and add records to it.
Table table = warehouse.createTable(tableId, TestFixtures.SCHEMA);

Config config =
Config.builder()
.setTable(identifier)
.setCatalogConfig(
CatalogConfig.builder()
.setCatalogName("hadoop")
.setCatalogType(CatalogUtil.ICEBERG_CATALOG_TYPE_HADOOP)
.setWarehouseLocation(warehouse.location)
.build())
.build();

PCollectionRowTuple input =
PCollectionRowTuple.of(
INPUT_TAG,
testPipeline
.apply(
"Records To Add", Create.of(TestFixtures.asRows(TestFixtures.FILE1SNAPSHOT1)))
.setRowSchema(SchemaHelper.convert(TestFixtures.SCHEMA)));

PCollection<Row> result =
input
.apply("Append To Table", new IcebergWriteSchemaTransformProvider().from(config))
.get(OUTPUT_TAG);

PAssert.that(result).satisfies(new VerifyOutputs(identifier, "append"));

testPipeline.run().waitUntilFinish();

List<Record> writtenRecords = ImmutableList.copyOf(IcebergGenerics.read(table).build());

assertThat(writtenRecords, Matchers.containsInAnyOrder(TestFixtures.FILE1SNAPSHOT1.toArray()));
}

private static class VerifyOutputs implements SerializableFunction<Iterable<Row>, Void> {
private final String tableId;
private final String operation;

public VerifyOutputs(String identifier, String operation) {
this.tableId = identifier;
this.operation = operation;
}

@Override
public Void apply(Iterable<Row> input) {
Row row = input.iterator().next();

assertEquals(tableId, row.getString("table"));
assertEquals(operation, row.getString("operation"));
return null;
}
}
}
Loading