TyrantLucifer commented on code in PR #4164: URL: https://github.com/apache/incubator-seatunnel/pull/4164#discussion_r1122541418
########## docs/en/connector-v2/sink/HdfsFile.md: ########## @@ -235,6 +246,25 @@ HdfsFile { ``` +For text file format with `sheet_name` and `max_rows_in_memory` Review Comment: ```suggestion For excel file format with `sheet_name` and `max_rows_in_memory` ``` ########## seatunnel-connectors-v2/connector-file/connector-file-base/src/main/java/org/apache/seatunnel/connectors/seatunnel/file/source/reader/ExcelReadStrategy.java: ########## @@ -0,0 +1,200 @@ +/* + * 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.seatunnel.connectors.seatunnel.file.source.reader; + +import org.apache.seatunnel.shade.com.fasterxml.jackson.databind.ObjectMapper; + +import org.apache.seatunnel.api.source.Collector; +import org.apache.seatunnel.api.table.type.SeaTunnelDataType; +import org.apache.seatunnel.api.table.type.SeaTunnelRow; +import org.apache.seatunnel.api.table.type.SeaTunnelRowType; +import org.apache.seatunnel.api.table.type.SqlType; +import org.apache.seatunnel.common.exception.CommonErrorCode; +import org.apache.seatunnel.common.utils.DateTimeUtils; +import org.apache.seatunnel.common.utils.DateUtils; +import org.apache.seatunnel.common.utils.TimeUtils; +import org.apache.seatunnel.connectors.seatunnel.file.config.BaseSourceConfig; +import org.apache.seatunnel.connectors.seatunnel.file.config.HadoopConf; +import org.apache.seatunnel.connectors.seatunnel.file.exception.FileConnectorException; + +import org.apache.hadoop.conf.Configuration; +import org.apache.hadoop.fs.FSDataInputStream; +import org.apache.hadoop.fs.FileSystem; +import org.apache.hadoop.fs.Path; +import org.apache.poi.ss.usermodel.Cell; +import org.apache.poi.ss.usermodel.CellType; +import org.apache.poi.ss.usermodel.DataFormatter; +import org.apache.poi.ss.usermodel.DateUtil; +import org.apache.poi.ss.usermodel.Row; +import org.apache.poi.ss.usermodel.Sheet; +import org.apache.poi.ss.usermodel.Workbook; +import org.apache.poi.xssf.usermodel.XSSFWorkbook; + +import lombok.SneakyThrows; + +import java.math.BigDecimal; +import java.nio.charset.StandardCharsets; +import java.time.LocalDate; +import java.time.LocalDateTime; +import java.time.LocalTime; +import java.time.format.DateTimeFormatter; +import java.util.Map; + +import static org.apache.seatunnel.common.utils.DateTimeUtils.Formatter.YYYY_MM_DD_HH_MM_SS; + +public class ExcelReadStrategy extends AbstractReadStrategy { + + private final DateUtils.Formatter dateFormat = DateUtils.Formatter.YYYY_MM_DD; + + private final DateTimeUtils.Formatter datetimeFormat = YYYY_MM_DD_HH_MM_SS; + private final TimeUtils.Formatter timeFormat = TimeUtils.Formatter.HH_MM_SS; + + @SneakyThrows + @Override + public void read(String path, Collector<SeaTunnelRow> output) { + Configuration conf = getConfiguration(); + FileSystem fs = FileSystem.get(conf); + Map<String, String> partitionsMap = parsePartitionsByPath(path); + Path filePath = new Path(path); + FSDataInputStream file = fs.open(filePath); + Workbook workbook = new XSSFWorkbook(file); + Sheet sheet = + pluginConfig.hasPath(BaseSourceConfig.SHEET_NAME.key()) + ? workbook.getSheet( + pluginConfig.getString(BaseSourceConfig.SHEET_NAME.key())) + : workbook.getSheetAt(0); + Row rowTitle = sheet.getRow(0); Review Comment: Excel can support this feature: `skip_header_row_number`, so getRow(0) is not suitable. ########## docs/en/connector-v2/sink/LocalFile.md: ########## @@ -47,6 +48,9 @@ By default, we use 2PC commit to ensure `exactly-once` | batch_size | int | no | 1000000 | | | compress_codec | string | no | none | | | common-options | object | no | - | | +| common-options | object | no | - | | Review Comment: Revert ########## docs/en/connector-v2/sink/LocalFile.md: ########## @@ -200,6 +212,24 @@ LocalFile { ``` +For text file format with `sheet_name` and `max_rows_in_memory` Review Comment: ```suggestion For excel file format with `sheet_name` and `max_rows_in_memory` ``` ########## seatunnel-connectors-v2/connector-file/connector-file-base/src/main/java/org/apache/seatunnel/connectors/seatunnel/file/source/reader/ExcelReadStrategy.java: ########## @@ -0,0 +1,200 @@ +/* + * 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.seatunnel.connectors.seatunnel.file.source.reader; + +import org.apache.seatunnel.shade.com.fasterxml.jackson.databind.ObjectMapper; + +import org.apache.seatunnel.api.source.Collector; +import org.apache.seatunnel.api.table.type.SeaTunnelDataType; +import org.apache.seatunnel.api.table.type.SeaTunnelRow; +import org.apache.seatunnel.api.table.type.SeaTunnelRowType; +import org.apache.seatunnel.api.table.type.SqlType; +import org.apache.seatunnel.common.exception.CommonErrorCode; +import org.apache.seatunnel.common.utils.DateTimeUtils; +import org.apache.seatunnel.common.utils.DateUtils; +import org.apache.seatunnel.common.utils.TimeUtils; +import org.apache.seatunnel.connectors.seatunnel.file.config.BaseSourceConfig; +import org.apache.seatunnel.connectors.seatunnel.file.config.HadoopConf; +import org.apache.seatunnel.connectors.seatunnel.file.exception.FileConnectorException; + +import org.apache.hadoop.conf.Configuration; +import org.apache.hadoop.fs.FSDataInputStream; +import org.apache.hadoop.fs.FileSystem; +import org.apache.hadoop.fs.Path; +import org.apache.poi.ss.usermodel.Cell; +import org.apache.poi.ss.usermodel.CellType; +import org.apache.poi.ss.usermodel.DataFormatter; +import org.apache.poi.ss.usermodel.DateUtil; +import org.apache.poi.ss.usermodel.Row; +import org.apache.poi.ss.usermodel.Sheet; +import org.apache.poi.ss.usermodel.Workbook; +import org.apache.poi.xssf.usermodel.XSSFWorkbook; + +import lombok.SneakyThrows; + +import java.math.BigDecimal; +import java.nio.charset.StandardCharsets; +import java.time.LocalDate; +import java.time.LocalDateTime; +import java.time.LocalTime; +import java.time.format.DateTimeFormatter; +import java.util.Map; + +import static org.apache.seatunnel.common.utils.DateTimeUtils.Formatter.YYYY_MM_DD_HH_MM_SS; + +public class ExcelReadStrategy extends AbstractReadStrategy { + + private final DateUtils.Formatter dateFormat = DateUtils.Formatter.YYYY_MM_DD; + + private final DateTimeUtils.Formatter datetimeFormat = YYYY_MM_DD_HH_MM_SS; + private final TimeUtils.Formatter timeFormat = TimeUtils.Formatter.HH_MM_SS; + + @SneakyThrows + @Override + public void read(String path, Collector<SeaTunnelRow> output) { + Configuration conf = getConfiguration(); + FileSystem fs = FileSystem.get(conf); + Map<String, String> partitionsMap = parsePartitionsByPath(path); + Path filePath = new Path(path); + FSDataInputStream file = fs.open(filePath); + Workbook workbook = new XSSFWorkbook(file); + Sheet sheet = + pluginConfig.hasPath(BaseSourceConfig.SHEET_NAME.key()) + ? workbook.getSheet( + pluginConfig.getString(BaseSourceConfig.SHEET_NAME.key())) + : workbook.getSheetAt(0); + Row rowTitle = sheet.getRow(0); + int cellCount = rowTitle.getPhysicalNumberOfCells(); Review Comment: `cellCount` should get from SeaTunnelRowType ########## seatunnel-connectors-v2/connector-file/connector-file-base/src/main/java/org/apache/seatunnel/connectors/seatunnel/file/source/reader/ExcelReadStrategy.java: ########## @@ -0,0 +1,200 @@ +/* + * 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.seatunnel.connectors.seatunnel.file.source.reader; + +import org.apache.seatunnel.shade.com.fasterxml.jackson.databind.ObjectMapper; + +import org.apache.seatunnel.api.source.Collector; +import org.apache.seatunnel.api.table.type.SeaTunnelDataType; +import org.apache.seatunnel.api.table.type.SeaTunnelRow; +import org.apache.seatunnel.api.table.type.SeaTunnelRowType; +import org.apache.seatunnel.api.table.type.SqlType; +import org.apache.seatunnel.common.exception.CommonErrorCode; +import org.apache.seatunnel.common.utils.DateTimeUtils; +import org.apache.seatunnel.common.utils.DateUtils; +import org.apache.seatunnel.common.utils.TimeUtils; +import org.apache.seatunnel.connectors.seatunnel.file.config.BaseSourceConfig; +import org.apache.seatunnel.connectors.seatunnel.file.config.HadoopConf; +import org.apache.seatunnel.connectors.seatunnel.file.exception.FileConnectorException; + +import org.apache.hadoop.conf.Configuration; +import org.apache.hadoop.fs.FSDataInputStream; +import org.apache.hadoop.fs.FileSystem; +import org.apache.hadoop.fs.Path; +import org.apache.poi.ss.usermodel.Cell; +import org.apache.poi.ss.usermodel.CellType; +import org.apache.poi.ss.usermodel.DataFormatter; +import org.apache.poi.ss.usermodel.DateUtil; +import org.apache.poi.ss.usermodel.Row; +import org.apache.poi.ss.usermodel.Sheet; +import org.apache.poi.ss.usermodel.Workbook; +import org.apache.poi.xssf.usermodel.XSSFWorkbook; + +import lombok.SneakyThrows; + +import java.math.BigDecimal; +import java.nio.charset.StandardCharsets; +import java.time.LocalDate; +import java.time.LocalDateTime; +import java.time.LocalTime; +import java.time.format.DateTimeFormatter; +import java.util.Map; + +import static org.apache.seatunnel.common.utils.DateTimeUtils.Formatter.YYYY_MM_DD_HH_MM_SS; + +public class ExcelReadStrategy extends AbstractReadStrategy { + + private final DateUtils.Formatter dateFormat = DateUtils.Formatter.YYYY_MM_DD; + + private final DateTimeUtils.Formatter datetimeFormat = YYYY_MM_DD_HH_MM_SS; + private final TimeUtils.Formatter timeFormat = TimeUtils.Formatter.HH_MM_SS; + + @SneakyThrows + @Override + public void read(String path, Collector<SeaTunnelRow> output) { + Configuration conf = getConfiguration(); + FileSystem fs = FileSystem.get(conf); + Map<String, String> partitionsMap = parsePartitionsByPath(path); + Path filePath = new Path(path); + FSDataInputStream file = fs.open(filePath); + Workbook workbook = new XSSFWorkbook(file); + Sheet sheet = + pluginConfig.hasPath(BaseSourceConfig.SHEET_NAME.key()) + ? workbook.getSheet( + pluginConfig.getString(BaseSourceConfig.SHEET_NAME.key())) + : workbook.getSheetAt(0); + Row rowTitle = sheet.getRow(0); + int cellCount = rowTitle.getPhysicalNumberOfCells(); + cellCount = partitionsMap.isEmpty() ? cellCount : cellCount + partitionsMap.size(); + SeaTunnelRow seaTunnelRow = new SeaTunnelRow(cellCount); + SeaTunnelDataType<?>[] fieldTypes = seaTunnelRowType.getFieldTypes(); + int rowCount = sheet.getPhysicalNumberOfRows(); + for (int i = 1; i < rowCount; i++) { + Row rowData = sheet.getRow(i); + if (rowData != null) { + for (int j = 0; j < cellCount; j++) { + Cell cell = rowData.getCell(j); + if (cell != null) { Review Comment: if cell == null, field should set null, BTW, should support column projection. `read_columns` can controll this feature. -- This is an automated message from the Apache Git Service. To respond to the message, please log on to GitHub and use the URL above to go to the specific comment. To unsubscribe, e-mail: commits-unsubscr...@seatunnel.apache.org For queries about this service, please contact Infrastructure at: us...@infra.apache.org