|
| 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, |
| 13 | + * software distributed under the License is distributed on an |
| 14 | + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY |
| 15 | + * KIND, either express or implied. See the License for the |
| 16 | + * specific language governing permissions and limitations |
| 17 | + * under the License. |
| 18 | + */ |
| 19 | +package org.apache.iceberg.flink; |
| 20 | + |
| 21 | +import static org.apache.iceberg.TableProperties.DEFAULT_FILE_FORMAT; |
| 22 | +import static org.assertj.core.api.Assertions.assertThat; |
| 23 | + |
| 24 | +import java.nio.ByteBuffer; |
| 25 | +import java.nio.file.Path; |
| 26 | +import java.util.Arrays; |
| 27 | +import java.util.List; |
| 28 | +import java.util.UUID; |
| 29 | +import org.apache.flink.table.data.GenericRowData; |
| 30 | +import org.apache.flink.table.data.RowData; |
| 31 | +import org.apache.flink.table.types.logical.RowType; |
| 32 | +import org.apache.iceberg.AppendFiles; |
| 33 | +import org.apache.iceberg.CombinedScanTask; |
| 34 | +import org.apache.iceberg.DataFile; |
| 35 | +import org.apache.iceberg.FileFormat; |
| 36 | +import org.apache.iceberg.Parameter; |
| 37 | +import org.apache.iceberg.Parameters; |
| 38 | +import org.apache.iceberg.PartitionSpec; |
| 39 | +import org.apache.iceberg.Schema; |
| 40 | +import org.apache.iceberg.Table; |
| 41 | +import org.apache.iceberg.catalog.Namespace; |
| 42 | +import org.apache.iceberg.catalog.TableIdentifier; |
| 43 | +import org.apache.iceberg.data.GenericAppenderHelper; |
| 44 | +import org.apache.iceberg.data.GenericRecord; |
| 45 | +import org.apache.iceberg.data.Record; |
| 46 | +import org.apache.iceberg.flink.sink.RowDataTaskWriterFactory; |
| 47 | +import org.apache.iceberg.flink.source.DataIterator; |
| 48 | +import org.apache.iceberg.flink.source.reader.ReaderUtil; |
| 49 | +import org.apache.iceberg.io.CloseableIterable; |
| 50 | +import org.apache.iceberg.io.TaskWriter; |
| 51 | +import org.apache.iceberg.relocated.com.google.common.collect.ImmutableList; |
| 52 | +import org.apache.iceberg.relocated.com.google.common.collect.ImmutableMap; |
| 53 | +import org.apache.iceberg.relocated.com.google.common.collect.Lists; |
| 54 | +import org.apache.iceberg.types.Types; |
| 55 | +import org.apache.iceberg.util.UUIDUtil; |
| 56 | +import org.junit.jupiter.api.BeforeEach; |
| 57 | +import org.junit.jupiter.api.TestTemplate; |
| 58 | +import org.junit.jupiter.api.io.TempDir; |
| 59 | + |
| 60 | +class TestFlinkUuidType extends CatalogTestBase { |
| 61 | + private static final String TABLE_NAME = "test_table"; |
| 62 | + private static final long TARGET_FILE_SIZE = 128 * 1024 * 1024; |
| 63 | + private static final UUID EXPECTED_UUID = UUID.fromString("0f8fad5b-d9cb-469f-a165-70867728950e"); |
| 64 | + private static final Schema SCHEMA = |
| 65 | + new Schema( |
| 66 | + Types.NestedField.required(1, "id", Types.IntegerType.get()), |
| 67 | + Types.NestedField.required(2, "uuid", Types.UUIDType.get())); |
| 68 | + |
| 69 | + private Table icebergTable; |
| 70 | + @TempDir private Path warehouseDir; |
| 71 | + |
| 72 | + @Parameter(index = 2) |
| 73 | + private FileFormat fileFormat; |
| 74 | + |
| 75 | + @Parameters(name = "catalogName={0}, baseNamespace={1}, fileFormat={2}") |
| 76 | + protected static List<Object[]> parameters() { |
| 77 | + return Arrays.asList( |
| 78 | + new Object[] {"testhadoop", Namespace.empty(), FileFormat.PARQUET}, |
| 79 | + new Object[] {"testhadoop", Namespace.empty(), FileFormat.AVRO}, |
| 80 | + new Object[] {"testhadoop", Namespace.empty(), FileFormat.ORC}); |
| 81 | + } |
| 82 | + |
| 83 | + @Override |
| 84 | + @BeforeEach |
| 85 | + public void before() { |
| 86 | + super.before(); |
| 87 | + sql("CREATE DATABASE %s", flinkDatabase); |
| 88 | + sql("USE CATALOG %s", catalogName); |
| 89 | + sql("USE %s", DATABASE); |
| 90 | + } |
| 91 | + |
| 92 | + /** Writes UUID via Generic writer, reads via Flink. */ |
| 93 | + @TestTemplate |
| 94 | + void uuidWrittenByGenericWriter() throws Exception { |
| 95 | + icebergTable = |
| 96 | + validationCatalog.createTable( |
| 97 | + TableIdentifier.of(icebergNamespace, TABLE_NAME), |
| 98 | + SCHEMA, |
| 99 | + PartitionSpec.unpartitioned(), |
| 100 | + ImmutableMap.of(DEFAULT_FILE_FORMAT, fileFormat.name())); |
| 101 | + |
| 102 | + Record record = |
| 103 | + GenericRecord.create(icebergTable.schema()).copy("id", 1, "uuid", EXPECTED_UUID); |
| 104 | + new GenericAppenderHelper(icebergTable, fileFormat, warehouseDir) |
| 105 | + .appendToTable(ImmutableList.of(record)); |
| 106 | + icebergTable.refresh(); |
| 107 | + |
| 108 | + List<GenericRowData> genericRowData = Lists.newArrayList(); |
| 109 | + try (CloseableIterable<CombinedScanTask> combinedScanTasks = |
| 110 | + icebergTable.newScan().planTasks()) { |
| 111 | + for (CombinedScanTask combinedScanTask : combinedScanTasks) { |
| 112 | + try (DataIterator<RowData> dataIterator = |
| 113 | + ReaderUtil.createDataIterator( |
| 114 | + combinedScanTask, icebergTable.schema(), icebergTable.schema())) { |
| 115 | + while (dataIterator.hasNext()) { |
| 116 | + GenericRowData rowData = (GenericRowData) dataIterator.next(); |
| 117 | + genericRowData.add(rowData); |
| 118 | + } |
| 119 | + } |
| 120 | + } |
| 121 | + } |
| 122 | + |
| 123 | + assertThat(genericRowData).hasSize(1); |
| 124 | + assertThat(genericRowData.get(0).getField(1)).isInstanceOf(byte[].class); |
| 125 | + byte[] bytes = (byte[]) genericRowData.get(0).getField(1); |
| 126 | + assertThat(bytes).hasSize(16); |
| 127 | + |
| 128 | + ByteBuffer byteBuffer = ByteBuffer.wrap(bytes); |
| 129 | + UUID actualUuid = new UUID(byteBuffer.getLong(), byteBuffer.getLong()); |
| 130 | + assertThat(actualUuid).isEqualTo(EXPECTED_UUID); |
| 131 | + } |
| 132 | + |
| 133 | + /** Writes UUID via Flink TaskWriter, reads via Generic reader. */ |
| 134 | + @TestTemplate |
| 135 | + void writeUuidViaFlinkWriter() throws Exception { |
| 136 | + icebergTable = |
| 137 | + validationCatalog.createTable( |
| 138 | + TableIdentifier.of(icebergNamespace, TABLE_NAME), |
| 139 | + SCHEMA, |
| 140 | + PartitionSpec.unpartitioned(), |
| 141 | + ImmutableMap.of(DEFAULT_FILE_FORMAT, fileFormat.name())); |
| 142 | + |
| 143 | + RowType rowType = FlinkSchemaUtil.convert(SCHEMA); |
| 144 | + RowDataTaskWriterFactory rowDataTaskWriterFactory = |
| 145 | + new RowDataTaskWriterFactory( |
| 146 | + icebergTable, |
| 147 | + rowType, |
| 148 | + TARGET_FILE_SIZE, |
| 149 | + fileFormat, |
| 150 | + icebergTable.properties(), |
| 151 | + null, |
| 152 | + false); |
| 153 | + rowDataTaskWriterFactory.initialize(1, 1); |
| 154 | + |
| 155 | + byte[] uuidBytes = UUIDUtil.convert(EXPECTED_UUID); |
| 156 | + GenericRowData genericRowData = GenericRowData.of(1, uuidBytes); |
| 157 | + |
| 158 | + try (TaskWriter<RowData> writer = rowDataTaskWriterFactory.create()) { |
| 159 | + writer.write(genericRowData); |
| 160 | + writer.close(); |
| 161 | + |
| 162 | + AppendFiles append = icebergTable.newAppend(); |
| 163 | + for (DataFile dataFile : writer.dataFiles()) { |
| 164 | + append.appendFile(dataFile); |
| 165 | + } |
| 166 | + |
| 167 | + append.commit(); |
| 168 | + } |
| 169 | + |
| 170 | + List<Record> records = SimpleDataUtil.tableRecords(icebergTable); |
| 171 | + assertThat(records).hasSize(1); |
| 172 | + assertThat(records.get(0).getField("uuid")).isEqualTo(EXPECTED_UUID); |
| 173 | + } |
| 174 | + |
| 175 | + /** Writes UUID via SQL INSERT, reads via Generic reader. */ |
| 176 | + @TestTemplate |
| 177 | + void sqlInsertUuid() throws Exception { |
| 178 | + icebergTable = |
| 179 | + validationCatalog.createTable( |
| 180 | + TableIdentifier.of(icebergNamespace, TABLE_NAME), |
| 181 | + SCHEMA, |
| 182 | + PartitionSpec.unpartitioned(), |
| 183 | + ImmutableMap.of(DEFAULT_FILE_FORMAT, fileFormat.name())); |
| 184 | + |
| 185 | + String uuidHex = EXPECTED_UUID.toString().replace("-", ""); |
| 186 | + sql("INSERT INTO %s VALUES (1, CAST(X'%s' AS BINARY(16)))", TABLE_NAME, uuidHex); |
| 187 | + |
| 188 | + List<Record> records = SimpleDataUtil.tableRecords(icebergTable); |
| 189 | + assertThat(records).hasSize(1); |
| 190 | + assertThat(records.get(0).getField("uuid")).isEqualTo(EXPECTED_UUID); |
| 191 | + } |
| 192 | +} |
0 commit comments