Skip to content

[Fix][connector-file-base] fix parquet int32 convert error #9142

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

Merged
merged 4 commits into from
Apr 17, 2025
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 @@ -297,6 +297,8 @@ private SeaTunnelDataType<?> parquetType2SeaTunnelType(Type type, String name) {
return BasicType.BYTE_TYPE;
case INT_16:
return BasicType.SHORT_TYPE;
case INT_32:
return BasicType.INT_TYPE;
case DATE:
return LocalTimeType.LOCAL_DATE_TYPE;
default:
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -36,8 +36,17 @@
import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.fs.Path;
import org.apache.parquet.avro.AvroParquetWriter;
import org.apache.parquet.example.data.Group;
import org.apache.parquet.example.data.simple.SimpleGroup;
import org.apache.parquet.hadoop.ParquetFileReader;
import org.apache.parquet.hadoop.ParquetWriter;
import org.apache.parquet.hadoop.example.ExampleParquetWriter;
import org.apache.parquet.hadoop.example.GroupWriteSupport;
import org.apache.parquet.hadoop.metadata.CompressionCodecName;
import org.apache.parquet.hadoop.util.HadoopInputFile;
import org.apache.parquet.schema.LogicalTypeAnnotation;
import org.apache.parquet.schema.MessageType;
import org.apache.parquet.schema.Types;

import org.junit.jupiter.api.Assertions;
import org.junit.jupiter.api.Test;
Expand All @@ -56,6 +65,7 @@
import java.util.TimeZone;

import static org.apache.hadoop.fs.CommonConfigurationKeysPublic.FS_DEFAULT_NAME_DEFAULT;
import static org.apache.parquet.schema.PrimitiveType.PrimitiveTypeName.INT32;

@Slf4j
public class ParquetReadStrategyTest {
Expand Down Expand Up @@ -211,6 +221,34 @@ public void testParquetReadUnsupportedType() throws Exception {
AutoGenerateParquetData.deleteFile();
}

@DisabledOnOs(OS.WINDOWS)
@Test
public void testParquetTypeInt32WithLogicalTypeAnnotation() throws IOException {

NativeParquetWriter.generateTestData();

try (ParquetFileReader reader =
ParquetFileReader.open(
HadoopInputFile.fromPath(
new Path(NativeParquetWriter.DATA_FILE_PATH),
new Configuration()))) {

MessageType schema = reader.getFileMetaData().getSchema();
LogicalTypeAnnotation type = schema.getType("id").getLogicalTypeAnnotation();
Assertions.assertTrue(type instanceof LogicalTypeAnnotation.IntLogicalTypeAnnotation);
}

ParquetReadStrategy parquetReadStrategy = new ParquetReadStrategy();
LocalConf localConf = new LocalConf(FS_DEFAULT_NAME_DEFAULT);
parquetReadStrategy.init(localConf);
SeaTunnelRowType seaTunnelRowTypeInfo =
parquetReadStrategy.getSeaTunnelRowTypeInfo(NativeParquetWriter.DATA_FILE_PATH);
Assertions.assertNotNull(seaTunnelRowTypeInfo);
Assertions.assertEquals(seaTunnelRowTypeInfo.getFieldType(0).getTypeClass(), Integer.class);
TestCollector testCollector = new TestCollector();
parquetReadStrategy.read(NativeParquetWriter.DATA_FILE_PATH, "", testCollector);
}

public static class TestCollector implements Collector<SeaTunnelRow> {

private final List<SeaTunnelRow> rows = new ArrayList<>();
Expand Down Expand Up @@ -344,4 +382,48 @@ public static void deleteFile() {
}
}
}

/** Write data based on the Parquet native api */
public static class NativeParquetWriter {

public static final String DATA_FILE_PATH = "/tmp/data_native.parquet";

// 1. Define Parquet Native Schema (MessageType)
public static MessageType createSchema() {
return Types.buildMessage()
.required(INT32)
.as(LogicalTypeAnnotation.intType(32, true))
.named("id")
.named("User");
}

// 2. write data
public static void generateTestData() throws IOException {
deleteFile();
MessageType schema = createSchema();
Configuration conf = new Configuration();

GroupWriteSupport.setSchema(schema, conf);

Path file = new Path(DATA_FILE_PATH);
try (ParquetWriter<Group> writer =
ExampleParquetWriter.builder(file)
.withConf(conf)
.withCompressionCodec(CompressionCodecName.SNAPPY)
.build()) {

Group record1 = new SimpleGroup(schema);
record1.add("id", 1);

writer.write(record1);
}
}

private static void deleteFile() {
File parquetFile = new File(DATA_FILE_PATH);
if (parquetFile.exists()) {
parquetFile.delete();
}
}
}
}