-
Notifications
You must be signed in to change notification settings - Fork 4.6k
[IcebergIO] Support hash distribution mode when writing rows #38061
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
Changes from 8 commits
6c2546f
0795912
3c29bbd
97a71f8
5ee5278
5c55fc7
7696e1e
fa5ccda
12f67d6
e0f8cd5
945783d
9ada7da
091360d
1beac3b
6e35081
b62e3a4
65297d3
File filter
Filter by extension
Conversations
Jump to
Diff view
Diff view
There are no files selected for viewing
| Original file line number | Diff line number | Diff line change |
|---|---|---|
| @@ -0,0 +1,139 @@ | ||
| /* | ||
| * 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.util.Preconditions.checkStateNotNull; | ||
|
|
||
| import java.util.HashMap; | ||
| import java.util.Map; | ||
| import org.apache.beam.sdk.coders.KvCoder; | ||
| import org.apache.beam.sdk.coders.RowCoder; | ||
| import org.apache.beam.sdk.transforms.DoFn; | ||
| import org.apache.beam.sdk.transforms.PTransform; | ||
| import org.apache.beam.sdk.transforms.ParDo; | ||
| import org.apache.beam.sdk.transforms.windowing.BoundedWindow; | ||
| import org.apache.beam.sdk.transforms.windowing.PaneInfo; | ||
| import org.apache.beam.sdk.values.KV; | ||
| import org.apache.beam.sdk.values.PCollection; | ||
| import org.apache.beam.sdk.values.Row; | ||
| import org.apache.beam.sdk.values.ValueInSingleWindow; | ||
| import org.apache.iceberg.PartitionKey; | ||
| import org.apache.iceberg.PartitionSpec; | ||
| import org.apache.iceberg.Schema; | ||
| import org.apache.iceberg.catalog.TableIdentifier; | ||
| import org.apache.iceberg.exceptions.NoSuchTableException; | ||
| import org.checkerframework.checker.nullness.qual.MonotonicNonNull; | ||
| import org.checkerframework.checker.nullness.qual.Nullable; | ||
| import org.joda.time.Instant; | ||
|
|
||
| /** | ||
| * Assigns destination metadata for each input record. | ||
| * | ||
| * <p>The output will have the format { {destination, partition}, data } | ||
| */ | ||
| class AssignDestinationsAndPartitions | ||
| extends PTransform<PCollection<Row>, PCollection<KV<Row, Row>>> { | ||
|
|
||
| private final DynamicDestinations dynamicDestinations; | ||
| private final IcebergCatalogConfig catalogConfig; | ||
| static final String DESTINATION = "destination"; | ||
| static final String PARTITION = "partition"; | ||
| static final org.apache.beam.sdk.schemas.Schema OUTPUT_SCHEMA = | ||
| org.apache.beam.sdk.schemas.Schema.builder() | ||
| .addStringField(DESTINATION) | ||
| .addStringField(PARTITION) | ||
| .build(); | ||
|
|
||
| public AssignDestinationsAndPartitions( | ||
| DynamicDestinations dynamicDestinations, IcebergCatalogConfig catalogConfig) { | ||
| this.dynamicDestinations = dynamicDestinations; | ||
| this.catalogConfig = catalogConfig; | ||
| } | ||
|
|
||
| @Override | ||
| public PCollection<KV<Row, Row>> expand(PCollection<Row> input) { | ||
| return input | ||
| .apply(ParDo.of(new AssignDoFn(dynamicDestinations, catalogConfig))) | ||
| .setCoder( | ||
| KvCoder.of( | ||
| RowCoder.of(OUTPUT_SCHEMA), RowCoder.of(dynamicDestinations.getDataSchema()))); | ||
| } | ||
|
|
||
| static class AssignDoFn extends DoFn<Row, KV<Row, Row>> { | ||
| private final Map<String, PartitionKey> partitionKeys = new HashMap<>(); | ||
| private transient @MonotonicNonNull Map<String, BeamRowWrapper> wrappers; | ||
| private final DynamicDestinations dynamicDestinations; | ||
| private final IcebergCatalogConfig catalogConfig; | ||
|
|
||
| AssignDoFn(DynamicDestinations dynamicDestinations, IcebergCatalogConfig catalogConfig) { | ||
| this.dynamicDestinations = dynamicDestinations; | ||
| this.catalogConfig = catalogConfig; | ||
| } | ||
|
|
||
| @Setup | ||
| public void setup() { | ||
| this.wrappers = new HashMap<>(); | ||
|
ahmedabu98 marked this conversation as resolved.
|
||
| } | ||
|
|
||
| @ProcessElement | ||
| public void processElement( | ||
| @Element Row element, | ||
| BoundedWindow window, | ||
| PaneInfo paneInfo, | ||
| @Timestamp Instant timestamp, | ||
| OutputReceiver<KV<Row, Row>> out) { | ||
| String tableIdentifier = | ||
| dynamicDestinations.getTableStringIdentifier( | ||
| ValueInSingleWindow.of(element, timestamp, window, paneInfo)); | ||
| Row data = dynamicDestinations.getData(element); | ||
|
|
||
| @Nullable PartitionKey partitionKey = partitionKeys.get(tableIdentifier); | ||
|
ahmedabu98 marked this conversation as resolved.
Outdated
|
||
| @Nullable BeamRowWrapper wrapper = checkStateNotNull(wrappers).get(tableIdentifier); | ||
| if (partitionKey == null || wrapper == null) { | ||
| PartitionSpec spec = PartitionSpec.unpartitioned(); | ||
|
Contributor
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. Is this guaranteed to be unique across bundles ?
Contributor
Author
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. Not always. If they provide partition fields then yes. Otherwise we fetch the spec from the table, which can change between one bundle and another. |
||
| Schema schema = IcebergUtils.beamSchemaToIcebergSchema(data.getSchema()); | ||
| @Nullable | ||
| IcebergTableCreateConfig createConfig = | ||
| dynamicDestinations.instantiateDestination(tableIdentifier).getTableCreateConfig(); | ||
| if (createConfig != null && createConfig.getPartitionFields() != null) { | ||
| spec = | ||
| PartitionUtils.toPartitionSpec(createConfig.getPartitionFields(), data.getSchema()); | ||
| } else { | ||
| try { | ||
| // see if table already exists with a spec | ||
| // TODO(ahmedabu98): improve this by periodically refreshing the table to fetch updated | ||
|
Contributor
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. Add a Github issue to the TODO ?
Contributor
Author
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. Done |
||
| // specs | ||
| spec = catalogConfig.catalog().loadTable(TableIdentifier.parse(tableIdentifier)).spec(); | ||
|
Contributor
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. Calling |
||
| } catch (NoSuchTableException ignored) { | ||
| // no partition to apply | ||
| } | ||
| } | ||
| partitionKey = new PartitionKey(spec, schema); | ||
| wrapper = new BeamRowWrapper(data.getSchema(), schema.asStruct()); | ||
| partitionKeys.put(tableIdentifier, partitionKey); | ||
|
ahmedabu98 marked this conversation as resolved.
Outdated
|
||
| checkStateNotNull(wrappers).put(tableIdentifier, wrapper); | ||
| } | ||
| partitionKey.partition(wrapper.wrap(data)); | ||
| String partitionPath = partitionKey.toPath(); | ||
|
|
||
| Row destAndPartition = | ||
| Row.withSchema(OUTPUT_SCHEMA).addValues(tableIdentifier, partitionPath).build(); | ||
| out.output(KV.of(destAndPartition, data)); | ||
| } | ||
| } | ||
| } | ||
| Original file line number | Diff line number | Diff line change |
|---|---|---|
| @@ -0,0 +1,149 @@ | ||
| /* | ||
| * 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.util.Preconditions.checkStateNotNull; | ||
|
|
||
| import java.lang.reflect.Array; | ||
| import java.math.BigDecimal; | ||
| import java.nio.ByteBuffer; | ||
| import java.time.Instant; | ||
| import java.time.LocalDate; | ||
| import java.time.LocalDateTime; | ||
| import java.time.LocalTime; | ||
| import java.util.concurrent.TimeUnit; | ||
| import org.apache.beam.sdk.schemas.Schema; | ||
| import org.apache.beam.sdk.schemas.Schema.FieldType; | ||
| import org.apache.beam.sdk.schemas.logicaltypes.Date; | ||
| import org.apache.beam.sdk.schemas.logicaltypes.DateTime; | ||
| import org.apache.beam.sdk.schemas.logicaltypes.FixedPrecisionNumeric; | ||
| import org.apache.beam.sdk.schemas.logicaltypes.MicrosInstant; | ||
| import org.apache.beam.sdk.schemas.logicaltypes.Time; | ||
| import org.apache.beam.sdk.values.Row; | ||
| import org.apache.iceberg.StructLike; | ||
| import org.apache.iceberg.types.Type; | ||
| import org.apache.iceberg.types.Types; | ||
| import org.apache.iceberg.util.DateTimeUtil; | ||
| import org.apache.iceberg.util.UUIDUtil; | ||
| import org.checkerframework.checker.nullness.qual.Nullable; | ||
|
|
||
| public class BeamRowWrapper implements StructLike { | ||
|
Contributor
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. Please add documentation on why this class is needed and how to use it. A BeamRowWrapper sounds like a more general utility that might have to live outside Iceberg ?
Contributor
Author
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. This class is specifically for Iceberg (helps Iceberg fetch data directly from a Beam Row, instead of having to copy the data over to an Iceberg Record) |
||
|
|
||
| private final FieldType[] types; | ||
| private final @Nullable PositionalGetter<?>[] getters; | ||
| private @Nullable Row row = null; | ||
|
|
||
| public BeamRowWrapper(Schema schema, Types.StructType struct) { | ||
| int size = schema.getFieldCount(); | ||
|
|
||
| types = (FieldType[]) Array.newInstance(FieldType.class, size); | ||
| getters = (PositionalGetter[]) Array.newInstance(PositionalGetter.class, size); | ||
|
|
||
| for (int i = 0; i < size; i++) { | ||
| types[i] = schema.getField(i).getType(); | ||
| getters[i] = buildGetter(types[i], struct.fields().get(i).type()); | ||
| } | ||
| } | ||
|
|
||
| public BeamRowWrapper wrap(@Nullable Row row) { | ||
| this.row = row; | ||
| return this; | ||
| } | ||
|
|
||
| @Override | ||
| public int size() { | ||
| return types.length; | ||
| } | ||
|
|
||
| @Override | ||
| public <T> @Nullable T get(int pos, Class<T> javaClass) { | ||
| if (row == null || row.getValue(pos) == null) { | ||
| return null; | ||
| } else if (getters[pos] != null) { | ||
| return javaClass.cast(getters[pos].get(checkStateNotNull(row), pos)); | ||
| } | ||
|
|
||
| return javaClass.cast(checkStateNotNull(row).getValue(pos)); | ||
| } | ||
|
|
||
| @Override | ||
| public <T> void set(int pos, T value) { | ||
| throw new UnsupportedOperationException( | ||
| "Could not set a field in the BeamRowWrapper because rowData is read-only"); | ||
| } | ||
|
|
||
| private interface PositionalGetter<T> { | ||
| T get(Row data, int pos); | ||
| } | ||
|
|
||
| private static @Nullable PositionalGetter<?> buildGetter(FieldType beamType, Type icebergType) { | ||
|
Contributor
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. I think this also came up in a previous PR. Can you clarify why we would need the wrapper class here ? Also is there a way to avoid having to iterate through the set of types in multiple places ?
Contributor
Author
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. Added some java doc |
||
| switch (beamType.getTypeName()) { | ||
| case BYTE: | ||
| return Row::getByte; | ||
| case INT16: | ||
| return Row::getInt16; | ||
| case STRING: | ||
| return Row::getString; | ||
| case BYTES: | ||
| return (row, pos) -> { | ||
| byte[] bytes = checkStateNotNull(row.getBytes(pos)); | ||
| if (Type.TypeID.UUID == icebergType.typeId()) { | ||
| return UUIDUtil.convert(bytes); | ||
| } else { | ||
| return ByteBuffer.wrap(bytes); | ||
| } | ||
| }; | ||
| case DECIMAL: | ||
| return Row::getDecimal; | ||
| case DATETIME: | ||
| return (row, pos) -> | ||
| TimeUnit.MILLISECONDS.toMicros(checkStateNotNull(row.getDateTime(pos)).getMillis()); | ||
| case ROW: | ||
| Schema beamSchema = checkStateNotNull(beamType.getRowSchema()); | ||
| Types.StructType structType = (Types.StructType) icebergType; | ||
|
|
||
| BeamRowWrapper nestedWrapper = new BeamRowWrapper(beamSchema, structType); | ||
| return (row, pos) -> nestedWrapper.wrap(row.getRow(pos)); | ||
| case LOGICAL_TYPE: | ||
| if (beamType.isLogicalType(MicrosInstant.IDENTIFIER)) { | ||
| return (row, pos) -> { | ||
| Instant instant = checkStateNotNull(row.getLogicalTypeValue(pos, Instant.class)); | ||
| return TimeUnit.SECONDS.toMicros(instant.getEpochSecond()) + instant.getNano() / 1000; | ||
| }; | ||
| } else if (beamType.isLogicalType(DateTime.IDENTIFIER)) { | ||
| return (row, pos) -> | ||
| DateTimeUtil.microsFromTimestamp( | ||
| checkStateNotNull(row.getLogicalTypeValue(pos, LocalDateTime.class))); | ||
| } else if (beamType.isLogicalType(Date.IDENTIFIER)) { | ||
| return (row, pos) -> | ||
| DateTimeUtil.daysFromDate( | ||
| checkStateNotNull(row.getLogicalTypeValue(pos, LocalDate.class))); | ||
| } else if (beamType.isLogicalType(Time.IDENTIFIER)) { | ||
| return (row, pos) -> | ||
| DateTimeUtil.microsFromTime( | ||
| checkStateNotNull(row.getLogicalTypeValue(pos, LocalTime.class))); | ||
| } else if (beamType.isLogicalType(FixedPrecisionNumeric.IDENTIFIER)) { | ||
| return (row, pos) -> row.getLogicalTypeValue(pos, BigDecimal.class); | ||
| } else { | ||
| return null; | ||
| } | ||
| default: | ||
| return null; | ||
| } | ||
| } | ||
| } | ||
| Original file line number | Diff line number | Diff line change |
|---|---|---|
|
|
@@ -31,6 +31,7 @@ | |
| import org.apache.beam.sdk.values.Row; | ||
| 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.base.Predicates; | ||
| import org.apache.iceberg.DistributionMode; | ||
| import org.apache.iceberg.Table; | ||
| import org.apache.iceberg.catalog.Catalog; | ||
| import org.apache.iceberg.catalog.TableIdentifier; | ||
|
|
@@ -381,7 +382,10 @@ | |
| public class IcebergIO { | ||
|
|
||
| public static WriteRows writeRows(IcebergCatalogConfig catalog) { | ||
| return new AutoValue_IcebergIO_WriteRows.Builder().setCatalogConfig(catalog).build(); | ||
| return new AutoValue_IcebergIO_WriteRows.Builder() | ||
| .setCatalogConfig(catalog) | ||
| .setDistributionMode(DistributionMode.NONE) | ||
| .build(); | ||
| } | ||
|
|
||
| @AutoValue | ||
|
|
@@ -397,6 +401,8 @@ public abstract static class WriteRows extends PTransform<PCollection<Row>, Iceb | |
|
|
||
| abstract @Nullable Integer getDirectWriteByteLimit(); | ||
|
|
||
| abstract DistributionMode getDistributionMode(); | ||
|
|
||
| abstract Builder toBuilder(); | ||
|
|
||
| @AutoValue.Builder | ||
|
|
@@ -411,6 +417,8 @@ abstract static class Builder { | |
|
|
||
| abstract Builder setDirectWriteByteLimit(Integer directWriteByteLimit); | ||
|
|
||
| abstract Builder setDistributionMode(DistributionMode mode); | ||
|
|
||
| abstract WriteRows build(); | ||
| } | ||
|
|
||
|
|
@@ -443,6 +451,15 @@ public WriteRows withDirectWriteByteLimit(Integer directWriteByteLimit) { | |
| return toBuilder().setDirectWriteByteLimit(directWriteByteLimit).build(); | ||
| } | ||
|
|
||
| /** | ||
| * Groups incoming rows by partition before sending to writes, ensuring that a given bundle is | ||
|
Contributor
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. Probably summarize the three potential values here and mention the default. |
||
| * written to only one partition. For partitioned tables, this helps significantly to reduce the | ||
| * number of small files. | ||
| */ | ||
| public WriteRows withDistributionMode(DistributionMode mode) { | ||
| return toBuilder().setDistributionMode(mode).build(); | ||
| } | ||
|
|
||
| @Override | ||
| public IcebergWriteResult expand(PCollection<Row> input) { | ||
| List<?> allToArgs = Arrays.asList(getTableIdentifier(), getDynamicDestinations()); | ||
|
|
@@ -464,15 +481,31 @@ public IcebergWriteResult expand(PCollection<Row> input) { | |
| IcebergUtils.isUnbounded(input), | ||
| "Must only provide direct write limit for unbounded pipelines."); | ||
| } | ||
| return input | ||
| .apply("Assign Table Destinations", new AssignDestinations(destinations)) | ||
| .apply( | ||
| "Write Rows to Destinations", | ||
| new WriteToDestinations( | ||
| getCatalogConfig(), | ||
| destinations, | ||
| getTriggeringFrequency(), | ||
| getDirectWriteByteLimit())); | ||
|
|
||
| switch (getDistributionMode()) { | ||
| case NONE: | ||
|
Contributor
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. We don't support "RANGE" ?
Contributor
Author
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. Not yet. We can add it in the future if there's demand |
||
| return input | ||
| .apply("Assign Table Destinations", new AssignDestinations(destinations)) | ||
| .apply( | ||
| "Write Rows to Destinations", | ||
| new WriteToDestinations( | ||
| getCatalogConfig(), | ||
| destinations, | ||
| getTriggeringFrequency(), | ||
| getDirectWriteByteLimit())); | ||
| case HASH: | ||
| return input | ||
| .apply( | ||
| "AssignDestinationAndPartition", | ||
| new AssignDestinationsAndPartitions(destinations, getCatalogConfig())) | ||
| .apply( | ||
| "Write Rows to Partitions", | ||
| new WriteToPartitions( | ||
| getCatalogConfig(), destinations, getTriggeringFrequency())); | ||
| default: | ||
| throw new UnsupportedOperationException( | ||
| "Unsupported distribution mode: " + getDistributionMode()); | ||
| } | ||
| } | ||
| } | ||
|
|
||
|
|
||
Uh oh!
There was an error while loading. Please reload this page.