|
| 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 | + |
| 19 | +package com.alibaba.fluss.lake.iceberg.tiering; |
| 20 | + |
| 21 | +import com.alibaba.fluss.record.LogRecord; |
| 22 | +import com.alibaba.fluss.row.InternalRow; |
| 23 | +import com.alibaba.fluss.types.BigIntType; |
| 24 | +import com.alibaba.fluss.types.BinaryType; |
| 25 | +import com.alibaba.fluss.types.BooleanType; |
| 26 | +import com.alibaba.fluss.types.DataType; |
| 27 | +import com.alibaba.fluss.types.DateType; |
| 28 | +import com.alibaba.fluss.types.DecimalType; |
| 29 | +import com.alibaba.fluss.types.DoubleType; |
| 30 | +import com.alibaba.fluss.types.FloatType; |
| 31 | +import com.alibaba.fluss.types.IntType; |
| 32 | +import com.alibaba.fluss.types.LocalZonedTimestampType; |
| 33 | +import com.alibaba.fluss.types.RowType; |
| 34 | +import com.alibaba.fluss.types.SmallIntType; |
| 35 | +import com.alibaba.fluss.types.StringType; |
| 36 | +import com.alibaba.fluss.types.TimeType; |
| 37 | +import com.alibaba.fluss.types.TimestampType; |
| 38 | +import com.alibaba.fluss.types.TinyIntType; |
| 39 | +import com.alibaba.fluss.utils.DateTimeUtils; |
| 40 | + |
| 41 | +import org.apache.iceberg.Schema; |
| 42 | +import org.apache.iceberg.data.Record; |
| 43 | +import org.apache.iceberg.types.Types; |
| 44 | + |
| 45 | +import java.nio.ByteBuffer; |
| 46 | +import java.time.Instant; |
| 47 | +import java.time.OffsetDateTime; |
| 48 | +import java.time.ZoneOffset; |
| 49 | +import java.util.Map; |
| 50 | + |
| 51 | +import static com.alibaba.fluss.utils.Preconditions.checkState; |
| 52 | + |
| 53 | +/** |
| 54 | + * Wrap Fluss {@link LogRecord} as Iceberg {@link Record}. |
| 55 | + * |
| 56 | + * <p>todo: refactor to implement ParquetWriters, OrcWriters, AvroWriters just like Flink & Spark |
| 57 | + * write to iceberg for higher performance |
| 58 | + */ |
| 59 | +public class FlussRecordAsIcebergRecord implements Record { |
| 60 | + |
| 61 | + // Lake table for iceberg will append three system columns: __bucket, __offset,__timestamp |
| 62 | + private static final int LAKE_ICEBERG_SYSTEM_COLUMNS = 3; |
| 63 | + |
| 64 | + private LogRecord logRecord; |
| 65 | + private final int bucket; |
| 66 | + private final Schema icebergSchema; |
| 67 | + private final RowType flussRowType; |
| 68 | + |
| 69 | + // the origin row fields in fluss, excluding the system columns in iceberg |
| 70 | + private int originRowFieldCount; |
| 71 | + private InternalRow internalRow; |
| 72 | + |
| 73 | + public FlussRecordAsIcebergRecord(int bucket, Schema icebergSchema, RowType flussRowType) { |
| 74 | + this.bucket = bucket; |
| 75 | + this.icebergSchema = icebergSchema; |
| 76 | + this.flussRowType = flussRowType; |
| 77 | + } |
| 78 | + |
| 79 | + public void setFlussRecord(LogRecord logRecord) { |
| 80 | + this.logRecord = logRecord; |
| 81 | + this.internalRow = logRecord.getRow(); |
| 82 | + this.originRowFieldCount = internalRow.getFieldCount(); |
| 83 | + checkState( |
| 84 | + originRowFieldCount |
| 85 | + == icebergSchema.asStruct().fields().size() - LAKE_ICEBERG_SYSTEM_COLUMNS, |
| 86 | + "The Iceberg table fields count must equals to LogRecord's fields count."); |
| 87 | + } |
| 88 | + |
| 89 | + @Override |
| 90 | + public Types.StructType struct() { |
| 91 | + return icebergSchema.asStruct(); |
| 92 | + } |
| 93 | + |
| 94 | + @Override |
| 95 | + public Object getField(String name) { |
| 96 | + return icebergSchema; |
| 97 | + } |
| 98 | + |
| 99 | + @Override |
| 100 | + public void setField(String name, Object value) { |
| 101 | + throw new UnsupportedOperationException("method setField is not supported."); |
| 102 | + } |
| 103 | + |
| 104 | + @Override |
| 105 | + public Object get(int pos) { |
| 106 | + // firstly, for system columns |
| 107 | + if (pos == originRowFieldCount) { |
| 108 | + // bucket column |
| 109 | + return bucket; |
| 110 | + } else if (pos == originRowFieldCount + 1) { |
| 111 | + // log offset column |
| 112 | + return logRecord.logOffset(); |
| 113 | + } else if (pos == originRowFieldCount + 2) { |
| 114 | + // timestamp column |
| 115 | + return getTimestampLtz(logRecord.timestamp()); |
| 116 | + } |
| 117 | + |
| 118 | + // handle normal columns |
| 119 | + if (internalRow.isNullAt(pos)) { |
| 120 | + return null; |
| 121 | + } |
| 122 | + |
| 123 | + DataType dataType = flussRowType.getTypeAt(pos); |
| 124 | + if (dataType instanceof BooleanType) { |
| 125 | + return internalRow.getBoolean(pos); |
| 126 | + } else if (dataType instanceof TinyIntType) { |
| 127 | + return (int) internalRow.getByte(pos); |
| 128 | + } else if (dataType instanceof SmallIntType) { |
| 129 | + return internalRow.getShort(pos); |
| 130 | + } else if (dataType instanceof IntType) { |
| 131 | + return internalRow.getInt(pos); |
| 132 | + } else if (dataType instanceof BigIntType) { |
| 133 | + return internalRow.getLong(pos); |
| 134 | + } else if (dataType instanceof FloatType) { |
| 135 | + return internalRow.getFloat(pos); |
| 136 | + } else if (dataType instanceof DoubleType) { |
| 137 | + return internalRow.getDouble(pos); |
| 138 | + } else if (dataType instanceof StringType) { |
| 139 | + return internalRow.getString(pos).toString(); |
| 140 | + } else if (dataType instanceof BinaryType) { |
| 141 | + // Iceberg's Record interface expects ByteBuffer for binary types. |
| 142 | + return ByteBuffer.wrap(internalRow.getBytes(pos)); |
| 143 | + } else if (dataType instanceof DecimalType) { |
| 144 | + // Iceberg expects BigDecimal for decimal types. |
| 145 | + DecimalType decimalType = (DecimalType) dataType; |
| 146 | + return internalRow |
| 147 | + .getDecimal(pos, decimalType.getPrecision(), decimalType.getScale()) |
| 148 | + .toBigDecimal(); |
| 149 | + } else if (dataType instanceof LocalZonedTimestampType) { |
| 150 | + // Iceberg expects OffsetDateTime for timestamp with local timezone. |
| 151 | + return getTimestampLtz( |
| 152 | + internalRow |
| 153 | + .getTimestampLtz( |
| 154 | + pos, ((LocalZonedTimestampType) dataType).getPrecision()) |
| 155 | + .toInstant()); |
| 156 | + } else if (dataType instanceof TimestampType) { |
| 157 | + // Iceberg expects LocalDateType for timestamp without local timezone. |
| 158 | + return internalRow |
| 159 | + .getTimestampNtz(pos, ((TimestampType) dataType).getPrecision()) |
| 160 | + .toLocalDateTime(); |
| 161 | + } else if (dataType instanceof DateType) { |
| 162 | + return DateTimeUtils.toLocalDate(internalRow.getInt(pos)); |
| 163 | + } else if (dataType instanceof TimeType) { |
| 164 | + return DateTimeUtils.toLocalTime(internalRow.getInt(pos)); |
| 165 | + } |
| 166 | + throw new UnsupportedOperationException( |
| 167 | + "Unsupported data type conversion for Fluss type: " |
| 168 | + + dataType.getClass().getName()); |
| 169 | + } |
| 170 | + |
| 171 | + private OffsetDateTime getTimestampLtz(long timestamp) { |
| 172 | + return OffsetDateTime.ofInstant(Instant.ofEpochMilli(timestamp), ZoneOffset.UTC); |
| 173 | + } |
| 174 | + |
| 175 | + private OffsetDateTime getTimestampLtz(Instant instant) { |
| 176 | + return OffsetDateTime.ofInstant(instant, ZoneOffset.UTC); |
| 177 | + } |
| 178 | + |
| 179 | + @Override |
| 180 | + public Record copy() { |
| 181 | + throw new UnsupportedOperationException("method copy is not supported."); |
| 182 | + } |
| 183 | + |
| 184 | + @Override |
| 185 | + public Record copy(Map<String, Object> overwriteValues) { |
| 186 | + throw new UnsupportedOperationException("method copy is not supported."); |
| 187 | + } |
| 188 | + |
| 189 | + @Override |
| 190 | + public int size() { |
| 191 | + return icebergSchema.asStruct().fields().size(); |
| 192 | + } |
| 193 | + |
| 194 | + @Override |
| 195 | + public <T> T get(int pos, Class<T> javaClass) { |
| 196 | + Object value = get(pos); |
| 197 | + if (value == null || javaClass.isInstance(value)) { |
| 198 | + return javaClass.cast(value); |
| 199 | + } else { |
| 200 | + throw new IllegalStateException( |
| 201 | + "Not an instance of " + javaClass.getName() + ": " + value); |
| 202 | + } |
| 203 | + } |
| 204 | + |
| 205 | + @Override |
| 206 | + public <T> void set(int pos, T value) { |
| 207 | + throw new UnsupportedOperationException("method set is not supported."); |
| 208 | + } |
| 209 | +} |
0 commit comments