Skip to content
Closed
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
37 changes: 35 additions & 2 deletions api/src/main/java/org/apache/iceberg/types/TypeUtil.java
Original file line number Diff line number Diff line change
Expand Up @@ -38,6 +38,7 @@
import org.apache.iceberg.relocated.com.google.common.collect.Iterables;
import org.apache.iceberg.relocated.com.google.common.collect.Lists;
import org.apache.iceberg.relocated.com.google.common.collect.Maps;
import org.apache.iceberg.relocated.com.google.common.collect.Sets;

public class TypeUtil {

Expand Down Expand Up @@ -130,15 +131,47 @@ private static Set<Integer> getIdsInternal(Type type, boolean includeStructIds)
return visit(type, new GetProjectedIds(includeStructIds));
}

private static Set<Integer> innerCheck(Set<Integer> filteredIds, Types.NestedField innerField) {
// If it's a struct and if all ids need to be filtered we keep all.
// So we remove all ids from filtered ids set
if (innerField.type().isStructType()) {
if (innerField.type().asStructType().fields()
.stream().allMatch(f -> filteredIds.contains(f.fieldId()))) {
filteredIds.removeAll(innerField.type().asStructType().fields().stream()
.map(Types.NestedField::fieldId).collect(Collectors.toSet()));
}
innerField.type().asStructType().fields().forEach(f -> innerCheck(filteredIds, f));
}
return filteredIds;
}

private static void filter(Set<Integer> projectedIds, Types.StructType struct, Set<Integer> fieldIds,
boolean doesNeedToKeepInnerStruct) {
Set<Integer> filteredIds = Sets.newHashSet(fieldIds);
if (doesNeedToKeepInnerStruct) {
struct.fields().forEach(f -> innerCheck(filteredIds, f));
}
projectedIds.removeAll(filteredIds);
}

public static Types.StructType selectNot(Types.StructType struct, Set<Integer> fieldIds) {
return selectNot(struct, fieldIds, false);
}

public static Types.StructType selectNot(Types.StructType struct, Set<Integer> fieldIds,
boolean doesNeedToKeepInnerStruct) {
Set<Integer> projectedIds = getIdsInternal(struct, false);
projectedIds.removeAll(fieldIds);
filter(projectedIds, struct, fieldIds, doesNeedToKeepInnerStruct);
return project(struct, projectedIds);
}

public static Schema selectNot(Schema schema, Set<Integer> fieldIds) {
return selectNot(schema, fieldIds, false);
}

public static Schema selectNot(Schema schema, Set<Integer> fieldIds, boolean doesNeedToKeepInnerStruct) {
Set<Integer> projectedIds = getIdsInternal(schema.asStruct(), false);
projectedIds.removeAll(fieldIds);
filter(projectedIds, schema.asStruct(), fieldIds, doesNeedToKeepInnerStruct);
return project(schema, projectedIds);
}

Expand Down
55 changes: 55 additions & 0 deletions api/src/test/java/org/apache/iceberg/types/TestTypeUtil.java
Original file line number Diff line number Diff line change
Expand Up @@ -482,4 +482,59 @@ public void testSelectNot() {
Schema actualNoStruct = TypeUtil.selectNot(schema, Sets.newHashSet(2));
Assert.assertEquals(schema.asStruct(), actualNoStruct.asStruct());
}

