Skip to content
Merged
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
Original file line number Diff line number Diff line change
Expand Up @@ -140,6 +140,7 @@ public ParquetValueReader<?> struct(
// match the expected struct's order
Map<Integer, ParquetValueReader<?>> readersById = Maps.newHashMap();
Map<Integer, Type> typesById = Maps.newHashMap();
Map<Integer, Integer> maxDefinitionLevelsById = Maps.newHashMap();
List<Type> fields = struct.getFields();
for (int i = 0; i < fields.size(); i += 1) {
Type fieldType = fields.get(i);
Expand All @@ -148,6 +149,9 @@ public ParquetValueReader<?> struct(
int id = fieldType.getId().intValue();
readersById.put(id, ParquetValueReaders.option(fieldType, fieldD, fieldReaders.get(i)));
typesById.put(id, fieldType);
if (idToConstant.containsKey(id)) {
maxDefinitionLevelsById.put(id, fieldD);
}
}
}

Expand All @@ -156,11 +160,16 @@ public ParquetValueReader<?> struct(
List<ParquetValueReader<?>> reorderedFields =
Lists.newArrayListWithExpectedSize(expectedFields.size());
List<Type> types = Lists.newArrayListWithExpectedSize(expectedFields.size());
// Defaulting to parent max definition level
int defaultMaxDefinitionLevel = type.getMaxDefinitionLevel(currentPath());
for (Types.NestedField field : expectedFields) {
int id = field.fieldId();
if (idToConstant.containsKey(id)) {
// containsKey is used because the constant may be null
reorderedFields.add(ParquetValueReaders.constant(idToConstant.get(id)));
int fieldMaxDefinitionLevel =
maxDefinitionLevelsById.getOrDefault(id, defaultMaxDefinitionLevel);
reorderedFields.add(
ParquetValueReaders.constant(idToConstant.get(id), fieldMaxDefinitionLevel));
types.add(null);
} else if (id == MetadataColumns.ROW_POSITION.fieldId()) {
reorderedFields.add(ParquetValueReaders.position());
Expand Down
Original file line number Diff line number Diff line change
@@ -0,0 +1,74 @@
/*
* 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.spark.source;

import org.apache.iceberg.relocated.com.google.common.base.MoreObjects;
import org.apache.iceberg.relocated.com.google.common.base.Objects;

public class ComplexRecord {
private long id;
private NestedRecord struct;

public ComplexRecord() {}

public ComplexRecord(long id, NestedRecord struct) {
this.id = id;
this.struct = struct;
}

public long getId() {
return id;
}

public void setId(long id) {
this.id = id;
}

public NestedRecord getStruct() {
return struct;
}

public void setStruct(NestedRecord struct) {
this.struct = struct;
}

@Override
public boolean equals(Object o) {
if (this == o) {
return true;
}

if (o == null || getClass() != o.getClass()) {
return false;
}

ComplexRecord record = (ComplexRecord) o;
return id == record.id && Objects.equal(struct, record.struct);
}

@Override
public int hashCode() {
return Objects.hashCode(id, struct);
}

@Override
public String toString() {
return MoreObjects.toStringHelper(this).add("id", id).add("struct", struct).toString();
}
}
Original file line number Diff line number Diff line change
@@ -0,0 +1,77 @@
/*
* 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.spark.source;

import org.apache.iceberg.relocated.com.google.common.base.MoreObjects;
import org.apache.iceberg.relocated.com.google.common.base.Objects;

public class NestedRecord {
private long innerId;
private String innerName;

public NestedRecord() {}

public NestedRecord(long innerId, String innerName) {
this.innerId = innerId;
this.innerName = innerName;
}

public long getInnerId() {
return innerId;
}

public String getInnerName() {
return innerName;
}

public void setInnerId(long iId) {
innerId = iId;
}

public void setInnerName(String name) {
innerName = name;
}

@Override
public boolean equals(Object o) {
if (this == o) {
return true;
}

if (o == null || getClass() != o.getClass()) {
return false;
}

NestedRecord that = (NestedRecord) o;
return innerId == that.innerId && Objects.equal(innerName, that.innerName);
}

@Override
public int hashCode() {
return Objects.hashCode(innerId, innerName);
}

@Override
public String toString() {
return MoreObjects.toStringHelper(this)
.add("innerId", innerId)
.add("innerName", innerName)
.toString();
}
}
Original file line number Diff line number Diff line change
Expand Up @@ -23,6 +23,8 @@

import java.io.File;
import java.util.List;
import java.util.stream.Collectors;
import java.util.stream.IntStream;
import org.apache.avro.generic.GenericData;
import org.apache.iceberg.DataFiles;
import org.apache.iceberg.Files;
Expand All @@ -39,6 +41,7 @@
import org.apache.iceberg.spark.data.RandomData;
import org.apache.iceberg.spark.data.TestHelpers;
import org.apache.iceberg.types.Types;
import org.apache.spark.api.java.function.MapFunction;
import org.apache.spark.sql.Dataset;
import org.apache.spark.sql.Encoders;
import org.apache.spark.sql.Row;
Expand All @@ -51,6 +54,7 @@
import org.apache.spark.sql.types.StructType;
import org.junit.AfterClass;
import org.junit.Assert;
import org.junit.Assume;
import org.junit.BeforeClass;
import org.junit.Rule;
import org.junit.Test;
Expand Down Expand Up @@ -436,4 +440,54 @@ public void testPartitionedByNestedString() throws Exception {

Assert.assertEquals("Number of rows should match", rows.size(), actual.size());
}

@Test
public void testReadPartitionColumn() throws Exception {
Assume.assumeTrue("Temporary skip ORC", !"orc".equals(format));

Schema nestedSchema =
new Schema(
Types.NestedField.optional(1, "id", Types.LongType.get()),
Types.NestedField.optional(
2,
"struct",
Types.StructType.of(
Types.NestedField.optional(3, "innerId", Types.LongType.get()),
Types.NestedField.optional(4, "innerName", Types.StringType.get()))));
PartitionSpec spec =
PartitionSpec.builderFor(nestedSchema).identity("struct.innerName").build();

// create table
HadoopTables tables = new HadoopTables(spark.sessionState().newHadoopConf());
String baseLocation = temp.newFolder("partition_by_nested_string").toString();
Table table = tables.create(nestedSchema, spec, baseLocation);
table.updateProperties().set(TableProperties.DEFAULT_FILE_FORMAT, format).commit();

// write into iceberg
MapFunction<Long, ComplexRecord> func =
value -> new ComplexRecord(value, new NestedRecord(value, "name_" + value));
spark
.range(0, 10, 1, 1)
.map(func, Encoders.bean(ComplexRecord.class))
.write()
.format("iceberg")
.mode(SaveMode.Append)
.save(baseLocation);

List<String> actual =
spark
.read()
.format("iceberg")
.option(SparkReadOptions.VECTORIZATION_ENABLED, String.valueOf(vectorized))
.load(baseLocation)
.select("struct.innerName")
.as(Encoders.STRING())
.collectAsList();

Assert.assertEquals("Number of rows should match", 10, actual.size());

List<String> inputRecords =
IntStream.range(0, 10).mapToObj(i -> "name_" + i).collect(Collectors.toList());
Assert.assertEquals("Read object should be matched", inputRecords, actual);
}
}
Original file line number Diff line number Diff line change
Expand Up @@ -140,6 +140,7 @@ public ParquetValueReader<?> struct(
// match the expected struct's order
Map<Integer, ParquetValueReader<?>> readersById = Maps.newHashMap();
Map<Integer, Type> typesById = Maps.newHashMap();
Map<Integer, Integer> maxDefinitionLevelsById = Maps.newHashMap();
List<Type> fields = struct.getFields();
for (int i = 0; i < fields.size(); i += 1) {
Type fieldType = fields.get(i);
Expand All @@ -148,6 +149,9 @@ public ParquetValueReader<?> struct(
int id = fieldType.getId().intValue();
readersById.put(id, ParquetValueReaders.option(fieldType, fieldD, fieldReaders.get(i)));
typesById.put(id, fieldType);
if (idToConstant.containsKey(id)) {
maxDefinitionLevelsById.put(id, fieldD);
}
}
}

Expand All @@ -156,11 +160,16 @@ public ParquetValueReader<?> struct(
List<ParquetValueReader<?>> reorderedFields =
Lists.newArrayListWithExpectedSize(expectedFields.size());
List<Type> types = Lists.newArrayListWithExpectedSize(expectedFields.size());
// Defaulting to parent max definition level
int defaultMaxDefinitionLevel = type.getMaxDefinitionLevel(currentPath());
for (Types.NestedField field : expectedFields) {
int id = field.fieldId();
if (idToConstant.containsKey(id)) {
// containsKey is used because the constant may be null
reorderedFields.add(ParquetValueReaders.constant(idToConstant.get(id)));
int fieldMaxDefinitionLevel =
maxDefinitionLevelsById.getOrDefault(id, defaultMaxDefinitionLevel);
reorderedFields.add(
ParquetValueReaders.constant(idToConstant.get(id), fieldMaxDefinitionLevel));
types.add(null);
} else if (id == MetadataColumns.ROW_POSITION.fieldId()) {
reorderedFields.add(ParquetValueReaders.position());
Expand Down
Original file line number Diff line number Diff line change
@@ -0,0 +1,74 @@
/*
* 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.spark.source;

import org.apache.iceberg.relocated.com.google.common.base.MoreObjects;
import org.apache.iceberg.relocated.com.google.common.base.Objects;

public class ComplexRecord {
private long id;
private NestedRecord struct;

public ComplexRecord() {}

public ComplexRecord(long id, NestedRecord struct) {
this.id = id;
this.struct = struct;
}

public long getId() {
return id;
}

public void setId(long id) {
this.id = id;
}

public NestedRecord getStruct() {
return struct;
}

public void setStruct(NestedRecord struct) {
this.struct = struct;
}

@Override
public boolean equals(Object o) {
if (this == o) {
return true;
}

if (o == null || getClass() != o.getClass()) {
return false;
}

ComplexRecord record = (ComplexRecord) o;
return id == record.id && Objects.equal(struct, record.struct);
}

@Override
public int hashCode() {
return Objects.hashCode(id, struct);
}

@Override
public String toString() {
return MoreObjects.toStringHelper(this).add("id", id).add("struct", struct).toString();
}
}
Loading