-
Notifications
You must be signed in to change notification settings - Fork 3k
Flink: support to RowData partition. #1299
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 all commits
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,144 @@ | ||
| /* | ||
| * 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.iceberg.flink; | ||
|
|
||
| import java.lang.reflect.Array; | ||
| import java.nio.ByteBuffer; | ||
| import java.time.LocalDateTime; | ||
| import java.util.UUID; | ||
| import org.apache.flink.table.data.RowData; | ||
| import org.apache.flink.table.data.TimestampData; | ||
| import org.apache.flink.table.types.logical.DecimalType; | ||
| import org.apache.flink.table.types.logical.LocalZonedTimestampType; | ||
| import org.apache.flink.table.types.logical.LogicalType; | ||
| import org.apache.flink.table.types.logical.RowType; | ||
| import org.apache.flink.table.types.logical.TimestampType; | ||
| import org.apache.iceberg.StructLike; | ||
| import org.apache.iceberg.types.Type; | ||
| import org.apache.iceberg.types.Types; | ||
| import org.apache.iceberg.util.DateTimeUtil; | ||
|
|
||
| class RowDataWrapper implements StructLike { | ||
|
|
||
| private final LogicalType[] types; | ||
| private final PositionalGetter<?>[] getters; | ||
| private RowData rowData = null; | ||
|
|
||
| RowDataWrapper(RowType rowType, Types.StructType struct) { | ||
| int size = rowType.getFieldCount(); | ||
|
|
||
| types = (LogicalType[]) Array.newInstance(LogicalType.class, size); | ||
| getters = (PositionalGetter[]) Array.newInstance(PositionalGetter.class, size); | ||
|
|
||
| for (int i = 0; i < size; i++) { | ||
| types[i] = rowType.getTypeAt(i); | ||
| getters[i] = buildGetter(types[i], struct.fields().get(i).type()); | ||
| } | ||
| } | ||
|
|
||
| RowDataWrapper wrap(RowData data) { | ||
| this.rowData = data; | ||
| return this; | ||
| } | ||
|
|
||
| @Override | ||
| public int size() { | ||
| return types.length; | ||
| } | ||
|
|
||
| @Override | ||
| public <T> T get(int pos, Class<T> javaClass) { | ||
| if (rowData.isNullAt(pos)) { | ||
| return null; | ||
| } else if (getters[pos] != null) { | ||
| return javaClass.cast(getters[pos].get(rowData, pos)); | ||
| } | ||
|
|
||
| Object value = RowData.createFieldGetter(types[pos], pos).getFieldOrNull(rowData); | ||
| return javaClass.cast(value); | ||
| } | ||
|
|
||
| @Override | ||
| public <T> void set(int pos, T value) { | ||
| throw new UnsupportedOperationException("Could not set a field in the RowDataWrapper because rowData is read-only"); | ||
| } | ||
|
|
||
| private interface PositionalGetter<T> { | ||
| T get(RowData data, int pos); | ||
| } | ||
|
|
||
| private static PositionalGetter<?> buildGetter(LogicalType logicalType, Type type) { | ||
| switch (type.typeId()) { | ||
| case STRING: | ||
| return (row, pos) -> row.getString(pos).toString(); | ||
|
|
||
| case FIXED: | ||
| case BINARY: | ||
| return (row, pos) -> ByteBuffer.wrap(row.getBinary(pos)); | ||
|
|
||
| case UUID: | ||
| return (row, pos) -> { | ||
| ByteBuffer bb = ByteBuffer.wrap(row.getBinary(pos)); | ||
| long mostSigBits = bb.getLong(); | ||
| long leastSigBits = bb.getLong(); | ||
| return new UUID(mostSigBits, leastSigBits); | ||
| }; | ||
|
|
||
| case DECIMAL: | ||
| DecimalType decimalType = (DecimalType) logicalType; | ||
| return (row, pos) -> row.getDecimal(pos, decimalType.getPrecision(), decimalType.getScale()).toBigDecimal(); | ||
|
|
||
| case TIME: | ||
| // Time in RowData is in milliseconds (Integer), while iceberg's time is microseconds (Long). | ||
| return (row, pos) -> ((long) row.getInt(pos)) * 1_000; | ||
|
|
||
| case TIMESTAMP: | ||
| switch (logicalType.getTypeRoot()) { | ||
| case TIMESTAMP_WITHOUT_TIME_ZONE: | ||
| TimestampType timestampType = (TimestampType) logicalType; | ||
| return (row, pos) -> { | ||
| LocalDateTime localDateTime = row.getTimestamp(pos, timestampType.getPrecision()).toLocalDateTime(); | ||
| return DateTimeUtil.microsFromTimestamp(localDateTime); | ||
| }; | ||
|
|
||
| case TIMESTAMP_WITH_LOCAL_TIME_ZONE: | ||
| LocalZonedTimestampType lzTs = (LocalZonedTimestampType) logicalType; | ||
| return (row, pos) -> { | ||
| TimestampData timestampData = row.getTimestamp(pos, lzTs.getPrecision()); | ||
| return timestampData.getMillisecond() * 1000 + timestampData.getNanoOfMillisecond() / 1000; | ||
| }; | ||
|
|
||
| default: | ||
| throw new IllegalArgumentException("Unhandled iceberg type: " + type + " corresponding flink type: " + | ||
| logicalType); | ||
| } | ||
|
|
||
| case STRUCT: | ||
| RowType rowType = (RowType) logicalType; | ||
| Types.StructType structType = (Types.StructType) type; | ||
|
|
||
| RowDataWrapper nestedWrapper = new RowDataWrapper(rowType, structType); | ||
| return (row, pos) -> nestedWrapper.wrap(row.getRow(pos, rowType.getFieldCount())); | ||
|
|
||
| default: | ||
| return null; | ||
| } | ||
| } | ||
| } | ||
| Original file line number | Diff line number | Diff line change |
|---|---|---|
| @@ -0,0 +1,147 @@ | ||
| /* | ||
| * 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.iceberg.flink; | ||
|
|
||
| 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.time.OffsetDateTime; | ||
| import java.time.ZoneOffset; | ||
| import java.time.temporal.ChronoUnit; | ||
| import java.util.Arrays; | ||
| import java.util.List; | ||
| import java.util.Map; | ||
| import java.util.UUID; | ||
| import java.util.concurrent.TimeUnit; | ||
| import org.apache.flink.table.data.DecimalData; | ||
| import org.apache.flink.table.data.GenericRowData; | ||
| import org.apache.flink.table.data.RowData; | ||
| import org.apache.flink.table.data.StringData; | ||
| import org.apache.flink.table.data.TimestampData; | ||
| import org.apache.iceberg.Schema; | ||
| import org.apache.iceberg.data.Record; | ||
| import org.apache.iceberg.relocated.com.google.common.collect.Lists; | ||
| import org.apache.iceberg.relocated.com.google.common.collect.Maps; | ||
| import org.apache.iceberg.types.Type; | ||
| import org.apache.iceberg.types.Types; | ||
|
|
||
| class RowDataConverter { | ||
| private static final OffsetDateTime EPOCH = Instant.ofEpochSecond(0).atOffset(ZoneOffset.UTC); | ||
| private static final LocalDate EPOCH_DAY = EPOCH.toLocalDate(); | ||
|
|
||
| private RowDataConverter() { | ||
| } | ||
|
|
||
| static RowData convert(Schema iSchema, Record record) { | ||
| return convert(iSchema.asStruct(), record); | ||
| } | ||
|
|
||
| private static RowData convert(Types.StructType struct, Record record) { | ||
| GenericRowData rowData = new GenericRowData(struct.fields().size()); | ||
| List<Types.NestedField> fields = struct.fields(); | ||
| for (int i = 0; i < fields.size(); i += 1) { | ||
| Types.NestedField field = fields.get(i); | ||
|
|
||
| Type fieldType = field.type(); | ||
|
|
||
| switch (fieldType.typeId()) { | ||
| case STRUCT: | ||
| rowData.setField(i, convert(fieldType.asStructType(), record.get(i))); | ||
| break; | ||
| case LIST: | ||
| rowData.setField(i, convert(fieldType.asListType(), record.get(i))); | ||
| break; | ||
| case MAP: | ||
| rowData.setField(i, convert(fieldType.asMapType(), record.get(i))); | ||
| break; | ||
| default: | ||
| rowData.setField(i, convert(fieldType, record.get(i))); | ||
| } | ||
| } | ||
| return rowData; | ||
| } | ||
|
|
||
| private static Object convert(Type type, Object object) { | ||
| if (object == null) { | ||
| return null; | ||
| } | ||
|
|
||
| switch (type.typeId()) { | ||
| case BOOLEAN: | ||
| case INTEGER: | ||
| case LONG: | ||
| case FLOAT: | ||
| case DOUBLE: | ||
| case FIXED: | ||
| return object; | ||
| case DATE: | ||
| return (int) ChronoUnit.DAYS.between(EPOCH_DAY, (LocalDate) object); | ||
| case TIME: | ||
| // Iceberg's time is in microseconds, while flink's time is in milliseconds. | ||
| LocalTime localTime = (LocalTime) object; | ||
| return (int) TimeUnit.NANOSECONDS.toMillis(localTime.toNanoOfDay()); | ||
|
Member
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. Here, we will truncate the localTime to be milliseconds, so we will erase the microseconds part. That's to say, the partition value will be different between The following unit tests indicate the above thing : https://github.com/apache/iceberg/pull/1299/files#diff-97304b05e2faea4a749031f514361a70R193
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 don't think it is necessary to disable because any data written by Flink will necessarily be a millisecond-precision value. Partitioning is still correct with respect to the data that was written, because all of the data has millisecond values.
Member
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. For the same data with time type, if flink write them into an iceberg table A, and hive MR or spark read it, in this case, there should be no problem. But for the same data set, both flink and spark write them into difference tables A and B, then there should be difference between table A and B because of lost microseconds. The differences sounds reasonable because of the different behavior from different compute engines. |
||
| case TIMESTAMP: | ||
| if (((Types.TimestampType) type).shouldAdjustToUTC()) { | ||
| return TimestampData.fromInstant(((OffsetDateTime) object).toInstant()); | ||
| } else { | ||
| return TimestampData.fromLocalDateTime((LocalDateTime) object); | ||
| } | ||
| case STRING: | ||
| return StringData.fromString((String) object); | ||
| case UUID: | ||
| UUID uuid = (UUID) object; | ||
| ByteBuffer bb = ByteBuffer.allocate(16); | ||
| bb.putLong(uuid.getMostSignificantBits()); | ||
| bb.putLong(uuid.getLeastSignificantBits()); | ||
| return bb.array(); | ||
| case BINARY: | ||
| ByteBuffer buffer = (ByteBuffer) object; | ||
| return Arrays.copyOfRange(buffer.array(), buffer.arrayOffset() + buffer.position(), | ||
| buffer.arrayOffset() + buffer.remaining()); | ||
| case DECIMAL: | ||
| Types.DecimalType decimalType = (Types.DecimalType) type; | ||
| return DecimalData.fromBigDecimal((BigDecimal) object, decimalType.precision(), decimalType.scale()); | ||
| case STRUCT: | ||
| return convert(type.asStructType(), (Record) object); | ||
| case LIST: | ||
| List<Object> convertedList = Lists.newArrayList(); | ||
| List<?> list = (List<?>) object; | ||
| for (Object element : list) { | ||
| convertedList.add(convert(type.asListType().elementType(), element)); | ||
| } | ||
| return convertedList; | ||
| case MAP: | ||
| Map<Object, Object> convertedMap = Maps.newLinkedHashMap(); | ||
| Map<?, ?> map = (Map<?, ?>) object; | ||
| for (Map.Entry<?, ?> entry : map.entrySet()) { | ||
| convertedMap.put( | ||
| convert(type.asMapType().keyType(), entry.getKey()), | ||
| convert(type.asMapType().valueType(), entry.getValue()) | ||
| ); | ||
| } | ||
| return convertedMap; | ||
| default: | ||
| throw new UnsupportedOperationException("Not a supported type: " + type); | ||
| } | ||
| } | ||
| } | ||
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
Iceberg requires strings to be
CharSequence, not necessarilyString. So if you have UTF8 data, you can potentially just wrap it to produce aCharSequencerather than building an immutable JVM string.Not a blocker, just something to keep in mind for the future.