|
| 1 | +/* |
| 2 | + * Licensed to the Apache Software Foundation (ASF) under one |
| 3 | + * or more contributor license agreements. See the NOTICE file |
| 4 | + * distributed with this work for additional information |
| 5 | + * regarding copyright ownership. The ASF licenses this file |
| 6 | + * to you under the Apache License, Version 2.0 (the |
| 7 | + * "License"); you may not use this file except in compliance |
| 8 | + * with the License. You may obtain a copy of the License at |
| 9 | + * |
| 10 | + * http://www.apache.org/licenses/LICENSE-2.0 |
| 11 | + * |
| 12 | + * Unless required by applicable law or agreed to in writing, software |
| 13 | + * distributed under the License is distributed on an "AS IS" BASIS, |
| 14 | + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. |
| 15 | + * See the License for the specific language governing permissions and |
| 16 | + * limitations under the License. |
| 17 | + */ |
| 18 | +package org.apache.beam.sdk.io.iceberg; |
| 19 | + |
| 20 | +import static org.apache.beam.sdk.util.Preconditions.checkStateNotNull; |
| 21 | + |
| 22 | +import java.lang.reflect.Array; |
| 23 | +import java.math.BigDecimal; |
| 24 | +import java.nio.ByteBuffer; |
| 25 | +import java.time.Instant; |
| 26 | +import java.time.LocalDate; |
| 27 | +import java.time.LocalDateTime; |
| 28 | +import java.time.LocalTime; |
| 29 | +import java.util.concurrent.TimeUnit; |
| 30 | +import org.apache.beam.sdk.schemas.Schema; |
| 31 | +import org.apache.beam.sdk.schemas.Schema.FieldType; |
| 32 | +import org.apache.beam.sdk.schemas.logicaltypes.Date; |
| 33 | +import org.apache.beam.sdk.schemas.logicaltypes.DateTime; |
| 34 | +import org.apache.beam.sdk.schemas.logicaltypes.FixedPrecisionNumeric; |
| 35 | +import org.apache.beam.sdk.schemas.logicaltypes.MicrosInstant; |
| 36 | +import org.apache.beam.sdk.schemas.logicaltypes.Time; |
| 37 | +import org.apache.beam.sdk.values.Row; |
| 38 | +import org.apache.iceberg.StructLike; |
| 39 | +import org.apache.iceberg.types.Type; |
| 40 | +import org.apache.iceberg.types.Types; |
| 41 | +import org.apache.iceberg.util.DateTimeUtil; |
| 42 | +import org.apache.iceberg.util.UUIDUtil; |
| 43 | +import org.checkerframework.checker.nullness.qual.Nullable; |
| 44 | + |
| 45 | +/** |
| 46 | + * A wrapper that adapts a Beam {@link Row} to Iceberg's {@link StructLike} interface. |
| 47 | + * |
| 48 | + * <p>This class allows Beam rows to be processed by Iceberg internal components (like partition |
| 49 | + * keys or writers) without requiring a full conversion into Iceberg's internal Record format. It |
| 50 | + * handles the mapping between Beam's {@link Schema} and Iceberg's {@link Types.StructType}, |
| 51 | + * including complex type conversions for timestamps, logical types, and UUIDs. |
| 52 | + * |
| 53 | + * <p><b>Note:</b> This implementation is <b>read-only</b>. Calls to {@link #set(int, Object)} will |
| 54 | + * throw an {@link UnsupportedOperationException}. |
| 55 | + */ |
| 56 | +public class BeamRowWrapper implements StructLike { |
| 57 | + |
| 58 | + private final FieldType[] types; |
| 59 | + private final @Nullable PositionalGetter<?>[] getters; |
| 60 | + private @Nullable Row row = null; |
| 61 | + |
| 62 | + /** Constructs a new wrapper and pre-computes the mapping between Beam and Iceberg fields. */ |
| 63 | + public BeamRowWrapper(Schema schema, Types.StructType struct) { |
| 64 | + int size = schema.getFieldCount(); |
| 65 | + |
| 66 | + types = (FieldType[]) Array.newInstance(FieldType.class, size); |
| 67 | + getters = (PositionalGetter[]) Array.newInstance(PositionalGetter.class, size); |
| 68 | + |
| 69 | + for (int i = 0; i < size; i++) { |
| 70 | + types[i] = schema.getField(i).getType(); |
| 71 | + getters[i] = buildGetter(types[i], struct.fields().get(i).type()); |
| 72 | + } |
| 73 | + } |
| 74 | + |
| 75 | + /** |
| 76 | + * Sets the current Beam {@link Row} to be wrapped. This method allows the wrapper to be reused |
| 77 | + * across different rows to minimize object allocation. |
| 78 | + */ |
| 79 | + public BeamRowWrapper wrap(@Nullable Row row) { |
| 80 | + this.row = row; |
| 81 | + return this; |
| 82 | + } |
| 83 | + |
| 84 | + @Override |
| 85 | + public int size() { |
| 86 | + return types.length; |
| 87 | + } |
| 88 | + |
| 89 | + /** |
| 90 | + * Retrieves a field value from the wrapped row, performing any necessary type conversion to match |
| 91 | + * Iceberg's internal expectations (e.g., converting Timestamps to microseconds). |
| 92 | + */ |
| 93 | + @Override |
| 94 | + public <T> @Nullable T get(int pos, Class<T> javaClass) { |
| 95 | + if (row == null || row.getValue(pos) == null) { |
| 96 | + return null; |
| 97 | + } else if (getters[pos] != null) { |
| 98 | + return javaClass.cast(getters[pos].get(checkStateNotNull(row), pos)); |
| 99 | + } |
| 100 | + |
| 101 | + return javaClass.cast(checkStateNotNull(row).getValue(pos)); |
| 102 | + } |
| 103 | + |
| 104 | + @Override |
| 105 | + public <T> void set(int pos, T value) { |
| 106 | + throw new UnsupportedOperationException( |
| 107 | + "Could not set a field in the BeamRowWrapper because rowData is read-only"); |
| 108 | + } |
| 109 | + |
| 110 | + private interface PositionalGetter<T> { |
| 111 | + T get(Row data, int pos); |
| 112 | + } |
| 113 | + |
| 114 | + /** |
| 115 | + * Factory method to create a getter that handles type-specific conversion logic. |
| 116 | + * |
| 117 | + * <p>Handles special cases: |
| 118 | + * |
| 119 | + * <ul> |
| 120 | + * <li>UUID: Converts {@code byte[]} to Iceberg's UUID representation. |
| 121 | + * <li>DateTime: Converts Beam {@code DateTime} or logical types to microsecond timestamps. |
| 122 | + * <li>Nested Rows: Recursively wraps nested structures in a new {@code BeamRowWrapper}. |
| 123 | + * </ul> |
| 124 | + */ |
| 125 | + private static @Nullable PositionalGetter<?> buildGetter(FieldType beamType, Type icebergType) { |
| 126 | + switch (beamType.getTypeName()) { |
| 127 | + case BYTE: |
| 128 | + return Row::getByte; |
| 129 | + case INT16: |
| 130 | + return Row::getInt16; |
| 131 | + case STRING: |
| 132 | + return Row::getString; |
| 133 | + case BYTES: |
| 134 | + return (row, pos) -> { |
| 135 | + byte[] bytes = checkStateNotNull(row.getBytes(pos)); |
| 136 | + if (Type.TypeID.UUID == icebergType.typeId()) { |
| 137 | + return UUIDUtil.convert(bytes); |
| 138 | + } else { |
| 139 | + return ByteBuffer.wrap(bytes); |
| 140 | + } |
| 141 | + }; |
| 142 | + case DECIMAL: |
| 143 | + return Row::getDecimal; |
| 144 | + case DATETIME: |
| 145 | + return (row, pos) -> |
| 146 | + TimeUnit.MILLISECONDS.toMicros(checkStateNotNull(row.getDateTime(pos)).getMillis()); |
| 147 | + case ROW: |
| 148 | + Schema beamSchema = checkStateNotNull(beamType.getRowSchema()); |
| 149 | + Types.StructType structType = (Types.StructType) icebergType; |
| 150 | + |
| 151 | + BeamRowWrapper nestedWrapper = new BeamRowWrapper(beamSchema, structType); |
| 152 | + return (row, pos) -> nestedWrapper.wrap(row.getRow(pos)); |
| 153 | + case LOGICAL_TYPE: |
| 154 | + if (beamType.isLogicalType(MicrosInstant.IDENTIFIER)) { |
| 155 | + return (row, pos) -> { |
| 156 | + Instant instant = checkStateNotNull(row.getLogicalTypeValue(pos, Instant.class)); |
| 157 | + return TimeUnit.SECONDS.toMicros(instant.getEpochSecond()) + instant.getNano() / 1000; |
| 158 | + }; |
| 159 | + } else if (beamType.isLogicalType(DateTime.IDENTIFIER)) { |
| 160 | + return (row, pos) -> |
| 161 | + DateTimeUtil.microsFromTimestamp( |
| 162 | + checkStateNotNull(row.getLogicalTypeValue(pos, LocalDateTime.class))); |
| 163 | + } else if (beamType.isLogicalType(Date.IDENTIFIER)) { |
| 164 | + return (row, pos) -> |
| 165 | + DateTimeUtil.daysFromDate( |
| 166 | + checkStateNotNull(row.getLogicalTypeValue(pos, LocalDate.class))); |
| 167 | + } else if (beamType.isLogicalType(Time.IDENTIFIER)) { |
| 168 | + return (row, pos) -> |
| 169 | + DateTimeUtil.microsFromTime( |
| 170 | + checkStateNotNull(row.getLogicalTypeValue(pos, LocalTime.class))); |
| 171 | + } else if (beamType.isLogicalType(FixedPrecisionNumeric.IDENTIFIER)) { |
| 172 | + return (row, pos) -> row.getLogicalTypeValue(pos, BigDecimal.class); |
| 173 | + } else { |
| 174 | + return null; |
| 175 | + } |
| 176 | + default: |
| 177 | + return null; |
| 178 | + } |
| 179 | + } |
| 180 | +} |
0 commit comments