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 51 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
Expand Up @@ -111,4 +111,4 @@ jobs:
- name: run Cross-Language Wrapper Validation script
uses: ./.github/actions/gradle-command-self-hosted-action
with:
gradle-command: :sdks:python:test-suites:direct:crossLanguageWrapperValidationPreCommit
gradle-command: :sdks:python:test-suites:direct:crossLanguageWrapperValidationPreCommit --info
Original file line number Diff line number Diff line change
Expand Up @@ -111,6 +111,10 @@ message BuilderMethod {
bytes payload = 3;
}

// Payload for a Schema-aware PTransform.
// This is a transform that is aware of its input and output PCollection schemas
// and is configured using Beam Schema-compatible parameters.
// The information available in the payload can be used to instantiate the schema-aware transform.
message SchemaTransformPayload {
// The identifier of the SchemaTransform (typically a URN).
string identifier = 1;
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -37,6 +37,7 @@
import java.util.stream.Collectors;
import javax.annotation.concurrent.Immutable;
import org.apache.beam.sdk.values.Row;
import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.base.CaseFormat;
import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.base.Preconditions;
import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.collect.BiMap;
import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.collect.HashBiMap;
Expand Down Expand Up @@ -326,7 +327,10 @@ public static Schema of(Field... fields) {
return Schema.builder().addFields(fields).build();
}

/** Returns an identical Schema with sorted fields. */
/**
* Returns an identical Schema with lexicographically sorted fields. Recursively sorts nested
* fields.
*/
public Schema sorted() {
// Create a new schema and copy over the appropriate Schema object attributes:
// {fields, uuid, options}
Expand All @@ -336,6 +340,16 @@ public Schema sorted() {
Schema sortedSchema =
this.fields.stream()
.sorted(Comparator.comparing(Field::getName))
.map(
field -> {
FieldType innerType = field.getType();
if (innerType.getRowSchema() != null) {
Schema innerSortedSchema = innerType.getRowSchema().sorted();
innerType = innerType.toBuilder().setRowSchema(innerSortedSchema).build();
return field.toBuilder().setType(innerType).build();
}
return field;
})
.collect(Schema.toSchema())
.withOptions(getOptions());
sortedSchema.setUUID(getUUID());
Expand Down Expand Up @@ -1451,4 +1465,42 @@ public int getFieldCount() {
public Options getOptions() {
return this.options;
}

/** Recursively converts all field names to `lower_snake_case`. */
public Schema toSnakeCase() {
return this.getFields().stream()
.map(
field -> {
FieldType innerType = field.getType();
if (innerType.getRowSchema() != null) {
Schema innerSnakeCaseSchema = innerType.getRowSchema().toSnakeCase();
innerType = innerType.toBuilder().setRowSchema(innerSnakeCaseSchema).build();
field = field.toBuilder().setType(innerType).build();
}
return field
.toBuilder()
.setName(CaseFormat.LOWER_CAMEL.to(CaseFormat.LOWER_UNDERSCORE, field.getName()))
.build();
})
.collect(toSchema());
}

/** Recursively converts all field names to `lowerCamelCase`. */
public Schema toCamelCase() {
return this.getFields().stream()
.map(
field -> {
FieldType innerType = field.getType();
if (innerType.getRowSchema() != null) {
Schema innerCamelCaseSchema = innerType.getRowSchema().toCamelCase();
innerType = innerType.toBuilder().setRowSchema(innerCamelCaseSchema).build();
field = field.toBuilder().setType(innerType).build();
}
return field
.toBuilder()
.setName(CaseFormat.LOWER_UNDERSCORE.to(CaseFormat.LOWER_CAMEL, field.getName()))
.build();
})
.collect(toSchema());
}
}
Original file line number Diff line number Diff line change
Expand Up @@ -53,7 +53,7 @@ public class YamlUtils {
.put(Schema.TypeName.BYTES, str -> BaseEncoding.base64().decode(str))
.build();

public static Row toBeamRow(@Nullable String yamlString, Schema schema) {
public static @Nullable Row toBeamRow(@Nullable String yamlString, Schema schema) {
return toBeamRow(yamlString, schema, false);
}

Expand Down Expand Up @@ -147,14 +147,27 @@ public static Row toBeamRow(
}

@SuppressWarnings("nullness")
public static Row toBeamRow(Map<String, Object> yamlMap, Schema rowSchema, boolean toCamelCase) {
public static Row toBeamRow(
@Nullable Map<String, Object> map, Schema rowSchema, boolean toCamelCase) {
if (map == null || map.isEmpty()) {
List<Field> requiredFields =
rowSchema.getFields().stream()
.filter(field -> !field.getType().getNullable())
.collect(Collectors.toList());
if (requiredFields.isEmpty()) {
return Row.nullRow(rowSchema);
} else {
chamikaramj marked this conversation as resolved.
Show resolved Hide resolved
throw new IllegalArgumentException(
String.format(
"Received an empty Map, but output schema contains required fields: %s",
requiredFields));
}
}
return rowSchema.getFields().stream()
.map(
field ->
toBeamValue(
field,
yamlMap.get(maybeGetSnakeCase(field.getName(), toCamelCase)),
toCamelCase))
field, map.get(maybeGetSnakeCase(field.getName(), toCamelCase)), toCamelCase))
.collect(toRow(rowSchema));
}

Expand Down
40 changes: 40 additions & 0 deletions sdks/java/core/src/main/java/org/apache/beam/sdk/values/Row.java
Original file line number Diff line number Diff line change
Expand Up @@ -893,4 +893,44 @@ public static Row nullRow(Schema schema) {
.addValues(Collections.nCopies(schema.getFieldCount(), null))
.build();
}

/** Returns an identical {@link Row} with fields lexicographically sorted by their name. */
public Row sorted() {
Schema sortedSchema = getSchema().sorted();
return sortedSchema.getFields().stream()
.map(
field -> {
if (field.getType().getRowSchema() != null) {
return ((Row) getValue(field.getName())).sorted();
}
return (Object) getValue(field.getName());
})
.collect(Row.toRow(sortedSchema));
}

/** Returns an equivalent {@link Row} with `lower_snake_case` field names. */
public Row toSnakeCase() {
return getSchema().getFields().stream()
.map(
field -> {
if (field.getType().getRowSchema() != null) {
return ((Row) getValue(field.getName())).toSnakeCase();
}
return (Object) getValue(field.getName());
})
.collect(toRow(getSchema().toSnakeCase()));
}

/** Returns an equivalent {@link Row} with `lowerCamelCase` field names. */
public Row toCamelCase() {
return getSchema().getFields().stream()
.map(
field -> {
if (field.getType().getRowSchema() != null) {
return ((Row) getValue(field.getName())).toCamelCase();
}
return (Object) getValue(field.getName());
})
.collect(toRow(getSchema().toCamelCase()));
}
}
Original file line number Diff line number Diff line change
Expand Up @@ -194,6 +194,71 @@ public void testCollector() {
assertEquals(FieldType.STRING, schema.getField(1).getType());
}

@Test
public void testToSnakeCase() {
Schema innerSchema =
Schema.builder()
.addStringField("myFirstNestedStringField")
.addStringField("mySecondNestedStringField")
.build();
Schema schema =
Schema.builder()
.addStringField("myFirstStringField")
.addStringField("mySecondStringField")
.addRowField("myRowField", innerSchema)
.build();

Schema expectedInnerSnakeCaseSchema =
Schema.builder()
.addStringField("my_first_nested_string_field")
.addStringField("my_second_nested_string_field")
.build();
Schema expectedSnakeCaseSchema =
Schema.builder()
.addStringField("my_first_string_field")
.addStringField("my_second_string_field")
.addRowField("my_row_field", expectedInnerSnakeCaseSchema)
.build();

assertEquals(
expectedInnerSnakeCaseSchema,
schema.toSnakeCase().getField("my_row_field").getType().getRowSchema());
assertEquals(expectedSnakeCaseSchema, schema.toSnakeCase());
}

@Test
public void testToCamelCase() {
Schema innerSchema =
Schema.builder()
.addStringField("my_first_nested_string_field")
.addStringField("my_second_nested_string_field")
.build();
Schema schema =
Schema.builder()
.addStringField("my_first_string_field")
.addStringField("my_second_string_field")
.addRowField("my_row_field", innerSchema)
.build();

Schema expectedInnerCamelCaseSchema =
Schema.builder()
.addStringField("myFirstNestedStringField")
.addStringField("mySecondNestedStringField")
.build();
Schema expectedCamelCaseSchema =
Schema.builder()
.addStringField("myFirstStringField")
.addStringField("mySecondStringField")
.addRowField("myRowField", expectedInnerCamelCaseSchema)
.build();

assertTrue(schema.toCamelCase().hasField("myRowField"));
assertEquals(
expectedInnerCamelCaseSchema,
schema.toCamelCase().getField("myRowField").getType().getRowSchema());
assertEquals(expectedCamelCaseSchema, schema.toCamelCase());
}

@Test
public void testSorted() {
Options testOptions =
Expand Down Expand Up @@ -232,6 +297,41 @@ public void testSorted() {
sortedSchema.getEncodingPositions()));
}

@Test
public void testNestedSorted() {
Schema unsortedNestedSchema =
Schema.builder().addStringField("bb").addInt32Field("aa").addStringField("cc").build();
Schema unsortedSchema =
Schema.builder()
.addStringField("d")
.addInt32Field("c")
.addRowField("e", unsortedNestedSchema)
.addStringField("b")
.addByteField("a")
.build();

Schema sortedSchema = unsortedSchema.sorted();

Schema expectedInnerSortedSchema =
Schema.builder().addInt32Field("aa").addStringField("bb").addStringField("cc").build();
Schema expectedSortedSchema =
Schema.builder()
.addByteField("a")
.addStringField("b")
.addInt32Field("c")
.addStringField("d")
.addRowField("e", expectedInnerSortedSchema)
.build();

assertTrue(unsortedSchema.equivalent(sortedSchema));
assertEquals(expectedSortedSchema.getFields(), sortedSchema.getFields());
assertEquals(expectedSortedSchema.getEncodingPositions(), sortedSchema.getEncodingPositions());
assertEquals(expectedInnerSortedSchema, sortedSchema.getField("e").getType().getRowSchema());
assertEquals(
expectedInnerSortedSchema.getEncodingPositions(),
sortedSchema.getField("e").getType().getRowSchema().getEncodingPositions());
}

@Test
public void testSortedMethodIncludesAllSchemaFields() {
// This test is most likely to break when new Schema object attributes are added. It is designed
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -21,11 +21,13 @@

import java.math.BigDecimal;
import java.util.Arrays;
import java.util.Map;
import java.util.stream.Collectors;
import java.util.stream.IntStream;
import org.apache.beam.sdk.schemas.Schema;
import org.apache.beam.sdk.schemas.utils.YamlUtils;
import org.apache.beam.sdk.values.Row;
import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.base.CaseFormat;
import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.io.BaseEncoding;
import org.junit.Rule;
import org.junit.Test;
Expand Down Expand Up @@ -225,4 +227,35 @@ public void testNestedArray() {

assertEquals(expectedRow, YamlUtils.toBeamRow(yamlString, schema));
}

private static final Schema FLAT_SCHEMA_CAMEL_CASE =
Schema.builder()
.addFields(
FLAT_SCHEMA.getFields().stream()
.map(
field ->
field.withName(
CaseFormat.LOWER_UNDERSCORE.to(
CaseFormat.LOWER_CAMEL, field.getName())))
.collect(Collectors.toList()))
.build();

private static final Map<String, Object> FLAT_MAP =
FLAT_SCHEMA.getFields().stream()
.collect(
Collectors.toMap(
Schema.Field::getName,
field -> Preconditions.checkArgumentNotNull(FLAT_ROW.getValue(field.getName()))));

@Test
public void testSnakeCaseMapToCamelCaseRow() {
Row expectedRow =
FLAT_SCHEMA.getFields().stream()
.map(field -> Preconditions.checkStateNotNull(FLAT_ROW.getValue(field.getName())))
.collect(Row.toRow(FLAT_SCHEMA_CAMEL_CASE));

Row convertedRow = YamlUtils.toBeamRow(FLAT_MAP, FLAT_SCHEMA_CAMEL_CASE, true);

assertEquals(expectedRow, convertedRow);
}
}
Loading
Loading