You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@seatunnel.apache.org by "TyrantLucifer (via GitHub)" <gi...@apache.org> on 2023/04/06 06:37:08 UTC

[GitHub] [incubator-seatunnel] TyrantLucifer commented on a diff in pull request #4164: [Feature][ConnectorV2]add file excel sink and source

TyrantLucifer commented on code in PR #4164:
URL: https://github.com/apache/incubator-seatunnel/pull/4164#discussion_r1159337050


##########
seatunnel-connectors-v2/connector-file/connector-file-base/src/main/java/org/apache/seatunnel/connectors/seatunnel/file/source/reader/ExcelReadStrategy.java:
##########
@@ -0,0 +1,254 @@
+/*
+ * 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.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 java.util.Objects;
+import java.util.stream.IntStream;
+
+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;
+
+    private int[] indexes;
+
+    private int cellCount;
+
+    @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);
+        cellCount = seaTunnelRowType.getTotalFields();
+        cellCount = partitionsMap.isEmpty() ? cellCount : cellCount + partitionsMap.size();
+        SeaTunnelRow seaTunnelRow = new SeaTunnelRow(cellCount);
+        SeaTunnelDataType<?>[] fieldTypes = seaTunnelRowType.getFieldTypes();
+        int rowCount = sheet.getPhysicalNumberOfRows();
+        if (skipHeaderNumber > Integer.MAX_VALUE
+                || skipHeaderNumber < Integer.MIN_VALUE
+                || skipHeaderNumber > rowCount) {
+            throw new FileConnectorException(
+                    CommonErrorCode.UNSUPPORTED_OPERATION,
+                    "Skip the number of rows exceeds the maximum or minimum limit of Sheet");
+        }
+        IntStream.range((int) skipHeaderNumber, rowCount)
+                .mapToObj(sheet::getRow)
+                .filter(Objects::nonNull)
+                .forEach(
+                        rowData -> {
+                            int[] cellIndexes =
+                                    indexes == null
+                                            ? IntStream.range(0, cellCount).toArray()
+                                            : indexes;
+                            int z = 0;
+                            for (int j : cellIndexes) {
+                                Cell cell = rowData.getCell(j);
+                                seaTunnelRow.setField(
+                                        z++,
+                                        cell == null
+                                                ? null
+                                                : convert(
+                                                        getCellValue(cell.getCellType(), cell),
+                                                        fieldTypes[z - 1]));
+                            }
+                            if (isMergePartition) {
+                                int index = seaTunnelRowType.getTotalFields();
+                                for (String value : partitionsMap.values()) {
+                                    seaTunnelRow.setField(index++, value);
+                                }
+                            }
+                            output.collect(seaTunnelRow);
+                        });
+    }
+
+    @Override
+    public void setSeaTunnelRowTypeInfo(SeaTunnelRowType seaTunnelRowType) {
+        if (isNullOrEmpty(seaTunnelRowType.getFieldNames())
+                || isNullOrEmpty(seaTunnelRowType.getFieldTypes())) {
+            throw new FileConnectorException(
+                    CommonErrorCode.UNSUPPORTED_OPERATION,
+                    "Schmea information is not set or incorrect schmea settings");
+        }
+        SeaTunnelRowType userDefinedRowTypeWithPartition =
+                mergePartitionTypes(fileNames.get(0), seaTunnelRowType);
+        // column projection
+        if (pluginConfig.hasPath(BaseSourceConfig.READ_COLUMNS.key())) {
+            // get the read column index from user-defined row type
+            indexes = new int[readColumns.size()];
+            String[] fields = new String[readColumns.size()];
+            SeaTunnelDataType<?>[] types = new SeaTunnelDataType[readColumns.size()];
+            for (int i = 0; i < indexes.length; i++) {
+                indexes[i] = seaTunnelRowType.indexOf(readColumns.get(i));
+                fields[i] = seaTunnelRowType.getFieldName(indexes[i]);
+                types[i] = seaTunnelRowType.getFieldType(indexes[i]);
+            }
+            this.seaTunnelRowType = new SeaTunnelRowType(fields, types);
+            this.seaTunnelRowTypeWithPartition =
+                    mergePartitionTypes(fileNames.get(0), this.seaTunnelRowType);
+        } else {
+            this.seaTunnelRowType = seaTunnelRowType;
+            this.seaTunnelRowTypeWithPartition = userDefinedRowTypeWithPartition;
+        }
+    }
+
+    Configuration getConfiguration() {
+        return getConfiguration(hadoopConf);
+    }
+
+    @Override
+    public SeaTunnelRowType getSeaTunnelRowTypeInfo(HadoopConf hadoopConf, String path)
+            throws FileConnectorException {
+        throw new FileConnectorException(
+                CommonErrorCode.UNSUPPORTED_OPERATION,
+                "User must defined schema for json file type");
+    }
+
+    private Object getCellValue(CellType cellType, Cell cell) {
+        switch (cellType) {
+            case STRING:
+                return cell.getStringCellValue();
+            case BOOLEAN:
+                return cell.getBooleanCellValue();
+            case NUMERIC:
+                if (DateUtil.isCellDateFormatted(cell)) {
+                    DataFormatter formatter = new DataFormatter();
+                    return formatter.formatCellValue(cell);
+                }
+                return cell.getNumericCellValue();
+            case ERROR:
+                break;
+            default:
+                throw new FileConnectorException(
+                        CommonErrorCode.UNSUPPORTED_DATA_TYPE,
+                        String.format("[%s] type not support ", cellType));
+        }
+        return null;
+    }
+
+    @SneakyThrows
+    private Object convert(Object field, SeaTunnelDataType<?> fieldType) {
+        ObjectMapper objectMapper = new ObjectMapper();

Review Comment:
   Use static parameters



##########
seatunnel-connectors-v2/connector-file/connector-file-ftp/src/main/java/org/apache/seatunnel/connectors/seatunnel/file/ftp/sink/FtpFileSinkFactory.java:
##########
@@ -57,6 +57,7 @@ public OptionRule optionRule() {
                         BaseSinkConfig.FILE_FORMAT_TYPE,
                         FileFormat.JSON,
                         BaseSinkConfig.TXT_COMPRESS)
+                .conditional(BaseSinkConfig.FILE_FORMAT_TYPE, FileFormat.EXCEL)

Review Comment:
   Why add this?



##########
seatunnel-connectors-v2/connector-file/connector-file-oss-jindo/src/main/java/org/apache/seatunnel/connectors/seatunnel/file/oss/sink/OssFileSinkFactory.java:
##########
@@ -57,6 +57,7 @@ public OptionRule optionRule() {
                         BaseSinkConfig.FILE_FORMAT_TYPE,
                         FileFormat.JSON,
                         BaseSinkConfig.TXT_COMPRESS)
+                .conditional(BaseSinkConfig.FILE_FORMAT_TYPE, FileFormat.EXCEL)

Review Comment:
   The same as above



##########
seatunnel-connectors-v2/connector-file/connector-file-local/src/main/java/org/apache/seatunnel/connectors/seatunnel/file/local/sink/LocalFileSinkFactory.java:
##########
@@ -52,6 +52,7 @@ public OptionRule optionRule() {
                         BaseSinkConfig.FILE_FORMAT_TYPE,
                         FileFormat.JSON,
                         BaseSinkConfig.TXT_COMPRESS)
+                .conditional(BaseSinkConfig.FILE_FORMAT_TYPE, FileFormat.EXCEL)

Review Comment:
   The same as above



##########
seatunnel-connectors-v2/connector-file/connector-file-sftp/src/main/java/org/apache/seatunnel/connectors/seatunnel/file/sftp/sink/SftpFileSinkFactory.java:
##########
@@ -57,6 +57,7 @@ public OptionRule optionRule() {
                         BaseSinkConfig.FILE_FORMAT_TYPE,
                         FileFormat.JSON,
                         BaseSinkConfig.TXT_COMPRESS)
+                .conditional(BaseSinkConfig.FILE_FORMAT_TYPE, FileFormat.EXCEL)

Review Comment:
   The same as above



##########
seatunnel-connectors-v2/connector-file/connector-file-s3/src/main/java/org/apache/seatunnel/connectors/seatunnel/file/s3/sink/S3FileSinkFactory.java:
##########
@@ -62,6 +62,7 @@ public OptionRule optionRule() {
                         BaseSinkConfig.FILE_FORMAT_TYPE,
                         FileFormat.JSON,
                         BaseSinkConfig.TXT_COMPRESS)
+                .conditional(BaseSinkConfig.FILE_FORMAT_TYPE, FileFormat.EXCEL)

Review Comment:
   The same as above



##########
seatunnel-connectors-v2/connector-file/connector-file-oss/src/main/java/org/apache/seatunnel/connectors/seatunnel/file/oss/sink/OssFileSinkFactory.java:
##########
@@ -57,6 +57,7 @@ public OptionRule optionRule() {
                         BaseSinkConfig.FILE_FORMAT_TYPE,
                         FileFormat.JSON,
                         BaseSinkConfig.TXT_COMPRESS)
+                .conditional(BaseSinkConfig.FILE_FORMAT_TYPE, FileFormat.EXCEL)

Review Comment:
   The same as above



##########
seatunnel-connectors-v2/connector-file/connector-file-hadoop/src/main/java/org/apache/seatunnel/connectors/seatunnel/file/hdfs/sink/HdfsFileSinkFactory.java:
##########
@@ -54,6 +54,7 @@ public OptionRule optionRule() {
                         BaseSinkConfig.FILE_FORMAT_TYPE,
                         FileFormat.JSON,
                         BaseSinkConfig.TXT_COMPRESS)
+                .conditional(BaseSinkConfig.FILE_FORMAT_TYPE, FileFormat.EXCEL)

Review Comment:
   The same as above



##########
seatunnel-connectors-v2/connector-file/connector-file-local/src/main/java/org/apache/seatunnel/connectors/seatunnel/file/local/sink/LocalFileSinkFactory.java:
##########
@@ -52,6 +52,7 @@ public OptionRule optionRule() {
                         BaseSinkConfig.FILE_FORMAT_TYPE,
                         FileFormat.JSON,
                         BaseSinkConfig.TXT_COMPRESS)
+                .conditional(BaseSinkConfig.FILE_FORMAT_TYPE, FileFormat.EXCEL)

Review Comment:
   The same as above



##########
seatunnel-connectors-v2/connector-file/connector-file-ftp/src/main/java/org/apache/seatunnel/connectors/seatunnel/file/ftp/sink/FtpFileSinkFactory.java:
##########
@@ -57,6 +57,7 @@ public OptionRule optionRule() {
                         BaseSinkConfig.FILE_FORMAT_TYPE,
                         FileFormat.JSON,
                         BaseSinkConfig.TXT_COMPRESS)
+                .conditional(BaseSinkConfig.FILE_FORMAT_TYPE, FileFormat.EXCEL)

Review Comment:
   ![image](https://user-images.githubusercontent.com/51053924/230291279-c4d8ce2d-f88a-4424-8829-1515db301f2b.png)
   
   you need update it.
   



-- 
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-unsubscribe@seatunnel.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org