Skip to content
Open
Show file tree
Hide file tree
Changes from all commits
Commits
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 @@ -196,6 +196,13 @@ public static FieldWriter createFieldWriter(DataType fieldType) {

case BIGINT:
return (writer, value) -> writer.writeLong((long) value);

case FLOAT:
return (writer, value) -> writer.writeFloat((float) value);

case DOUBLE:
return (writer, value) -> writer.writeDouble((double) value);

// support for nanoseconds come check again after #1195 merge
case TIMESTAMP_WITHOUT_TIME_ZONE:
return (writer, value) -> {
Expand All @@ -215,6 +222,14 @@ public static FieldWriter createFieldWriter(DataType fieldType) {
case BYTES:
return (writer, value) -> writer.writeBytes((byte[]) value, true);

case ARRAY:
throw new IllegalArgumentException(
"Array types cannot be used as bucket keys. Bucket keys must be scalar types.");

case MAP:
throw new IllegalArgumentException(
"Map types cannot be used as bucket keys. Bucket keys must be scalar types.");

default:
throw new IllegalArgumentException(
"Unsupported type for Iceberg binary row writer: " + fieldType);
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -167,7 +167,19 @@ public Type visit(ArrayType arrayType) {

@Override
public Type visit(MapType mapType) {
throw new UnsupportedOperationException("Unsupported map type");
// According to the Iceberg spec,
// the key and value fields of a map should have consecutive IDs
int keyFieldId = getNextId();
int valueFieldId = getNextId();

Type keyType = mapType.getKeyType().accept(this);
Type valueType = mapType.getValueType().accept(this);

if (mapType.getValueType().isNullable()) {
return Types.MapType.ofOptional(keyFieldId, valueFieldId, keyType, valueType);
} else {
return Types.MapType.ofRequired(keyFieldId, valueFieldId, keyType, valueType);
}
}

@Override
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -18,6 +18,7 @@
package org.apache.fluss.lake.iceberg.source;

import org.apache.fluss.row.InternalArray;
import org.apache.fluss.row.InternalMap;
import org.apache.fluss.types.ArrayType;
import org.apache.fluss.types.BigIntType;
import org.apache.fluss.types.BinaryType;
Expand All @@ -31,6 +32,7 @@
import org.apache.fluss.types.FloatType;
import org.apache.fluss.types.IntType;
import org.apache.fluss.types.LocalZonedTimestampType;
import org.apache.fluss.types.MapType;
import org.apache.fluss.types.SmallIntType;
import org.apache.fluss.types.StringType;
import org.apache.fluss.types.TimeType;
Expand Down Expand Up @@ -104,6 +106,13 @@ public Object get(int index) {
? null
: new FlussArrayAsIcebergList(
innerArray, ((ArrayType) elementType).getElementType());
} else if (elementType instanceof MapType) {
MapType mapType = (MapType) elementType;
InternalMap internalMap = flussArray.getMap(index);
return internalMap == null
? null
: new FlussMapAsIcebergMap(
internalMap, mapType.getKeyType(), mapType.getValueType());
} else {
throw new UnsupportedOperationException(
"Unsupported array element type conversion for Fluss type: "
Expand Down
Original file line number Diff line number Diff line change
@@ -0,0 +1,180 @@
/*
* 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.fluss.lake.iceberg.source;

import org.apache.fluss.lake.iceberg.FlussDataTypeToIcebergDataType;
import org.apache.fluss.row.InternalArray;
import org.apache.fluss.row.InternalMap;
import org.apache.fluss.row.InternalRow;
import org.apache.fluss.types.ArrayType;
import org.apache.fluss.types.BigIntType;
import org.apache.fluss.types.BinaryType;
import org.apache.fluss.types.BooleanType;
import org.apache.fluss.types.BytesType;
import org.apache.fluss.types.CharType;
import org.apache.fluss.types.DataType;
import org.apache.fluss.types.DateType;
import org.apache.fluss.types.DecimalType;
import org.apache.fluss.types.DoubleType;
import org.apache.fluss.types.FloatType;
import org.apache.fluss.types.IntType;
import org.apache.fluss.types.LocalZonedTimestampType;
import org.apache.fluss.types.MapType;
import org.apache.fluss.types.RowType;
import org.apache.fluss.types.SmallIntType;
import org.apache.fluss.types.StringType;
import org.apache.fluss.types.TimeType;
import org.apache.fluss.types.TimestampType;
import org.apache.fluss.types.TinyIntType;
import org.apache.fluss.utils.DateTimeUtils;

import org.apache.iceberg.types.Types;

import java.nio.ByteBuffer;
import java.time.Instant;
import java.time.OffsetDateTime;
import java.time.ZoneOffset;
import java.util.AbstractMap;
import java.util.AbstractSet;
import java.util.Iterator;
import java.util.Set;

/** Adapter class for converting Fluss InternalMap to a Java Map for Iceberg. */
public class FlussMapAsIcebergMap extends AbstractMap<Object, Object> {

private final InternalMap flussMap;
private final DataType keyType;
private final DataType valueType;

public FlussMapAsIcebergMap(InternalMap flussMap, DataType keyType, DataType valueType) {
this.flussMap = flussMap;
this.keyType = keyType;
this.valueType = valueType;
}

@Override
public int size() {
return flussMap.size();
}

@Override
public Set<Entry<Object, Object>> entrySet() {
return new AbstractSet<>() {
@Override
public Iterator<Entry<Object, Object>> iterator() {
return new Iterator<>() {
private final InternalArray keyArray = flussMap.keyArray();
private final InternalArray valueArray = flussMap.valueArray();
private final int size = flussMap.size();
private int currentIndex = 0;

@Override
public boolean hasNext() {
return currentIndex < size;
}

@Override
public Entry<Object, Object> next() {
Object key = convertElement(keyArray, currentIndex, keyType);
Object value = convertElement(valueArray, currentIndex, valueType);
currentIndex++;
return new AbstractMap.SimpleEntry<>(key, value);
}
};
}

@Override
public int size() {
return flussMap.size();
}
};
}

private Object convertElement(InternalArray array, int index, DataType elementType) {
if (array.isNullAt(index)) {
return null;
}

if (elementType instanceof BooleanType) {
return array.getBoolean(index);
} else if (elementType instanceof TinyIntType) {
return (int) array.getByte(index);
} else if (elementType instanceof SmallIntType) {
return (int) array.getShort(index);
} else if (elementType instanceof IntType) {
return array.getInt(index);
} else if (elementType instanceof BigIntType) {
return array.getLong(index);
} else if (elementType instanceof FloatType) {
return array.getFloat(index);
} else if (elementType instanceof DoubleType) {
return array.getDouble(index);
} else if (elementType instanceof StringType) {
return array.getString(index).toString();
} else if (elementType instanceof CharType) {
CharType charType = (CharType) elementType;
return array.getChar(index, charType.getLength()).toString();
} else if (elementType instanceof DecimalType) {
DecimalType decimalType = (DecimalType) elementType;
return array.getDecimal(index, decimalType.getPrecision(), decimalType.getScale())
.toBigDecimal();
} else if (elementType instanceof LocalZonedTimestampType) {
LocalZonedTimestampType ltzType = (LocalZonedTimestampType) elementType;
return toIcebergTimestampLtz(
array.getTimestampLtz(index, ltzType.getPrecision()).toInstant());
} else if (elementType instanceof TimestampType) {
TimestampType tsType = (TimestampType) elementType;
return array.getTimestampNtz(index, tsType.getPrecision()).toLocalDateTime();
} else if (elementType instanceof DateType) {
return DateTimeUtils.toLocalDate(array.getInt(index));
} else if (elementType instanceof TimeType) {
return DateTimeUtils.toLocalTime(array.getInt(index));
} else if (elementType instanceof BytesType || elementType instanceof BinaryType) {
return ByteBuffer.wrap(array.getBytes(index));
} else if (elementType instanceof ArrayType) {
ArrayType arrayType = (ArrayType) elementType;
InternalArray internalArray = array.getArray(index);
return internalArray == null
? null
: new FlussArrayAsIcebergList(internalArray, arrayType.getElementType());
} else if (elementType instanceof MapType) {
MapType mapType = (MapType) elementType;
InternalMap internalMap = array.getMap(index);
return internalMap == null
? null
: new FlussMapAsIcebergMap(
internalMap, mapType.getKeyType(), mapType.getValueType());
} else if (elementType instanceof RowType) {
RowType rowType = (RowType) elementType;
Types.StructType nestedStructType =
(Types.StructType) rowType.accept(FlussDataTypeToIcebergDataType.INSTANCE);
InternalRow internalRow = array.getRow(index, rowType.getFieldCount());
return internalRow == null
? null
: new FlussRowAsIcebergRecord(nestedStructType, rowType, internalRow);
} else {
throw new UnsupportedOperationException(
"Unsupported array element type conversion for Fluss type: "
+ elementType.getClass().getSimpleName());
}
}

private OffsetDateTime toIcebergTimestampLtz(Instant instant) {
return OffsetDateTime.ofInstant(instant, ZoneOffset.UTC);
}
}
Original file line number Diff line number Diff line change
Expand Up @@ -19,6 +19,7 @@

import org.apache.fluss.lake.iceberg.FlussDataTypeToIcebergDataType;
import org.apache.fluss.row.InternalArray;
import org.apache.fluss.row.InternalMap;
import org.apache.fluss.row.InternalRow;
import org.apache.fluss.types.ArrayType;
import org.apache.fluss.types.BigIntType;
Expand All @@ -33,6 +34,7 @@
import org.apache.fluss.types.FloatType;
import org.apache.fluss.types.IntType;
import org.apache.fluss.types.LocalZonedTimestampType;
import org.apache.fluss.types.MapType;
import org.apache.fluss.types.RowType;
import org.apache.fluss.types.SmallIntType;
import org.apache.fluss.types.StringType;
Expand Down Expand Up @@ -189,6 +191,15 @@ private FlussRowToIcebergFieldConverter createTypeConverter(DataType flussType,
InternalRow nestedRow = row.getRow(pos, rowType.getFieldCount());
return new FlussRowAsIcebergRecord(nestedStructType, rowType, nestedRow);
};
} else if (flussType instanceof MapType) {
MapType mapType = (MapType) flussType;
return row -> {
InternalMap map = row.getMap(pos);
return map == null
? null
: new FlussMapAsIcebergMap(
map, mapType.getKeyType(), mapType.getValueType());
};
} else {
throw new UnsupportedOperationException(
"Unsupported data type conversion for Fluss type: "
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -32,6 +32,7 @@
import java.time.LocalDateTime;
import java.time.OffsetDateTime;
import java.util.List;
import java.util.Map;

/** Adapter for Iceberg List as Fluss InternalArray. */
public class IcebergArrayAsFlussArray implements InternalArray {
Expand Down Expand Up @@ -139,7 +140,8 @@ public InternalArray getArray(int pos) {

@Override
public InternalMap getMap(int pos) {
throw new UnsupportedOperationException();
Map<?, ?> nestedMap = (Map<?, ?>) icebergList.get(pos);
return nestedMap == null ? null : new IcebergMapAsFlussMap(nestedMap);
}

@Override
Expand Down
Original file line number Diff line number Diff line change
@@ -0,0 +1,50 @@
/*
* 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.fluss.lake.iceberg.source;

import org.apache.fluss.row.InternalArray;
import org.apache.fluss.row.InternalMap;

import java.util.ArrayList;
import java.util.Map;

/** Adapter for Iceberg Map as Fluss InternalMap. */
public class IcebergMapAsFlussMap implements InternalMap {

private final Map<?, ?> icebergMap;

public IcebergMapAsFlussMap(Map<?, ?> icebergMap) {
this.icebergMap = icebergMap;
}

@Override
public int size() {
return icebergMap.size();
}

@Override
public InternalArray keyArray() {
return new IcebergArrayAsFlussArray(new ArrayList<>(icebergMap.keySet()));
}

@Override
public InternalArray valueArray() {
return new IcebergArrayAsFlussArray(new ArrayList<>(icebergMap.values()));
}
}
Original file line number Diff line number Diff line change
Expand Up @@ -34,6 +34,7 @@
import java.time.LocalDateTime;
import java.time.OffsetDateTime;
import java.util.List;
import java.util.Map;

import static org.apache.fluss.lake.iceberg.IcebergLakeCatalog.SYSTEM_COLUMNS;

Expand Down Expand Up @@ -167,8 +168,12 @@ public InternalArray getArray(int pos) {

@Override
public InternalMap getMap(int pos) {
// TODO: Support Map type conversion from Iceberg to Fluss
throw new UnsupportedOperationException();
Object value = icebergRecord.get(pos);
if (value == null) {
return null;
}
Map<?, ?> icebergMap = (Map<?, ?>) value;
return new IcebergMapAsFlussMap(icebergMap);
}

@Override
Expand Down
Loading
Loading