Skip to content
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

[Bug][Connector][FileBase]Parquet reader parsing array type exception. #4457

Merged
merged 14 commits into from
Nov 28, 2023
Original file line number Diff line number Diff line change
Expand Up @@ -37,6 +37,7 @@
import org.apache.avro.data.TimeConversions;
import org.apache.avro.generic.GenericData;
import org.apache.avro.generic.GenericRecord;
import org.apache.avro.util.Utf8;
import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.fs.FSDataInputStream;
import org.apache.hadoop.fs.FileSystem;
Expand Down Expand Up @@ -131,7 +132,16 @@ private Object resolveObject(Object field, SeaTunnelDataType<?> fieldType) {
switch (fieldType.getSqlType()) {
case ARRAY:
ArrayList<Object> origArray = new ArrayList<>();
((GenericData.Array<?>) field).iterator().forEachRemaining(origArray::add);
((GenericData.Array<?>) field)
.iterator()
.forEachRemaining(
ele -> {
if (ele instanceof Utf8) {
origArray.add(ele.toString());
} else {
origArray.add(ele);
}
});
SeaTunnelDataType<?> elementType = ((ArrayType<?, ?>) fieldType).getElementType();
switch (elementType.getSqlType()) {
case STRING:
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -21,6 +21,7 @@
import org.apache.seatunnel.shade.com.typesafe.config.ConfigFactory;

import org.apache.seatunnel.api.source.Collector;
import org.apache.seatunnel.api.table.type.ArrayType;
import org.apache.seatunnel.api.table.type.SeaTunnelRow;
import org.apache.seatunnel.api.table.type.SeaTunnelRowType;
import org.apache.seatunnel.connectors.seatunnel.file.config.HadoopConf;
Expand Down Expand Up @@ -121,6 +122,22 @@ public void testParquetReadProjection2() throws Exception {
parquetReadStrategy.read(path, testCollector);
}

@Test
public void testParquetReadArray() throws Exception {
URL resource = ParquetReadStrategyTest.class.getResource("/array.parquet");
Assertions.assertNotNull(resource);
String path = Paths.get(resource.toURI()).toString();
ParquetReadStrategy parquetReadStrategy = new ParquetReadStrategy();
LocalConf localConf = new LocalConf(FS_DEFAULT_NAME_DEFAULT);
parquetReadStrategy.init(localConf);
SeaTunnelRowType seaTunnelRowTypeInfo =
parquetReadStrategy.getSeaTunnelRowTypeInfo(localConf, path);
Assertions.assertNotNull(seaTunnelRowTypeInfo);
Assertions.assertEquals(seaTunnelRowTypeInfo.getFieldType(2).getClass(), ArrayType.class);
TestCollector testCollector = new TestCollector();
parquetReadStrategy.read(path, testCollector);
}

public static class TestCollector implements Collector<SeaTunnelRow> {

private final List<SeaTunnelRow> rows = new ArrayList<>();
Expand Down
Copy link
Member

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Based on #5545, we need to remove the binary and automatically generate it

Copy link
Member

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

+1

Copy link
Contributor Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

done.

Binary file not shown.