-
Notifications
You must be signed in to change notification settings - Fork 3k
Data: Add UUID conversion support for InternalRecordWrapper #14208
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
Closed
Closed
Changes from all commits
Commits
Show all changes
5 commits
Select commit
Hold shift + click to select a range
c9d1348
Data: Add UUID conversion support for InternalRecordWrapper
xxubai 6a3f87e
add UUID type
xxubai 12ab952
Test: Add UUID writing support in PartitionedAppendWriter
xxubai 06fcaaf
Use Guava's Lists.newArrayList
xxubai 35deb8a
Use Guava's Lists.newArrayList
xxubai File filter
Filter by extension
Conversations
Failed to load comments.
Loading
Jump to
Jump to file
Failed to load files.
Loading
Diff view
Diff view
There are no files selected for viewing
This file contains hidden or bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
145 changes: 145 additions & 0 deletions
145
data/src/test/java/org/apache/iceberg/data/TestInternalRecordWrapper.java
This file contains hidden or bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
| Original file line number | Diff line number | Diff line change |
|---|---|---|
| @@ -0,0 +1,145 @@ | ||
| /* | ||
| * 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.data; | ||
|
|
||
| import static org.apache.iceberg.types.Types.NestedField.required; | ||
| import static org.assertj.core.api.Assertions.assertThat; | ||
|
|
||
| import java.nio.ByteBuffer; | ||
| import java.time.LocalDate; | ||
| import java.time.LocalDateTime; | ||
| import java.time.LocalTime; | ||
| import java.time.OffsetDateTime; | ||
| import java.time.ZoneOffset; | ||
| import java.util.Random; | ||
| import java.util.UUID; | ||
| import org.apache.iceberg.Schema; | ||
| import org.apache.iceberg.StructLike; | ||
| import org.apache.iceberg.types.Types; | ||
| import org.apache.iceberg.util.DateTimeUtil; | ||
| import org.apache.iceberg.util.UUIDUtil; | ||
| import org.junit.jupiter.api.Test; | ||
|
|
||
| public class TestInternalRecordWrapper { | ||
|
|
||
| @Test | ||
| public void testDateConversion() { | ||
| Schema schema = new Schema(required(1, "date_field", Types.DateType.get())); | ||
|
|
||
| LocalDate testDate = LocalDate.of(2025, 10, 1); | ||
| Record record = GenericRecord.create(schema); | ||
| record.set(0, testDate); | ||
|
|
||
| InternalRecordWrapper wrapper = new InternalRecordWrapper(schema.asStruct()); | ||
| StructLike wrappedRecord = wrapper.wrap(record); | ||
|
|
||
| Integer convertedValue = wrappedRecord.get(0, Integer.class); | ||
| Integer expectedValue = DateTimeUtil.daysFromDate(testDate); | ||
|
|
||
| assertThat(convertedValue).isEqualTo(expectedValue); | ||
| } | ||
|
|
||
| @Test | ||
| public void testTimeConversion() { | ||
| Schema schema = new Schema(required(1, "time_field", Types.TimeType.get())); | ||
|
|
||
| LocalTime testTime = LocalTime.of(14, 30, 45, 123456000); | ||
| Record record = GenericRecord.create(schema); | ||
| record.set(0, testTime); | ||
|
|
||
| InternalRecordWrapper wrapper = new InternalRecordWrapper(schema.asStruct()); | ||
| StructLike wrappedRecord = wrapper.wrap(record); | ||
|
|
||
| Long convertedValue = wrappedRecord.get(0, Long.class); | ||
| Long expectedValue = DateTimeUtil.microsFromTime(testTime); | ||
|
|
||
| assertThat(convertedValue).isEqualTo(expectedValue); | ||
| } | ||
|
|
||
| @Test | ||
| public void testTimestampWithZoneConversion() { | ||
| Schema schema = new Schema(required(1, "timestamp_tz_field", Types.TimestampType.withZone())); | ||
|
|
||
| OffsetDateTime testTimestamp = | ||
| OffsetDateTime.of(2025, 10, 1, 14, 30, 45, 123456000, ZoneOffset.UTC); | ||
| Record record = GenericRecord.create(schema); | ||
| record.set(0, testTimestamp); | ||
|
|
||
| InternalRecordWrapper wrapper = new InternalRecordWrapper(schema.asStruct()); | ||
| StructLike wrappedRecord = wrapper.wrap(record); | ||
|
|
||
| Long convertedValue = wrappedRecord.get(0, Long.class); | ||
| Long expectedValue = DateTimeUtil.microsFromTimestamptz(testTimestamp); | ||
|
|
||
| assertThat(convertedValue).isEqualTo(expectedValue); | ||
| } | ||
|
|
||
| @Test | ||
| public void testTimestampWithoutZoneConversion() { | ||
| Schema schema = new Schema(required(1, "timestamp_field", Types.TimestampType.withoutZone())); | ||
|
|
||
| LocalDateTime testTimestamp = LocalDateTime.of(2025, 10, 1, 14, 30, 45, 123456000); | ||
| Record record = GenericRecord.create(schema); | ||
| record.set(0, testTimestamp); | ||
|
|
||
| InternalRecordWrapper wrapper = new InternalRecordWrapper(schema.asStruct()); | ||
| StructLike wrappedRecord = wrapper.wrap(record); | ||
|
|
||
| Long convertedValue = wrappedRecord.get(0, Long.class); | ||
| Long expectedValue = DateTimeUtil.microsFromTimestamp(testTimestamp); | ||
|
|
||
| assertThat(convertedValue).isEqualTo(expectedValue); | ||
| } | ||
|
|
||
| @Test | ||
| public void testFixedConversion() { | ||
| Schema schema = new Schema(required(1, "fixed_field", Types.FixedType.ofLength(16))); | ||
|
|
||
| byte[] testBytes = "test fixed bytes".getBytes(); | ||
| Record record = GenericRecord.create(schema); | ||
| record.set(0, testBytes); | ||
|
|
||
| InternalRecordWrapper wrapper = new InternalRecordWrapper(schema.asStruct()); | ||
| StructLike wrappedRecord = wrapper.wrap(record); | ||
|
|
||
| ByteBuffer convertedValue = wrappedRecord.get(0, ByteBuffer.class); | ||
| ByteBuffer expectedValue = ByteBuffer.wrap(testBytes); | ||
|
|
||
| assertThat(convertedValue).isEqualTo(expectedValue); | ||
| } | ||
|
|
||
| @Test | ||
| public void testUuidConversion() { | ||
| Schema schema = new Schema(required(1, "uuid_field", Types.UUIDType.get())); | ||
|
|
||
| byte[] uuidBytes = new byte[16]; | ||
| Random random = new Random(); | ||
| random.nextBytes(uuidBytes); | ||
| Record record = GenericRecord.create(schema); | ||
| record.set(0, uuidBytes); | ||
|
|
||
| InternalRecordWrapper wrapper = new InternalRecordWrapper(schema.asStruct()); | ||
| StructLike wrappedRecord = wrapper.wrap(record); | ||
|
|
||
| UUID convertedValue = wrappedRecord.get(0, UUID.class); | ||
| UUID expectedValue = UUIDUtil.convert(uuidBytes); | ||
|
|
||
| assertThat(convertedValue).isEqualTo(expectedValue); | ||
| } | ||
| } | ||
This file contains hidden or bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
| Original file line number | Diff line number | Diff line change |
|---|---|---|
|
|
@@ -18,19 +18,26 @@ | |
| */ | ||
| package org.apache.iceberg.connect.data; | ||
|
|
||
| import static org.apache.iceberg.types.Types.NestedField.required; | ||
| import static org.assertj.core.api.Assertions.assertThat; | ||
| import static org.mockito.ArgumentMatchers.any; | ||
| import static org.mockito.Mockito.mock; | ||
| import static org.mockito.Mockito.when; | ||
|
|
||
| import java.util.List; | ||
| import java.util.UUID; | ||
| import org.apache.iceberg.FileFormat; | ||
| import org.apache.iceberg.PartitionSpec; | ||
| import org.apache.iceberg.Schema; | ||
| import org.apache.iceberg.connect.IcebergSinkConfig; | ||
| import org.apache.iceberg.connect.TableSinkConfig; | ||
| import org.apache.iceberg.data.GenericRecord; | ||
| import org.apache.iceberg.data.Record; | ||
| import org.apache.iceberg.io.WriteResult; | ||
| import org.apache.iceberg.relocated.com.google.common.collect.ImmutableList; | ||
| import org.apache.iceberg.relocated.com.google.common.collect.ImmutableMap; | ||
| import org.apache.iceberg.relocated.com.google.common.collect.Lists; | ||
| import org.apache.iceberg.types.Types; | ||
| import org.junit.jupiter.params.ParameterizedTest; | ||
| import org.junit.jupiter.params.provider.ValueSource; | ||
|
|
||
|
|
@@ -63,4 +70,34 @@ public void testPartitionedAppendWriter(String format) { | |
| assertThat(result.dataFiles()).allMatch(file -> file.format() == FileFormat.fromString(format)); | ||
| assertThat(result.deleteFiles()).hasSize(0); | ||
| } | ||
|
|
||
| @ParameterizedTest | ||
| @ValueSource(strings = {"parquet", "orc"}) | ||
| public void testWriteUuidWithFountWriter(String format) { | ||
|
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. This one is running without the change. |
||
| Schema schema = | ||
| new Schema( | ||
| required(1, "id", Types.IntegerType.get()), | ||
| required(2, "uuid_field", Types.UUIDType.get())); | ||
| PartitionSpec spec = PartitionSpec.builderFor(schema).bucket("uuid_field", 2).build(); | ||
| when(table.schema()).thenReturn(schema); | ||
| when(table.spec()).thenReturn(spec); | ||
|
|
||
| List<Record> rows = Lists.newArrayList(); | ||
| for (int i = 0; i < 10; i++) { | ||
| UUID uuid = UUID.randomUUID(); | ||
| Record record = GenericRecord.create(schema); | ||
| record.set(0, i); | ||
| record.set(1, uuid); | ||
| rows.add(record); | ||
| } | ||
|
|
||
| IcebergSinkConfig config = mock(IcebergSinkConfig.class); | ||
| when(config.tableConfig(any())).thenReturn(mock(TableSinkConfig.class)); | ||
| when(config.writeProps()).thenReturn(ImmutableMap.of("write.format.default", format)); | ||
| WriteResult result = writeTest(rows, config, PartitionedAppendWriter.class); | ||
|
|
||
| assertThat(result.dataFiles()).hasSizeGreaterThan(1); | ||
| assertThat(result.dataFiles()).allMatch(file -> file.format() == FileFormat.fromString(format)); | ||
| assertThat(result.deleteFiles()).hasSize(0); | ||
| } | ||
| } | ||
Add this suggestion to a batch that can be applied as a single commit.
This suggestion is invalid because no changes were made to the code.
Suggestions cannot be applied while the pull request is closed.
Suggestions cannot be applied while viewing a subset of changes.
Only one suggestion per line can be applied in a batch.
Add this suggestion to a batch that can be applied as a single commit.
Applying suggestions on deleted lines is not supported.
You must change the existing code in this line in order to create a valid suggestion.
Outdated suggestions cannot be applied.
This suggestion has been applied or marked resolved.
Suggestions cannot be applied from pending reviews.
Suggestions cannot be applied on multi-line comments.
Suggestions cannot be applied while the pull request is queued to merge.
Suggestion cannot be applied right now. Please check back later.
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.
Great test coverage
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.
I noticed there is an abstract test class for the wrapper classes: https://github.com/apache/iceberg/blob/main/data/src/test/java/org/apache/iceberg/RecordWrapperTestBase.java
Do you think we should extend it here?
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.
Not sure extending it wouldn’t add much value in this case.