JingsongLi commented on a change in pull request #11755:
URL: https://github.com/apache/flink/pull/11755#discussion_r412665572



##########
File path: 
flink-formats/flink-csv/src/test/java/org/apache/flink/formats/csv/BaseRowCsvInputformatTest.java
##########
@@ -0,0 +1,798 @@
+/*
+ * 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.flink.formats.csv;
+
+import org.apache.flink.configuration.Configuration;
+import org.apache.flink.core.fs.FileInputSplit;
+import org.apache.flink.core.fs.Path;
+import org.apache.flink.table.api.DataTypes;
+import org.apache.flink.table.api.TableSchema;
+import org.apache.flink.table.dataformat.BaseRow;
+import org.apache.flink.table.dataformat.GenericRow;
+import org.apache.flink.table.dataformat.SqlTimestamp;
+import org.apache.flink.table.dataformat.TypeGetterSetters;
+import org.apache.flink.table.types.DataType;
+import org.apache.flink.table.types.logical.LogicalType;
+
+import org.junit.Test;
+
+import java.io.File;
+import java.io.FileOutputStream;
+import java.io.IOException;
+import java.io.OutputStreamWriter;
+import java.nio.charset.StandardCharsets;
+import java.time.LocalDate;
+import java.time.LocalDateTime;
+import java.time.LocalTime;
+import java.util.ArrayList;
+import java.util.List;
+
+import static junit.framework.TestCase.assertEquals;
+import static org.junit.Assert.assertNotNull;
+import static org.junit.Assert.assertNull;
+import static org.junit.Assert.assertTrue;
+import static org.junit.Assert.fail;
+
+/**
+ * Test suites for {@link BaseRowCsvInputformat}.
+ */
+public class BaseRowCsvInputformatTest {

Review comment:
       Can we abstract `RowCsvInputFormatTest` to reuse code?

##########
File path: 
flink-formats/flink-csv/src/test/java/org/apache/flink/formats/csv/BaseRowCsvFilesystemITCase.java
##########
@@ -0,0 +1,41 @@
+package org.apache.flink.formats.csv;
+
+import 
org.apache.flink.table.planner.runtime.batch.sql.BatchFileSystemITCaseBase;
+
+import org.junit.runner.RunWith;
+import org.junit.runners.Parameterized;
+
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.Collection;
+import java.util.List;
+
+/**
+ * Test fot {@link BaseRowCsvFileSystemFormatFactory}.
+ */
+@RunWith(Parameterized.class)
+public class BaseRowCsvFilesystemITCase extends BatchFileSystemITCaseBase {

Review comment:
       After https://github.com/apache/flink/pull/11796 , please add streaming 
test too.

##########
File path: 
flink-formats/flink-csv/src/main/java/org/apache/flink/formats/csv/BaseRowCsvInputformat.java
##########
@@ -0,0 +1,310 @@
+package org.apache.flink.formats.csv;
+
+import org.apache.flink.api.common.typeinfo.TypeInformation;
+import org.apache.flink.api.common.typeinfo.Types;
+import org.apache.flink.api.java.typeutils.RowTypeInfo;
+import org.apache.flink.core.fs.FileInputSplit;
+import org.apache.flink.core.fs.Path;
+import org.apache.flink.table.api.TableSchema;
+import org.apache.flink.table.api.ValidationException;
+import org.apache.flink.table.dataformat.BaseRow;
+import org.apache.flink.table.dataformat.BinaryString;
+import org.apache.flink.table.dataformat.DataFormatConverters;
+import org.apache.flink.table.dataformat.GenericRow;
+import org.apache.flink.table.filesystem.PartitionPathUtils;
+import org.apache.flink.table.types.DataType;
+import org.apache.flink.table.types.utils.TypeConversions;
+import org.apache.flink.types.Row;
+import org.apache.flink.util.Preconditions;
+
+import 
org.apache.flink.shaded.jackson2.com.fasterxml.jackson.databind.JsonNode;
+import 
org.apache.flink.shaded.jackson2.com.fasterxml.jackson.databind.MappingIterator;
+import 
org.apache.flink.shaded.jackson2.com.fasterxml.jackson.dataformat.csv.CsvMapper;
+import 
org.apache.flink.shaded.jackson2.com.fasterxml.jackson.dataformat.csv.CsvSchema;
+
+import java.io.IOException;
+import java.util.Arrays;
+import java.util.LinkedHashMap;
+import java.util.List;
+import java.util.NoSuchElementException;
+import java.util.stream.Collectors;
+
+import static 
org.apache.flink.formats.csv.CsvRowDeserializationSchema.createFieldRuntimeConverters;
+import static 
org.apache.flink.formats.csv.CsvRowDeserializationSchema.validateArity;
+import static org.apache.flink.util.Preconditions.checkNotNull;
+
+/**
+ * Input format that reads csv file into {@link BaseRow}.
+ */
+public class BaseRowCsvInputformat extends AbstractCsvInputFormat<BaseRow> {
+
+       private static final long serialVersionUID = 1L;
+
+       private final List<TypeInformation> fieldTypes;
+       private final List<String> fieldNames;
+       private final int[] selectFields;
+       private final List<String> partitionKeys;
+       private final String defaultPartValue;
+       private final boolean ignoreParseErrors;
+       private final long limit;
+       private final List<String> csvFieldNames;
+       private final List<TypeInformation> csvFieldTypes;
+       private final List<String> csvSelectFieldNames;
+       private final List<TypeInformation> csvSelectTypes;
+       private final int[] csvFieldMapping;
+
+       private transient CsvRowDeserializationSchema.RuntimeConverter 
runtimeConverter;
+       private transient List<DataFormatConverters.DataFormatConverter> 
csvSelectConverters;
+       private transient MappingIterator<JsonNode> iterator;
+       private transient boolean end;
+       private transient long emitted;
+       // reuse object for per record
+       private transient GenericRow rowData;
+
+       private BaseRowCsvInputformat(
+               Path[] filePaths,
+               CsvSchema csvSchema,
+               RowTypeInfo rowType,
+               int[] selectFields,
+               List<String> partitionKeys,
+               String defaultPartValue,
+               boolean ignoreParseErrors,
+               long limit) {
+                       super(filePaths, csvSchema);
+                       this.partitionKeys = partitionKeys;
+                       this.defaultPartValue = defaultPartValue;
+                       this.selectFields = selectFields;
+                       this.ignoreParseErrors = ignoreParseErrors;
+                       this.limit = limit;
+                       this.fieldTypes = 
Arrays.asList(rowType.getFieldTypes());
+                       this.fieldNames = 
Arrays.asList(rowType.getFieldNames());
+                       this.emitted = 0;
+                       // partition field
+                       this.csvFieldNames = fieldNames.stream()
+                               .filter(name -> 
!partitionKeys.contains(name)).collect(Collectors.toList());
+                       this.csvFieldTypes = csvFieldNames.stream()
+                       .map(name -> 
fieldTypes.get(fieldNames.indexOf(name))).collect(Collectors.toList());
+                       // project field
+                       List<String> selectFieldNames = 
Arrays.stream(selectFields)
+                               .mapToObj(fieldNames::get)
+                               .collect(Collectors.toList());
+                       this.csvSelectFieldNames = selectFieldNames.stream()
+                               .filter(name -> 
!partitionKeys.contains(name)).collect(Collectors.toList());
+                       this.csvSelectTypes = csvSelectFieldNames.stream()
+                               .map(name -> 
fieldTypes.get(fieldNames.indexOf(name))).collect(Collectors.toList());
+                       this.csvFieldMapping = 
csvSelectFieldNames.stream().mapToInt(selectFieldNames::indexOf).toArray();
+       }
+
+       @Override
+       public void open(FileInputSplit split) throws IOException {
+               super.open(split);
+               this.end = false;
+               this.iterator = new CsvMapper()
+                       .readerFor(JsonNode.class)
+                       .with(csvSchema)
+                       .readValues(csvInputStream);
+               prepareRuntimeConverter();
+               fillPartitionValueForRecord();
+       }
+
+       private void fillPartitionValueForRecord() {
+               rowData = new GenericRow(selectFields.length);
+               Path path = currentSplit.getPath();
+               LinkedHashMap<String, String> partSpec = 
PartitionPathUtils.extractPartitionSpecFromPath(path);
+               for (int i = 0; i < selectFields.length; i++) {
+                       int selectField = selectFields[i];
+                       String name = fieldNames.get(selectField);
+                       if (partitionKeys.contains(name)) {
+                               String value = partSpec.get(name);
+                               value = defaultPartValue.equals(value) ? null : 
value;
+                               rowData.setField(i, 
convertStringToInternal(value, fieldTypes.get(selectField)));
+                       }
+               }
+       }
+
+       private Object convertStringToInternal(String value, TypeInformation 
type) {

Review comment:
       use `RowPartitionComputer.restorePartValueFromType` instead

##########
File path: 
flink-formats/flink-csv/src/test/java/org/apache/flink/formats/csv/BaseRowCsvDeSerializationTest.java
##########
@@ -0,0 +1,306 @@
+/*
+ * 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.flink.formats.csv;
+
+import org.apache.flink.api.common.serialization.Encoder;
+import org.apache.flink.configuration.Configuration;
+import org.apache.flink.core.fs.FileInputSplit;
+import org.apache.flink.core.fs.Path;
+import org.apache.flink.table.api.DataTypes;
+import org.apache.flink.table.api.TableSchema;
+import org.apache.flink.table.dataformat.BaseRow;
+import org.apache.flink.table.dataformat.DataFormatConverters;
+import org.apache.flink.table.dataformat.GenericRow;
+import org.apache.flink.table.types.DataType;
+import org.apache.flink.types.Row;
+import org.apache.flink.util.FileUtils;
+import org.apache.flink.util.TestLogger;
+
+import org.junit.Assert;
+import org.junit.ClassRule;
+import org.junit.Test;
+import org.junit.rules.TemporaryFolder;
+
+import java.io.File;
+import java.io.FileOutputStream;
+import java.io.IOException;
+import java.io.OutputStreamWriter;
+import java.math.BigDecimal;
+import java.nio.charset.StandardCharsets;
+import java.sql.Date;
+import java.sql.Time;
+import java.sql.Timestamp;
+import java.time.LocalDate;
+import java.time.LocalDateTime;
+import java.time.LocalTime;
+import java.util.ArrayList;
+import java.util.function.Consumer;
+
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertNull;
+import static org.junit.Assert.fail;
+
+/**
+ * Test for {@link BaseRowCsvInputformat} and {@link BaseRowCsvEncoder}.
+ */
+public class BaseRowCsvDeSerializationTest extends TestLogger {

Review comment:
       This class is for single row test? I think we don't need test single 
row, should be include in `CsvRowDeSerializationSchemaTest`.
   We should add multi-rows test, or just add tests in 
`RowCsvFilesystemITCase.java `




----------------------------------------------------------------
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.

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


Reply via email to