|
| 1 | +/* |
| 2 | + * Licensed to the Apache Software Foundation (ASF) under one or more |
| 3 | + * contributor license agreements. See the NOTICE file distributed with |
| 4 | + * this work for additional information regarding copyright ownership. |
| 5 | + * The ASF licenses this file to You under the Apache License, Version 2.0 |
| 6 | + * (the "License"); you may not use this file except in compliance with |
| 7 | + * the License. You may obtain a copy of the License at |
| 8 | + * |
| 9 | + * http://www.apache.org/licenses/LICENSE-2.0 |
| 10 | + * |
| 11 | + * Unless required by applicable law or agreed to in writing, software |
| 12 | + * distributed under the License is distributed on an "AS IS" BASIS, |
| 13 | + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. |
| 14 | + * See the License for the specific language governing permissions and |
| 15 | + * limitations under the License. |
| 16 | + */ |
| 17 | + |
| 18 | +package org.apache.fluss.lake.iceberg.source; |
| 19 | + |
| 20 | +import org.apache.fluss.lake.iceberg.FlussDataTypeToIcebergDataType; |
| 21 | +import org.apache.fluss.row.InternalArray; |
| 22 | +import org.apache.fluss.row.InternalMap; |
| 23 | +import org.apache.fluss.row.InternalRow; |
| 24 | +import org.apache.fluss.types.ArrayType; |
| 25 | +import org.apache.fluss.types.BigIntType; |
| 26 | +import org.apache.fluss.types.BinaryType; |
| 27 | +import org.apache.fluss.types.BooleanType; |
| 28 | +import org.apache.fluss.types.BytesType; |
| 29 | +import org.apache.fluss.types.CharType; |
| 30 | +import org.apache.fluss.types.DataType; |
| 31 | +import org.apache.fluss.types.DateType; |
| 32 | +import org.apache.fluss.types.DecimalType; |
| 33 | +import org.apache.fluss.types.DoubleType; |
| 34 | +import org.apache.fluss.types.FloatType; |
| 35 | +import org.apache.fluss.types.IntType; |
| 36 | +import org.apache.fluss.types.LocalZonedTimestampType; |
| 37 | +import org.apache.fluss.types.MapType; |
| 38 | +import org.apache.fluss.types.RowType; |
| 39 | +import org.apache.fluss.types.SmallIntType; |
| 40 | +import org.apache.fluss.types.StringType; |
| 41 | +import org.apache.fluss.types.TimeType; |
| 42 | +import org.apache.fluss.types.TimestampType; |
| 43 | +import org.apache.fluss.types.TinyIntType; |
| 44 | +import org.apache.fluss.utils.DateTimeUtils; |
| 45 | + |
| 46 | +import org.apache.iceberg.types.Types; |
| 47 | + |
| 48 | +import java.nio.ByteBuffer; |
| 49 | +import java.time.Instant; |
| 50 | +import java.time.OffsetDateTime; |
| 51 | +import java.time.ZoneOffset; |
| 52 | +import java.util.AbstractMap; |
| 53 | +import java.util.AbstractSet; |
| 54 | +import java.util.Iterator; |
| 55 | +import java.util.Set; |
| 56 | + |
| 57 | +/** Adapter class for converting Fluss InternalMap to a Java Map for Iceberg. */ |
| 58 | +public class FlussMapAsIcebergMap extends AbstractMap<Object, Object> { |
| 59 | + |
| 60 | + private final InternalMap flussMap; |
| 61 | + private final DataType keyType; |
| 62 | + private final DataType valueType; |
| 63 | + |
| 64 | + public FlussMapAsIcebergMap(InternalMap flussMap, DataType keyType, DataType valueType) { |
| 65 | + this.flussMap = flussMap; |
| 66 | + this.keyType = keyType; |
| 67 | + this.valueType = valueType; |
| 68 | + } |
| 69 | + |
| 70 | + @Override |
| 71 | + public int size() { |
| 72 | + return flussMap.size(); |
| 73 | + } |
| 74 | + |
| 75 | + @Override |
| 76 | + public Set<Entry<Object, Object>> entrySet() { |
| 77 | + return new AbstractSet<>() { |
| 78 | + @Override |
| 79 | + public Iterator<Entry<Object, Object>> iterator() { |
| 80 | + return new Iterator<>() { |
| 81 | + private final InternalArray keyArray = flussMap.keyArray(); |
| 82 | + private final InternalArray valueArray = flussMap.valueArray(); |
| 83 | + private final int size = flussMap.size(); |
| 84 | + private int currentIndex = 0; |
| 85 | + |
| 86 | + @Override |
| 87 | + public boolean hasNext() { |
| 88 | + return currentIndex < size; |
| 89 | + } |
| 90 | + |
| 91 | + @Override |
| 92 | + public Entry<Object, Object> next() { |
| 93 | + Object key = convertElement(keyArray, currentIndex, keyType); |
| 94 | + Object value = convertElement(valueArray, currentIndex, valueType); |
| 95 | + currentIndex++; |
| 96 | + return new AbstractMap.SimpleEntry<>(key, value); |
| 97 | + } |
| 98 | + }; |
| 99 | + } |
| 100 | + |
| 101 | + @Override |
| 102 | + public int size() { |
| 103 | + return flussMap.size(); |
| 104 | + } |
| 105 | + }; |
| 106 | + } |
| 107 | + |
| 108 | + private Object convertElement(InternalArray array, int index, DataType elementType) { |
| 109 | + if (array.isNullAt(index)) { |
| 110 | + return null; |
| 111 | + } |
| 112 | + |
| 113 | + if (elementType instanceof BooleanType) { |
| 114 | + return array.getBoolean(index); |
| 115 | + } else if (elementType instanceof TinyIntType) { |
| 116 | + return (int) array.getByte(index); |
| 117 | + } else if (elementType instanceof SmallIntType) { |
| 118 | + return (int) array.getShort(index); |
| 119 | + } else if (elementType instanceof IntType) { |
| 120 | + return array.getInt(index); |
| 121 | + } else if (elementType instanceof BigIntType) { |
| 122 | + return array.getLong(index); |
| 123 | + } else if (elementType instanceof FloatType) { |
| 124 | + return array.getFloat(index); |
| 125 | + } else if (elementType instanceof DoubleType) { |
| 126 | + return array.getDouble(index); |
| 127 | + } else if (elementType instanceof StringType) { |
| 128 | + return array.getString(index).toString(); |
| 129 | + } else if (elementType instanceof CharType) { |
| 130 | + CharType charType = (CharType) elementType; |
| 131 | + return array.getChar(index, charType.getLength()).toString(); |
| 132 | + } else if (elementType instanceof DecimalType) { |
| 133 | + DecimalType decimalType = (DecimalType) elementType; |
| 134 | + return array.getDecimal(index, decimalType.getPrecision(), decimalType.getScale()) |
| 135 | + .toBigDecimal(); |
| 136 | + } else if (elementType instanceof LocalZonedTimestampType) { |
| 137 | + LocalZonedTimestampType ltzType = (LocalZonedTimestampType) elementType; |
| 138 | + return toIcebergTimestampLtz( |
| 139 | + array.getTimestampLtz(index, ltzType.getPrecision()).toInstant()); |
| 140 | + } else if (elementType instanceof TimestampType) { |
| 141 | + TimestampType tsType = (TimestampType) elementType; |
| 142 | + return array.getTimestampNtz(index, tsType.getPrecision()).toLocalDateTime(); |
| 143 | + } else if (elementType instanceof DateType) { |
| 144 | + return DateTimeUtils.toLocalDate(array.getInt(index)); |
| 145 | + } else if (elementType instanceof TimeType) { |
| 146 | + return DateTimeUtils.toLocalTime(array.getInt(index)); |
| 147 | + } else if (elementType instanceof BytesType || elementType instanceof BinaryType) { |
| 148 | + return ByteBuffer.wrap(array.getBytes(index)); |
| 149 | + } else if (elementType instanceof ArrayType) { |
| 150 | + ArrayType arrayType = (ArrayType) elementType; |
| 151 | + InternalArray internalArray = array.getArray(index); |
| 152 | + return internalArray == null |
| 153 | + ? null |
| 154 | + : new FlussArrayAsIcebergList(internalArray, arrayType.getElementType()); |
| 155 | + } else if (elementType instanceof MapType) { |
| 156 | + MapType mapType = (MapType) elementType; |
| 157 | + InternalMap internalMap = array.getMap(index); |
| 158 | + return internalMap == null |
| 159 | + ? null |
| 160 | + : new FlussMapAsIcebergMap( |
| 161 | + internalMap, mapType.getKeyType(), mapType.getValueType()); |
| 162 | + } else if (elementType instanceof RowType) { |
| 163 | + RowType rowType = (RowType) elementType; |
| 164 | + Types.StructType nestedStructType = |
| 165 | + (Types.StructType) rowType.accept(FlussDataTypeToIcebergDataType.INSTANCE); |
| 166 | + InternalRow internalRow = array.getRow(index, rowType.getFieldCount()); |
| 167 | + return internalRow == null |
| 168 | + ? null |
| 169 | + : new FlussRowAsIcebergRecord(nestedStructType, rowType, internalRow); |
| 170 | + } else { |
| 171 | + throw new UnsupportedOperationException( |
| 172 | + "Unsupported array element type conversion for Fluss type: " |
| 173 | + + elementType.getClass().getSimpleName()); |
| 174 | + } |
| 175 | + } |
| 176 | + |
| 177 | + private OffsetDateTime toIcebergTimestampLtz(Instant instant) { |
| 178 | + return OffsetDateTime.ofInstant(instant, ZoneOffset.UTC); |
| 179 | + } |
| 180 | +} |
0 commit comments