@Test
public void testSelectNotInnerStruct() {
Schema schema = new Schema(
Lists.newArrayList(
required(1, "id", Types.LongType.get()),
required(2, "location", Types.StructType.of(
required(3, "lat", Types.DoubleType.get()),
required(4, "long", Types.DoubleType.get()),
required(5, "address", Types.StructType.of(
required(6, "number", Types.DoubleType.get()),
required(7, "street", Types.DoubleType.get())
)
)))));

// Expected that filter inner struct if all struct is not filtered
Schema expectedFilteredStruct = new Schema(
Lists.newArrayList(
required(1, "id", Types.LongType.get()),
required(2, "location", Types.StructType.of(
required(4, "long", Types.DoubleType.get()),
required(5, "address", Types.StructType.of(
required(6, "number", Types.DoubleType.get()),
required(7, "street", Types.DoubleType.get())
)
)))));
Schema actualFilteredStruct = TypeUtil.selectNot(schema, Sets.newHashSet(3),
true /* doesNeedToKeepInnerStruct */);
Assert.assertEquals(expectedFilteredStruct.asStruct(), actualFilteredStruct.asStruct());

// Expected that do not filter inner struct if all struct is filtered
Schema actualNotInnerStruct = TypeUtil.selectNot(schema, Sets.newHashSet(3, 4, 5),
true /* doesNeedToKeepInnerStruct */);
Assert.assertEquals(schema.asStruct(), actualNotInnerStruct.asStruct());

Schema actualNotSubInnerStruct = TypeUtil.selectNot(schema, Sets.newHashSet(6, 7),
true /* doesNeedToKeepInnerStruct */);
Assert.assertEquals(schema.asStruct(), actualNotSubInnerStruct.asStruct());

Schema expectedFilteredInnerStruct = new Schema(
Lists.newArrayList(
required(1, "id", Types.LongType.get()),
required(2, "location", Types.StructType.of(
required(3, "lat", Types.DoubleType.get()),
required(4, "long", Types.DoubleType.get()),
required(5, "address", Types.StructType.of(
required(7, "street", Types.DoubleType.get())
)
)))));

Schema actualSubInnerStruct = TypeUtil.selectNot(schema, Sets.newHashSet(6),
true /* doesNeedToFilteredInnerStruct */);
Assert.assertEquals(expectedFilteredInnerStruct.asStruct(), actualSubInnerStruct.asStruct());
}

}
Original file line number Diff line number Diff line change
Expand Up @@ -125,7 +125,7 @@ private CloseableIterable<Record> openFile(FileScanTask task, Schema fileProject

case ORC:
Schema projectionWithoutConstantAndMetadataFields = TypeUtil.selectNot(fileProjection,
Sets.union(partition.keySet(), MetadataColumns.metadataFieldIds()));
Sets.union(partition.keySet(), MetadataColumns.metadataFieldIds()), true);
ORC.ReadBuilder orc = ORC.read(input)
.project(projectionWithoutConstantAndMetadataFields)
.createReaderFunc(fileSchema -> GenericOrcReader.buildReader(fileProjection, fileSchema, partition))
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -149,7 +149,7 @@ private CloseableIterable<RowData> newParquetIterable(
private CloseableIterable<RowData> newOrcIterable(
FileScanTask task, Schema schema, Map<Integer, ?> idToConstant, InputFilesDecryptor inputFilesDecryptor) {
Schema readSchemaWithoutConstantAndMetadataFields = TypeUtil.selectNot(schema,
Sets.union(idToConstant.keySet(), MetadataColumns.metadataFieldIds()));
Sets.union(idToConstant.keySet(), MetadataColumns.metadataFieldIds()), true);

ORC.ReadBuilder builder = ORC.read(inputFilesDecryptor.getInputFile(task))
.project(readSchemaWithoutConstantAndMetadataFields)
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -393,7 +393,7 @@ private CloseableIterable<T> newParquetIterable(InputFile inputFile, FileScanTas
private CloseableIterable<T> newOrcIterable(InputFile inputFile, FileScanTask task, Schema readSchema) {
Map<Integer, ?> idToConstant = constantsMap(task, IdentityPartitionConverters::convertConstant);
Schema readSchemaWithoutConstantAndMetadataFields = TypeUtil.selectNot(readSchema,
Sets.union(idToConstant.keySet(), MetadataColumns.metadataFieldIds()));
Sets.union(idToConstant.keySet(), MetadataColumns.metadataFieldIds()), true);

CloseableIterable<T> orcIterator = null;
// ORC does not support reuse containers yet
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -22,6 +22,7 @@
import java.util.Map;
import org.apache.iceberg.PartitionSpec;
import org.apache.iceberg.Table;
import org.apache.iceberg.relocated.com.google.common.collect.ImmutableList;
import org.apache.iceberg.spark.source.SparkTable;
import org.apache.spark.sql.connector.catalog.CatalogManager;
import org.apache.spark.sql.connector.catalog.Identifier;
Expand Down Expand Up @@ -399,6 +400,26 @@ public void testSparkTableAddDropPartitions() throws Exception {
Assert.assertEquals("spark table partition should be empty", 0, sparkTable().partitioning().length);
}

@Test
public void testAddPartitionStruct() {
FORMATS.forEach(this::testAddPartitionInAStruct);
}

private void testAddPartitionInAStruct(String format) {
sql("DROP TABLE IF EXISTS %s", tableName);
sql("CREATE TABLE %s (id INT, st struct<id:INT, data:string>) USING iceberg " +
"TBLPROPERTIES ('write.format.default'='%s')", tableName, format);
sql("INSERT INTO TABLE %s VALUES (1, named_struct('id', 10, 'data', 'v1'))," +
"(2, named_struct('id', 20, 'data', 'v2'))", tableName);

sql("ALTER TABLE %s ADD PARTITION FIELD st.data as data", tableName);

sql("INSERT INTO TABLE %s VALUES (1, named_struct('id', 11, 'data', 'v1'))", tableName);
ImmutableList<Object[]> expectedRows = ImmutableList.of(row("v1"), row("v2"));
assertEquals("Should have expected rows", expectedRows,
sql("select distinct st.data from %s order by data", tableName));
}

private void assertPartitioningEquals(SparkTable table, int len, String transform) {
Assert.assertEquals("spark table partition should be " + len, len, table.partitioning().length);
Assert.assertEquals("latest spark table partition transform should match",
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -103,7 +103,8 @@ CloseableIterator<ColumnarBatch> open(FileScanTask task) {
Set<Integer> constantFieldIds = idToConstant.keySet();
Set<Integer> metadataFieldIds = MetadataColumns.metadataFieldIds();
Sets.SetView<Integer> constantAndMetadataFieldIds = Sets.union(constantFieldIds, metadataFieldIds);
Schema schemaWithoutConstantAndMetadataFields = TypeUtil.selectNot(expectedSchema, constantAndMetadataFieldIds);
Schema schemaWithoutConstantAndMetadataFields = TypeUtil.selectNot(
expectedSchema, constantAndMetadataFieldIds, true);
ORC.ReadBuilder builder = ORC.read(location)
.project(schemaWithoutConstantAndMetadataFields)
.split(task.start(), task.length())
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -155,7 +155,7 @@ private CloseableIterable<InternalRow> newOrcIterable(
Schema readSchema,
Map<Integer, ?> idToConstant) {
Schema readSchemaWithoutConstantAndMetadataFields = TypeUtil.selectNot(readSchema,
Sets.union(idToConstant.keySet(), MetadataColumns.metadataFieldIds()));
Sets.union(idToConstant.keySet(), MetadataColumns.metadataFieldIds()), true);

ORC.ReadBuilder builder = ORC.read(location)
.project(readSchemaWithoutConstantAndMetadataFields)
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -39,6 +39,7 @@
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.Iterables;
import org.apache.iceberg.relocated.com.google.common.collect.Lists;
import org.apache.iceberg.relocated.com.google.common.collect.Maps;
import org.apache.spark.sql.Dataset;
import org.apache.spark.sql.Encoders;
Expand All @@ -54,6 +55,7 @@
public abstract class SparkTestBase {

protected static final Object ANY = new Object();
protected static final List<String> FORMATS = Lists.newArrayList("orc", "parquet", "avro");

protected static TestHiveMetastore metastore = null;
protected static HiveConf hiveConf = null;
Expand Down