Skip to content

Commit

Permalink
[Improve][Iceberg] Add savemode create table primaryKey testcase (#7641)
Browse files Browse the repository at this point in the history
  • Loading branch information
hailin0 committed Sep 12, 2024
1 parent 87ec786 commit 6b36f90
Show file tree
Hide file tree
Showing 3 changed files with 36 additions and 29 deletions.
Original file line number Diff line number Diff line change
Expand Up @@ -23,6 +23,7 @@
import org.apache.seatunnel.api.table.catalog.InfoPreviewResult;
import org.apache.seatunnel.api.table.catalog.PhysicalColumn;
import org.apache.seatunnel.api.table.catalog.PreviewResult;
import org.apache.seatunnel.api.table.catalog.PrimaryKey;
import org.apache.seatunnel.api.table.catalog.TablePath;
import org.apache.seatunnel.api.table.catalog.TableSchema;
import org.apache.seatunnel.api.table.catalog.exception.CatalogException;
Expand All @@ -36,6 +37,7 @@
import org.apache.seatunnel.connectors.seatunnel.iceberg.utils.SchemaUtils;

import org.apache.iceberg.PartitionField;
import org.apache.iceberg.Schema;
import org.apache.iceberg.Snapshot;
import org.apache.iceberg.Table;
import org.apache.iceberg.catalog.Namespace;
Expand All @@ -48,8 +50,11 @@

import java.io.Closeable;
import java.io.IOException;
import java.util.ArrayList;
import java.util.List;
import java.util.Optional;
import java.util.Set;
import java.util.function.Function;
import java.util.stream.Collectors;

import static com.google.common.base.Preconditions.checkArgument;
Expand Down Expand Up @@ -222,7 +227,8 @@ public void truncateTable(TablePath tablePath, boolean ignoreIfNotExists)
}

public CatalogTable toCatalogTable(Table icebergTable, TablePath tablePath) {
List<Types.NestedField> columns = icebergTable.schema().columns();
Schema schema = icebergTable.schema();
List<Types.NestedField> columns = schema.columns();
TableSchema.Builder builder = TableSchema.builder();
columns.forEach(
nestedField -> {
Expand All @@ -234,12 +240,19 @@ public CatalogTable toCatalogTable(Table icebergTable, TablePath tablePath) {
name,
seaTunnelType,
(Long) null,
true,
nestedField.isOptional(),
null,
nestedField.doc());
builder.column(physicalColumn);
});

Optional.ofNullable(schema.identifierFieldNames())
.map(
(Function<Set<String>, Object>)
names ->
builder.primaryKey(
PrimaryKey.of(
tablePath.getTableName() + "_pk",
new ArrayList<>(names))));
List<String> partitionKeys =
icebergTable.spec().fields().stream()
.map(PartitionField::name)
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -36,7 +36,6 @@
import org.apache.seatunnel.connectors.seatunnel.iceberg.sink.schema.SchemaDeleteColumn;
import org.apache.seatunnel.connectors.seatunnel.iceberg.sink.schema.SchemaModifyColumn;

import org.apache.commons.collections.CollectionUtils;
import org.apache.iceberg.PartitionSpec;
import org.apache.iceberg.Schema;
import org.apache.iceberg.Table;
Expand All @@ -54,17 +53,17 @@
import lombok.extern.slf4j.Slf4j;

import java.util.ArrayList;
import java.util.Collections;
import java.util.HashMap;
import java.util.HashSet;
import java.util.List;
import java.util.Map;
import java.util.Objects;
import java.util.Optional;
import java.util.Set;
import java.util.concurrent.atomic.AtomicInteger;
import java.util.concurrent.atomic.AtomicReference;
import java.util.regex.Matcher;
import java.util.regex.Pattern;
import java.util.stream.Collectors;

import static java.util.stream.Collectors.toList;

Expand Down Expand Up @@ -161,27 +160,17 @@ private static Table createTable(
@NotNull protected static Schema toIcebergSchema(
TableSchema tableSchema, ReadonlyConfig readonlyConfig) {
Types.StructType structType = SchemaUtils.toIcebergType(tableSchema);
Set<Integer> identifierFieldIds = new HashSet<>();
if (Objects.nonNull(readonlyConfig)) {
List<String> pks =
SinkConfig.stringToList(readonlyConfig.get(SinkConfig.TABLE_PRIMARY_KEYS), ",");
if (CollectionUtils.isNotEmpty(pks)) {
for (String pk : pks) {
Optional<Integer> pkId =
structType.fields().stream()
.filter(nestedField -> nestedField.name().equals(pk))
.map(Types.NestedField::fieldId)
.findFirst();
if (!pkId.isPresent()) {
throw new IllegalArgumentException(
String.format(
"iceberg table pk:%s not present in the incoming struct",
pk));
}
identifierFieldIds.add(pkId.get());
}
}
}
Set<Integer> identifierFieldIds =
readonlyConfig.getOptional(SinkConfig.TABLE_PRIMARY_KEYS)
.map(e -> SinkConfig.stringToList(e, ","))
.orElseGet(
() ->
Optional.ofNullable(tableSchema.getPrimaryKey())
.map(e -> e.getColumnNames())
.orElse(Collections.emptyList()))
.stream()
.map(f -> structType.field(f).fieldId())
.collect(Collectors.toSet());
List<Types.NestedField> fields = new ArrayList<>();
structType
.fields()
Expand Down Expand Up @@ -281,7 +270,7 @@ public static Types.StructType toIcebergType(TableSchema tableSchema) {
Types.NestedField icebergField =
Types.NestedField.of(
idIncrementer.getAndIncrement(),
true,
column.isNullable(),
column.getName(),
IcebergTypeMapper.toIcebergType(column.getDataType(), idIncrementer),
column.getComment());
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -20,6 +20,7 @@
import org.apache.seatunnel.api.configuration.ReadonlyConfig;
import org.apache.seatunnel.api.table.catalog.CatalogTable;
import org.apache.seatunnel.api.table.catalog.PhysicalColumn;
import org.apache.seatunnel.api.table.catalog.PrimaryKey;
import org.apache.seatunnel.api.table.catalog.TableIdentifier;
import org.apache.seatunnel.api.table.catalog.TablePath;
import org.apache.seatunnel.api.table.catalog.TableSchema;
Expand Down Expand Up @@ -152,7 +153,8 @@ void dropTable() {
CatalogTable buildAllTypesTable(TableIdentifier tableIdentifier) {
TableSchema.Builder builder = TableSchema.builder();
builder.column(
PhysicalColumn.of("id", BasicType.INT_TYPE, (Long) null, true, null, "id comment"));
PhysicalColumn.of(
"id", BasicType.INT_TYPE, (Long) null, false, null, "id comment"));
builder.column(
PhysicalColumn.of(
"boolean_col", BasicType.BOOLEAN_TYPE, (Long) null, true, null, null));
Expand Down Expand Up @@ -185,6 +187,9 @@ CatalogTable buildAllTypesTable(TableIdentifier tableIdentifier) {
PhysicalColumn.of(
"decimal_col", new DecimalType(38, 18), (Long) null, true, null, null));
builder.column(PhysicalColumn.of("dt_col", STRING_TYPE, (Long) null, true, null, null));
builder.primaryKey(
PrimaryKey.of(
tableIdentifier.getTableName() + "_pk", Collections.singletonList("id")));

TableSchema schema = builder.build();
HashMap<String, String> options = new HashMap<>();
Expand Down

0 comments on commit 6b36f90

Please sign in to comment.