fapaul commented on a change in pull request #17598:
URL: https://github.com/apache/flink/pull/17598#discussion_r784874977
##########
File path: flink-formats/flink-csv/pom.xml
##########
@@ -77,6 +77,14 @@ under the License.
<!-- test dependencies -->
+ <dependency>
Review comment:
Nit: Why was needed?
##########
File path:
flink-formats/flink-csv/src/main/java/org/apache/flink/formats/csv/CsvBulkWriter.java
##########
@@ -0,0 +1,114 @@
+/*
+ * 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.BulkWriter;
+import org.apache.flink.core.fs.FSDataOutputStream;
+import org.apache.flink.formats.common.Converter;
+
+import
org.apache.flink.shaded.jackson2.com.fasterxml.jackson.core.JsonGenerator;
+import
org.apache.flink.shaded.jackson2.com.fasterxml.jackson.databind.ObjectWriter;
+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 javax.annotation.Nullable;
+
+import java.io.IOException;
+
+import static org.apache.flink.util.Preconditions.checkNotNull;
+
+/** A simple {@link BulkWriter} implementation based on Jackson CSV
transformations. */
+class CsvBulkWriter<T, R, C> implements BulkWriter<T> {
+
+ private final FSDataOutputStream stream;
+ private final Converter<T, R, C> converter;
+ private final C converterContext;
+ private final ObjectWriter csvWriter;
+
+ CsvBulkWriter(
+ CsvMapper mapper,
+ CsvSchema schema,
+ Converter<T, R, C> converter,
+ @Nullable C converterContext,
+ FSDataOutputStream stream) {
+ checkNotNull(mapper);
+ checkNotNull(schema);
+ checkNotNull(converter);
+ checkNotNull(stream);
Review comment:
Nit: `checkNotNull` returns the value if not null so you could do:
```java
this.converter = checkNotNull(converter)
```
##########
File path:
flink-formats/flink-csv/src/test/java/org/apache/flink/formats/csv/DataStreamCsvITCase.java
##########
@@ -0,0 +1,363 @@
+/*
+ * 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.eventtime.WatermarkStrategy;
+import org.apache.flink.api.common.restartstrategy.RestartStrategies;
+import org.apache.flink.api.common.serialization.BulkWriter;
+import org.apache.flink.connector.file.sink.FileSink;
+import org.apache.flink.connector.file.src.FileSource;
+import org.apache.flink.connector.file.src.reader.StreamFormat;
+import org.apache.flink.core.fs.Path;
+import org.apache.flink.core.testutils.AllCallbackWrapper;
+import org.apache.flink.formats.common.Converter;
+import org.apache.flink.runtime.minicluster.RpcServiceSharing;
+import org.apache.flink.runtime.testutils.MiniClusterExtension;
+import org.apache.flink.runtime.testutils.MiniClusterResourceConfiguration;
+import org.apache.flink.streaming.api.datastream.DataStream;
+import org.apache.flink.streaming.api.datastream.DataStreamUtils;
+import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment;
+import
org.apache.flink.streaming.api.functions.sink.filesystem.bucketassigners.BasePathBucketAssigner;
+import org.apache.flink.streaming.api.operators.collect.ClientAndIterator;
+import org.apache.flink.util.TestLogger;
+import org.apache.flink.util.function.FunctionWithException;
+
+import
org.apache.flink.shaded.jackson2.com.fasterxml.jackson.annotation.JsonPropertyOrder;
+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 org.apache.commons.io.FileUtils;
+import org.jetbrains.annotations.NotNull;
+import org.junit.jupiter.api.Test;
+import org.junit.jupiter.api.extension.RegisterExtension;
+import org.junit.jupiter.api.io.TempDir;
+
+import java.io.File;
+import java.io.FileOutputStream;
+import java.io.IOException;
+import java.io.OutputStream;
+import java.io.OutputStreamWriter;
+import java.io.PrintWriter;
+import java.io.Serializable;
+import java.math.BigDecimal;
+import java.nio.charset.StandardCharsets;
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.Collection;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+import java.util.Objects;
+import java.util.UUID;
+import java.util.stream.Collectors;
+
+import static org.assertj.core.api.Assertions.assertThat;
+import static org.junit.jupiter.api.Assertions.assertEquals;
+import static org.junit.jupiter.api.Assertions.assertTrue;
+
+/** MiniCluster-based integration tests CSV data format. */
+public class DataStreamCsvITCase extends TestLogger {
+
+ private static final int PARALLELISM = 4;
+
+ @TempDir File outDir;
+
+ public static final MiniClusterExtension MINI_CLUSTER_RESOURCE =
Review comment:
Does this need to be public?
##########
File path:
flink-formats/flink-csv/src/test/java/org/apache/flink/formats/csv/DataStreamCsvITCase.java
##########
@@ -0,0 +1,363 @@
+/*
+ * 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.eventtime.WatermarkStrategy;
+import org.apache.flink.api.common.restartstrategy.RestartStrategies;
+import org.apache.flink.api.common.serialization.BulkWriter;
+import org.apache.flink.connector.file.sink.FileSink;
+import org.apache.flink.connector.file.src.FileSource;
+import org.apache.flink.connector.file.src.reader.StreamFormat;
+import org.apache.flink.core.fs.Path;
+import org.apache.flink.core.testutils.AllCallbackWrapper;
+import org.apache.flink.formats.common.Converter;
+import org.apache.flink.runtime.minicluster.RpcServiceSharing;
+import org.apache.flink.runtime.testutils.MiniClusterExtension;
+import org.apache.flink.runtime.testutils.MiniClusterResourceConfiguration;
+import org.apache.flink.streaming.api.datastream.DataStream;
+import org.apache.flink.streaming.api.datastream.DataStreamUtils;
+import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment;
+import
org.apache.flink.streaming.api.functions.sink.filesystem.bucketassigners.BasePathBucketAssigner;
+import org.apache.flink.streaming.api.operators.collect.ClientAndIterator;
+import org.apache.flink.util.TestLogger;
+import org.apache.flink.util.function.FunctionWithException;
+
+import
org.apache.flink.shaded.jackson2.com.fasterxml.jackson.annotation.JsonPropertyOrder;
+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 org.apache.commons.io.FileUtils;
+import org.jetbrains.annotations.NotNull;
+import org.junit.jupiter.api.Test;
+import org.junit.jupiter.api.extension.RegisterExtension;
+import org.junit.jupiter.api.io.TempDir;
+
+import java.io.File;
+import java.io.FileOutputStream;
+import java.io.IOException;
+import java.io.OutputStream;
+import java.io.OutputStreamWriter;
+import java.io.PrintWriter;
+import java.io.Serializable;
+import java.math.BigDecimal;
+import java.nio.charset.StandardCharsets;
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.Collection;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+import java.util.Objects;
+import java.util.UUID;
+import java.util.stream.Collectors;
+
+import static org.assertj.core.api.Assertions.assertThat;
+import static org.junit.jupiter.api.Assertions.assertEquals;
+import static org.junit.jupiter.api.Assertions.assertTrue;
+
+/** MiniCluster-based integration tests CSV data format. */
+public class DataStreamCsvITCase extends TestLogger {
+
+ private static final int PARALLELISM = 4;
+
+ @TempDir File outDir;
+
+ public static final MiniClusterExtension MINI_CLUSTER_RESOURCE =
+ new MiniClusterExtension(
+ new MiniClusterResourceConfiguration.Builder()
+ .setNumberTaskManagers(1)
+ .setNumberSlotsPerTaskManager(PARALLELISM)
+ .setRpcServiceSharing(RpcServiceSharing.DEDICATED)
+ .withHaLeadershipControl()
+ .build());
+
+ @RegisterExtension
+ public static AllCallbackWrapper<MiniClusterExtension> allCallbackWrapper =
Review comment:
Public required?
##########
File path:
flink-formats/flink-csv/src/main/java/org/apache/flink/formats/csv/CsvCommons.java
##########
@@ -0,0 +1,103 @@
+/*
+ * 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.ConfigOption;
+import org.apache.flink.configuration.ReadableConfig;
+import org.apache.flink.table.api.ValidationException;
+
+import org.apache.commons.lang3.StringEscapeUtils;
+
+import java.util.HashSet;
+import java.util.Set;
+
+import static org.apache.flink.formats.csv.CsvFormatOptions.ALLOW_COMMENTS;
+import static
org.apache.flink.formats.csv.CsvFormatOptions.ARRAY_ELEMENT_DELIMITER;
+import static
org.apache.flink.formats.csv.CsvFormatOptions.DISABLE_QUOTE_CHARACTER;
+import static org.apache.flink.formats.csv.CsvFormatOptions.ESCAPE_CHARACTER;
+import static org.apache.flink.formats.csv.CsvFormatOptions.FIELD_DELIMITER;
+import static
org.apache.flink.formats.csv.CsvFormatOptions.IGNORE_PARSE_ERRORS;
+import static org.apache.flink.formats.csv.CsvFormatOptions.NULL_LITERAL;
+import static org.apache.flink.formats.csv.CsvFormatOptions.QUOTE_CHARACTER;
+
+/** A class with common CSV format constants and utility methods. */
+class CsvCommons {
+
+ public static final String IDENTIFIER = "csv";
+
+ // ------------------------------------------------------------------------
+ // Validation
+ // ------------------------------------------------------------------------
+
+ static void validateFormatOptions(ReadableConfig tableOptions) {
+ final boolean hasQuoteCharacter =
tableOptions.getOptional(QUOTE_CHARACTER).isPresent();
+ final boolean isDisabledQuoteCharacter =
tableOptions.get(DISABLE_QUOTE_CHARACTER);
+ if (isDisabledQuoteCharacter && hasQuoteCharacter) {
+ throw new ValidationException(
+ "Format cannot define a quote character and disabled quote
character at the same time.");
+ }
+ // Validate the option value must be a single char.
+ validateCharacterVal(tableOptions, FIELD_DELIMITER, true);
+ validateCharacterVal(tableOptions, ARRAY_ELEMENT_DELIMITER);
+ validateCharacterVal(tableOptions, QUOTE_CHARACTER);
+ validateCharacterVal(tableOptions, ESCAPE_CHARACTER);
+ }
+
+ /** Validates the option {@code option} value must be a Character. */
+ private static void validateCharacterVal(
+ ReadableConfig tableOptions, ConfigOption<String> option) {
+ validateCharacterVal(tableOptions, option, false);
+ }
+
+ /**
+ * Validates the option {@code option} value must be a Character.
+ *
+ * @param tableOptions the table options
+ * @param option the config option
+ * @param unescape whether to unescape the option value
+ */
+ private static void validateCharacterVal(
+ ReadableConfig tableOptions, ConfigOption<String> option, boolean
unescape) {
+ if (tableOptions.getOptional(option).isPresent()) {
Review comment:
Nit: I prefer explicit condition handling to improve readability.
```java
if (!tableOptions.getOptional(option).isPresent()) {
return;
}
...
```
##########
File path:
flink-formats/flink-csv/src/main/java/org/apache/flink/formats/csv/CsvReaderFormat.java
##########
@@ -0,0 +1,171 @@
+/*
+ * 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.annotation.PublicEvolving;
+import org.apache.flink.api.common.typeinfo.TypeInformation;
+import org.apache.flink.configuration.Configuration;
+import org.apache.flink.connector.file.src.reader.SimpleStreamFormat;
+import org.apache.flink.connector.file.src.reader.StreamFormat;
+import org.apache.flink.core.fs.FSDataInputStream;
+import org.apache.flink.formats.common.Converter;
+
+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 javax.annotation.Nullable;
+
+import java.io.IOException;
+
+import static org.apache.flink.util.Preconditions.checkNotNull;
+
+/**
+ * A {@code StreamFormat} for reading CSV files.
+ *
+ * <p>The following example shows how to create a {@code CsvReaderFormat}
where the schema for CSV
+ * parsing is automatically derived based on the fields of a POJO class.
+ *
+ * <pre>{@code
+ * CsvReaderFormat<SomePojo> csvFormat =
CsvReaderFormat.forPojo(SomePojo.class);
+ * FileSource<SomePojo> source =
+ * FileSource.forRecordStreamFormat(csvFormat,
Path.fromLocalFile(filesPath)).build();
+ * }</pre>
+ *
+ * <i> Note: you might need to add {@code @JsonPropertyOrder({field1, field2,
...})} annotation from
+ * the {@code Jackson} library to your class definition with the fields order
exactly matching those
+ * of the CSV file columns).</i>
+ *
+ * <p>If you need more fine-grained control over the CSV schema or the parsing
options, use the more
+ * low-level {@code forSchema} static factory method based on the {@code
Jackson} library utilities:
+ *
+ * <pre>{@code
+ * CsvReaderFormat<T> forSchema(CsvMapper mapper,
Review comment:
Nit: The example does not look very helpful. Maybe you can show a code
snipped in conjunction with using the format with the FileSource and it would
also be great two a second example in the same code block showing it for a POJO
class.
##########
File path:
flink-formats/flink-csv/src/test/java/org/apache/flink/formats/csv/DataStreamCsvITCase.java
##########
@@ -0,0 +1,363 @@
+/*
+ * 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.eventtime.WatermarkStrategy;
+import org.apache.flink.api.common.restartstrategy.RestartStrategies;
+import org.apache.flink.api.common.serialization.BulkWriter;
+import org.apache.flink.connector.file.sink.FileSink;
+import org.apache.flink.connector.file.src.FileSource;
+import org.apache.flink.connector.file.src.reader.StreamFormat;
+import org.apache.flink.core.fs.Path;
+import org.apache.flink.core.testutils.AllCallbackWrapper;
+import org.apache.flink.formats.common.Converter;
+import org.apache.flink.runtime.minicluster.RpcServiceSharing;
+import org.apache.flink.runtime.testutils.MiniClusterExtension;
+import org.apache.flink.runtime.testutils.MiniClusterResourceConfiguration;
+import org.apache.flink.streaming.api.datastream.DataStream;
+import org.apache.flink.streaming.api.datastream.DataStreamUtils;
+import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment;
+import
org.apache.flink.streaming.api.functions.sink.filesystem.bucketassigners.BasePathBucketAssigner;
+import org.apache.flink.streaming.api.operators.collect.ClientAndIterator;
+import org.apache.flink.util.TestLogger;
+import org.apache.flink.util.function.FunctionWithException;
+
+import
org.apache.flink.shaded.jackson2.com.fasterxml.jackson.annotation.JsonPropertyOrder;
+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 org.apache.commons.io.FileUtils;
+import org.jetbrains.annotations.NotNull;
+import org.junit.jupiter.api.Test;
+import org.junit.jupiter.api.extension.RegisterExtension;
+import org.junit.jupiter.api.io.TempDir;
+
+import java.io.File;
+import java.io.FileOutputStream;
+import java.io.IOException;
+import java.io.OutputStream;
+import java.io.OutputStreamWriter;
+import java.io.PrintWriter;
+import java.io.Serializable;
+import java.math.BigDecimal;
+import java.nio.charset.StandardCharsets;
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.Collection;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+import java.util.Objects;
+import java.util.UUID;
+import java.util.stream.Collectors;
+
+import static org.assertj.core.api.Assertions.assertThat;
+import static org.junit.jupiter.api.Assertions.assertEquals;
+import static org.junit.jupiter.api.Assertions.assertTrue;
+
+/** MiniCluster-based integration tests CSV data format. */
+public class DataStreamCsvITCase extends TestLogger {
+
+ private static final int PARALLELISM = 4;
+
+ @TempDir File outDir;
+
+ public static final MiniClusterExtension MINI_CLUSTER_RESOURCE =
+ new MiniClusterExtension(
+ new MiniClusterResourceConfiguration.Builder()
+ .setNumberTaskManagers(1)
+ .setNumberSlotsPerTaskManager(PARALLELISM)
+ .setRpcServiceSharing(RpcServiceSharing.DEDICATED)
+ .withHaLeadershipControl()
+ .build());
+
+ @RegisterExtension
+ public static AllCallbackWrapper<MiniClusterExtension> allCallbackWrapper =
+ new AllCallbackWrapper<>(MINI_CLUSTER_RESOURCE);
+
+ // ------------------------------------------------------------------------
+ // test cases
+ // ------------------------------------------------------------------------
+ @Test
+ public void testBoundedTextFileSourceWithJackson() throws Exception {
+ writeFile(outDir, "data.csv", CSV_LINES);
+
+ final CsvReaderFormat<CityPojo> csvFormat =
CsvReaderFormat.forPojo(CityPojo.class);
+ final List<CityPojo> result = initializeSourceAndReadData(outDir,
csvFormat);
+
+ assertEquals(Arrays.asList(pojos), result);
+ }
+
+ @Test
+ public void testCustomBulkWriter() throws Exception {
+
+ final StreamExecutionEnvironment env =
StreamExecutionEnvironment.getExecutionEnvironment();
+ env.setParallelism(PARALLELISM);
+ env.setRestartStrategy(RestartStrategies.fixedDelayRestart(1, 0));
+
+ // fromCollection is not bounded, using fromSequence instead
+ final List<CityPojo> pojosList = Arrays.asList(pojos); // needs to be
Serializable
+ final DataStream<Integer> sequence =
+ env.fromSequence(0, pojos.length - 1).map(Long::intValue);
+ final DataStream<CityPojo> stream =
sequence.map(pojosList::get).returns(CityPojo.class);
+
+ FileSink<CityPojo> sink =
+ FileSink.forBulkFormat(new Path(outDir.toURI()),
factoryForPojo(CityPojo.class))
+ .withBucketAssigner(new BasePathBucketAssigner<>())
+ .build();
+
+ stream.sinkTo(sink);
+ env.execute();
+
+ String[] result = getResultsFromSinkFiles(outDir);
+
+ assertThat(result).containsExactlyInAnyOrder(CSV_LINES);
Review comment:
Is the order really guaranteed if we run with higher parallelism than 1?
I would expect parallel threads writing to the file so they cannot ensure the
ordering.
##########
File path:
flink-formats/flink-csv/src/main/java/org/apache/flink/formats/csv/CsvBulkWriter.java
##########
@@ -0,0 +1,114 @@
+/*
+ * 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.BulkWriter;
+import org.apache.flink.core.fs.FSDataOutputStream;
+import org.apache.flink.formats.common.Converter;
+
+import
org.apache.flink.shaded.jackson2.com.fasterxml.jackson.core.JsonGenerator;
+import
org.apache.flink.shaded.jackson2.com.fasterxml.jackson.databind.ObjectWriter;
+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 javax.annotation.Nullable;
+
+import java.io.IOException;
+
+import static org.apache.flink.util.Preconditions.checkNotNull;
+
+/** A simple {@link BulkWriter} implementation based on Jackson CSV
transformations. */
+class CsvBulkWriter<T, R, C> implements BulkWriter<T> {
+
+ private final FSDataOutputStream stream;
+ private final Converter<T, R, C> converter;
+ private final C converterContext;
Review comment:
Nit: annotate with nullable
##########
File path:
flink-formats/flink-csv/src/test/java/org/apache/flink/formats/csv/DataStreamCsvITCase.java
##########
@@ -0,0 +1,363 @@
+/*
+ * 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.eventtime.WatermarkStrategy;
+import org.apache.flink.api.common.restartstrategy.RestartStrategies;
+import org.apache.flink.api.common.serialization.BulkWriter;
+import org.apache.flink.connector.file.sink.FileSink;
+import org.apache.flink.connector.file.src.FileSource;
+import org.apache.flink.connector.file.src.reader.StreamFormat;
+import org.apache.flink.core.fs.Path;
+import org.apache.flink.core.testutils.AllCallbackWrapper;
+import org.apache.flink.formats.common.Converter;
+import org.apache.flink.runtime.minicluster.RpcServiceSharing;
+import org.apache.flink.runtime.testutils.MiniClusterExtension;
+import org.apache.flink.runtime.testutils.MiniClusterResourceConfiguration;
+import org.apache.flink.streaming.api.datastream.DataStream;
+import org.apache.flink.streaming.api.datastream.DataStreamUtils;
+import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment;
+import
org.apache.flink.streaming.api.functions.sink.filesystem.bucketassigners.BasePathBucketAssigner;
+import org.apache.flink.streaming.api.operators.collect.ClientAndIterator;
+import org.apache.flink.util.TestLogger;
+import org.apache.flink.util.function.FunctionWithException;
+
+import
org.apache.flink.shaded.jackson2.com.fasterxml.jackson.annotation.JsonPropertyOrder;
+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 org.apache.commons.io.FileUtils;
+import org.jetbrains.annotations.NotNull;
+import org.junit.jupiter.api.Test;
+import org.junit.jupiter.api.extension.RegisterExtension;
+import org.junit.jupiter.api.io.TempDir;
+
+import java.io.File;
+import java.io.FileOutputStream;
+import java.io.IOException;
+import java.io.OutputStream;
+import java.io.OutputStreamWriter;
+import java.io.PrintWriter;
+import java.io.Serializable;
+import java.math.BigDecimal;
+import java.nio.charset.StandardCharsets;
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.Collection;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+import java.util.Objects;
+import java.util.UUID;
+import java.util.stream.Collectors;
+
+import static org.assertj.core.api.Assertions.assertThat;
+import static org.junit.jupiter.api.Assertions.assertEquals;
+import static org.junit.jupiter.api.Assertions.assertTrue;
+
+/** MiniCluster-based integration tests CSV data format. */
+public class DataStreamCsvITCase extends TestLogger {
+
+ private static final int PARALLELISM = 4;
+
+ @TempDir File outDir;
+
+ public static final MiniClusterExtension MINI_CLUSTER_RESOURCE =
+ new MiniClusterExtension(
+ new MiniClusterResourceConfiguration.Builder()
+ .setNumberTaskManagers(1)
+ .setNumberSlotsPerTaskManager(PARALLELISM)
+ .setRpcServiceSharing(RpcServiceSharing.DEDICATED)
+ .withHaLeadershipControl()
+ .build());
+
+ @RegisterExtension
+ public static AllCallbackWrapper<MiniClusterExtension> allCallbackWrapper =
+ new AllCallbackWrapper<>(MINI_CLUSTER_RESOURCE);
+
+ // ------------------------------------------------------------------------
+ // test cases
+ // ------------------------------------------------------------------------
+ @Test
+ public void testBoundedTextFileSourceWithJackson() throws Exception {
+ writeFile(outDir, "data.csv", CSV_LINES);
+
+ final CsvReaderFormat<CityPojo> csvFormat =
CsvReaderFormat.forPojo(CityPojo.class);
+ final List<CityPojo> result = initializeSourceAndReadData(outDir,
csvFormat);
+
+ assertEquals(Arrays.asList(pojos), result);
+ }
+
+ @Test
+ public void testCustomBulkWriter() throws Exception {
+
+ final StreamExecutionEnvironment env =
StreamExecutionEnvironment.getExecutionEnvironment();
+ env.setParallelism(PARALLELISM);
+ env.setRestartStrategy(RestartStrategies.fixedDelayRestart(1, 0));
+
+ // fromCollection is not bounded, using fromSequence instead
+ final List<CityPojo> pojosList = Arrays.asList(pojos); // needs to be
Serializable
+ final DataStream<Integer> sequence =
+ env.fromSequence(0, pojos.length - 1).map(Long::intValue);
+ final DataStream<CityPojo> stream =
sequence.map(pojosList::get).returns(CityPojo.class);
+
+ FileSink<CityPojo> sink =
+ FileSink.forBulkFormat(new Path(outDir.toURI()),
factoryForPojo(CityPojo.class))
+ .withBucketAssigner(new BasePathBucketAssigner<>())
+ .build();
+
+ stream.sinkTo(sink);
+ env.execute();
+
+ String[] result = getResultsFromSinkFiles(outDir);
+
+ assertThat(result).containsExactlyInAnyOrder(CSV_LINES);
+ }
+
+ @NotNull
+ private String[] getResultsFromSinkFiles(File outDir) throws IOException {
+ final Map<File, String> contents = getFileContentByPath(outDir);
+
+ List<String> resultList =
+ contents.entrySet().stream()
+ .flatMap(e -> Arrays.stream(e.getValue().split("\n")))
+ .collect(Collectors.toList());
+
+ String[] result = resultList.toArray(new String[0]);
+ return result;
+ }
+
+ private static <T> BulkWriter.Factory<T> factoryForPojo(Class<T>
pojoClass) {
+ final Converter<T, T, Void> converter = (value, context) -> value;
+ final CsvMapper csvMapper = new CsvMapper();
+ final CsvSchema schema =
csvMapper.schemaFor(pojoClass).withoutQuoteChar();
+ return (out) -> new CsvBulkWriter<>(csvMapper, schema, converter,
null, out);
+ }
+
+ private static Map<File, String> getFileContentByPath(File directory)
throws IOException {
+ Map<File, String> contents = new HashMap<>(4);
+
+ final Collection<File> filesInBucket = FileUtils.listFiles(directory,
null, true);
+ for (File file : filesInBucket) {
+ contents.put(file, FileUtils.readFileToString(file));
+ }
+ return contents;
+ }
+
+ @JsonPropertyOrder({
+ "city",
+ "lat",
+ "lng",
+ "country",
+ "iso2",
+ "adminName",
+ "capital",
+ "population"
+ })
+ public static class CityPojo implements Serializable {
+ public String city;
+ public BigDecimal lat;
+ public BigDecimal lng;
+ public String country;
+ public String iso2;
+ public String adminName;
+ public String capital;
+ public long population;
+
+ public CityPojo() {};
+
+ public CityPojo(
+ String city,
+ BigDecimal lat,
+ BigDecimal lng,
+ String country,
+ String iso2,
+ String admin_name,
+ String capital,
+ long population) {
+ this.city = city;
+ this.lat = lat;
+ this.lng = lng;
+ this.country = country;
+ this.iso2 = iso2;
+ this.adminName = admin_name;
+ this.capital = capital;
+ this.population = population;
+ }
+
+ @Override
+ public String toString() {
+ return "CitiesPojo{"
+ + "city='"
+ + city
+ + '\''
+ + ", lat="
+ + lat
+ + ", lng="
+ + lng
+ + ", country='"
+ + country
+ + '\''
+ + ", iso2='"
+ + iso2
+ + '\''
+ + ", adminName='"
+ + adminName
+ + '\''
+ + ", capital='"
+ + capital
+ + '\''
+ + ", population="
+ + population
+ + '}';
+ }
+
+ @Override
+ public boolean equals(Object o) {
+ if (this == o) {
+ return true;
+ }
+ if (o == null || getClass() != o.getClass()) {
+ return false;
+ }
+ CityPojo that = (CityPojo) o;
+ return population == that.population
+ && Objects.equals(city, that.city)
+ && Objects.equals(lat, that.lat)
+ && Objects.equals(lng, that.lng)
+ && Objects.equals(country, that.country)
+ && Objects.equals(iso2, that.iso2)
+ && Objects.equals(adminName, that.adminName)
+ && Objects.equals(capital, that.capital);
+ }
+
+ @Override
+ public int hashCode() {
+ return Objects.hash(city, lat, lng, country, iso2, adminName,
capital, population);
+ }
+ }
+
+ private static <T> List<T> initializeSourceAndReadData(File testDir,
StreamFormat<T> csvFormat)
+ throws Exception {
+ final FileSource<T> source =
+ FileSource.forRecordStreamFormat(csvFormat,
Path.fromLocalFile(testDir)).build();
+
+ final StreamExecutionEnvironment env =
StreamExecutionEnvironment.getExecutionEnvironment();
+ env.setParallelism(PARALLELISM);
+ env.setRestartStrategy(RestartStrategies.fixedDelayRestart(1, 0));
+
+ final DataStream<T> stream =
+ env.fromSource(source, WatermarkStrategy.noWatermarks(),
"file-source");
+
+ return getResultsFromStream(stream);
+ }
+
+ @NotNull
+ private static <T> List<T> getResultsFromStream(DataStream<T> stream)
throws Exception {
+ final ClientAndIterator<T> client =
+ DataStreamUtils.collectWithClient(stream, "Bounded Results
Fetch");
+
+ final List<T> result = new ArrayList<>();
+ while (client.iterator.hasNext()) {
+ T next = client.iterator.next();
+ result.add(next);
+ }
+ return result;
+ }
+
+ // ------------------------------------------------------------------------
+ // test data
+ // ------------------------------------------------------------------------
+
+ private static final String[] CSV_LINES =
+ new String[] {
+ "Berlin,52.5167,13.3833,Germany,DE,Berlin,primary,3644826",
+ "San Francisco,37.7562,-122.443,United
States,US,California,,3592294",
+ "Beijing,39.905,116.3914,China,CN,Beijing,primary,19433000"
+ };
+
+ final CityPojo[] pojos =
Review comment:
Nit: static
##########
File path:
flink-formats/flink-csv/src/test/java/org/apache/flink/formats/csv/DataStreamCsvITCase.java
##########
@@ -0,0 +1,363 @@
+/*
+ * 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.eventtime.WatermarkStrategy;
+import org.apache.flink.api.common.restartstrategy.RestartStrategies;
+import org.apache.flink.api.common.serialization.BulkWriter;
+import org.apache.flink.connector.file.sink.FileSink;
+import org.apache.flink.connector.file.src.FileSource;
+import org.apache.flink.connector.file.src.reader.StreamFormat;
+import org.apache.flink.core.fs.Path;
+import org.apache.flink.core.testutils.AllCallbackWrapper;
+import org.apache.flink.formats.common.Converter;
+import org.apache.flink.runtime.minicluster.RpcServiceSharing;
+import org.apache.flink.runtime.testutils.MiniClusterExtension;
+import org.apache.flink.runtime.testutils.MiniClusterResourceConfiguration;
+import org.apache.flink.streaming.api.datastream.DataStream;
+import org.apache.flink.streaming.api.datastream.DataStreamUtils;
+import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment;
+import
org.apache.flink.streaming.api.functions.sink.filesystem.bucketassigners.BasePathBucketAssigner;
+import org.apache.flink.streaming.api.operators.collect.ClientAndIterator;
+import org.apache.flink.util.TestLogger;
+import org.apache.flink.util.function.FunctionWithException;
+
+import
org.apache.flink.shaded.jackson2.com.fasterxml.jackson.annotation.JsonPropertyOrder;
+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 org.apache.commons.io.FileUtils;
+import org.jetbrains.annotations.NotNull;
+import org.junit.jupiter.api.Test;
+import org.junit.jupiter.api.extension.RegisterExtension;
+import org.junit.jupiter.api.io.TempDir;
+
+import java.io.File;
+import java.io.FileOutputStream;
+import java.io.IOException;
+import java.io.OutputStream;
+import java.io.OutputStreamWriter;
+import java.io.PrintWriter;
+import java.io.Serializable;
+import java.math.BigDecimal;
+import java.nio.charset.StandardCharsets;
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.Collection;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+import java.util.Objects;
+import java.util.UUID;
+import java.util.stream.Collectors;
+
+import static org.assertj.core.api.Assertions.assertThat;
+import static org.junit.jupiter.api.Assertions.assertEquals;
+import static org.junit.jupiter.api.Assertions.assertTrue;
+
+/** MiniCluster-based integration tests CSV data format. */
+public class DataStreamCsvITCase extends TestLogger {
Review comment:
You are using the `TestLogger` for junit4. Please replace it with the
`TestLoggerExtension` [1]
[1]
https://github.com/apache/flink/blob/3fe1de954bf3ee05247cdff0c43d71dc9535803d/flink-connectors/flink-connector-kafka/src/test/java/org/apache/flink/connector/kafka/sink/KafkaCommitterTest.java#L43
##########
File path:
flink-formats/flink-csv/src/test/java/org/apache/flink/formats/csv/DataStreamCsvITCase.java
##########
@@ -0,0 +1,363 @@
+/*
+ * 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.eventtime.WatermarkStrategy;
+import org.apache.flink.api.common.restartstrategy.RestartStrategies;
+import org.apache.flink.api.common.serialization.BulkWriter;
+import org.apache.flink.connector.file.sink.FileSink;
+import org.apache.flink.connector.file.src.FileSource;
+import org.apache.flink.connector.file.src.reader.StreamFormat;
+import org.apache.flink.core.fs.Path;
+import org.apache.flink.core.testutils.AllCallbackWrapper;
+import org.apache.flink.formats.common.Converter;
+import org.apache.flink.runtime.minicluster.RpcServiceSharing;
+import org.apache.flink.runtime.testutils.MiniClusterExtension;
+import org.apache.flink.runtime.testutils.MiniClusterResourceConfiguration;
+import org.apache.flink.streaming.api.datastream.DataStream;
+import org.apache.flink.streaming.api.datastream.DataStreamUtils;
+import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment;
+import
org.apache.flink.streaming.api.functions.sink.filesystem.bucketassigners.BasePathBucketAssigner;
+import org.apache.flink.streaming.api.operators.collect.ClientAndIterator;
+import org.apache.flink.util.TestLogger;
+import org.apache.flink.util.function.FunctionWithException;
+
+import
org.apache.flink.shaded.jackson2.com.fasterxml.jackson.annotation.JsonPropertyOrder;
+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 org.apache.commons.io.FileUtils;
+import org.jetbrains.annotations.NotNull;
+import org.junit.jupiter.api.Test;
+import org.junit.jupiter.api.extension.RegisterExtension;
+import org.junit.jupiter.api.io.TempDir;
+
+import java.io.File;
+import java.io.FileOutputStream;
+import java.io.IOException;
+import java.io.OutputStream;
+import java.io.OutputStreamWriter;
+import java.io.PrintWriter;
+import java.io.Serializable;
+import java.math.BigDecimal;
+import java.nio.charset.StandardCharsets;
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.Collection;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+import java.util.Objects;
+import java.util.UUID;
+import java.util.stream.Collectors;
+
+import static org.assertj.core.api.Assertions.assertThat;
+import static org.junit.jupiter.api.Assertions.assertEquals;
+import static org.junit.jupiter.api.Assertions.assertTrue;
+
+/** MiniCluster-based integration tests CSV data format. */
+public class DataStreamCsvITCase extends TestLogger {
+
+ private static final int PARALLELISM = 4;
+
+ @TempDir File outDir;
+
+ public static final MiniClusterExtension MINI_CLUSTER_RESOURCE =
+ new MiniClusterExtension(
+ new MiniClusterResourceConfiguration.Builder()
+ .setNumberTaskManagers(1)
+ .setNumberSlotsPerTaskManager(PARALLELISM)
+ .setRpcServiceSharing(RpcServiceSharing.DEDICATED)
+ .withHaLeadershipControl()
+ .build());
+
+ @RegisterExtension
+ public static AllCallbackWrapper<MiniClusterExtension> allCallbackWrapper =
+ new AllCallbackWrapper<>(MINI_CLUSTER_RESOURCE);
+
+ // ------------------------------------------------------------------------
+ // test cases
+ // ------------------------------------------------------------------------
+ @Test
+ public void testBoundedTextFileSourceWithJackson() throws Exception {
+ writeFile(outDir, "data.csv", CSV_LINES);
+
+ final CsvReaderFormat<CityPojo> csvFormat =
CsvReaderFormat.forPojo(CityPojo.class);
+ final List<CityPojo> result = initializeSourceAndReadData(outDir,
csvFormat);
+
+ assertEquals(Arrays.asList(pojos), result);
+ }
+
+ @Test
+ public void testCustomBulkWriter() throws Exception {
+
+ final StreamExecutionEnvironment env =
StreamExecutionEnvironment.getExecutionEnvironment();
+ env.setParallelism(PARALLELISM);
+ env.setRestartStrategy(RestartStrategies.fixedDelayRestart(1, 0));
Review comment:
Is this needed?
##########
File path:
flink-formats/flink-csv/src/test/java/org/apache/flink/formats/csv/TableCsvFormatITCase.java
##########
@@ -0,0 +1,204 @@
+/*
+ * 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.formats.common.TimeFormats;
+import org.apache.flink.table.planner.runtime.utils.TestData;
+import org.apache.flink.table.planner.utils.JavaScalaConversionUtil;
+import org.apache.flink.table.planner.utils.JsonPlanTestBase;
+
+import org.junit.Test;
+
+import java.io.File;
+import java.io.IOException;
+import java.nio.file.Files;
+import java.util.Arrays;
+import java.util.Collections;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+
+import static org.apache.flink.table.utils.DateTimeUtils.toLocalDateTime;
+
+/** Tests for the CSV file format. */
+public class TableCsvFormatITCase extends JsonPlanTestBase {
+
+ @Test
+ public void testProjectPushDown() throws Exception {
Review comment:
I thought projection push down is not supported. Why do you have a test
for it?
##########
File path:
flink-formats/flink-csv/src/test/java/org/apache/flink/formats/csv/DataStreamCsvITCase.java
##########
@@ -0,0 +1,363 @@
+/*
+ * 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.eventtime.WatermarkStrategy;
+import org.apache.flink.api.common.restartstrategy.RestartStrategies;
+import org.apache.flink.api.common.serialization.BulkWriter;
+import org.apache.flink.connector.file.sink.FileSink;
+import org.apache.flink.connector.file.src.FileSource;
+import org.apache.flink.connector.file.src.reader.StreamFormat;
+import org.apache.flink.core.fs.Path;
+import org.apache.flink.core.testutils.AllCallbackWrapper;
+import org.apache.flink.formats.common.Converter;
+import org.apache.flink.runtime.minicluster.RpcServiceSharing;
+import org.apache.flink.runtime.testutils.MiniClusterExtension;
+import org.apache.flink.runtime.testutils.MiniClusterResourceConfiguration;
+import org.apache.flink.streaming.api.datastream.DataStream;
+import org.apache.flink.streaming.api.datastream.DataStreamUtils;
+import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment;
+import
org.apache.flink.streaming.api.functions.sink.filesystem.bucketassigners.BasePathBucketAssigner;
+import org.apache.flink.streaming.api.operators.collect.ClientAndIterator;
+import org.apache.flink.util.TestLogger;
+import org.apache.flink.util.function.FunctionWithException;
+
+import
org.apache.flink.shaded.jackson2.com.fasterxml.jackson.annotation.JsonPropertyOrder;
+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 org.apache.commons.io.FileUtils;
+import org.jetbrains.annotations.NotNull;
+import org.junit.jupiter.api.Test;
+import org.junit.jupiter.api.extension.RegisterExtension;
+import org.junit.jupiter.api.io.TempDir;
+
+import java.io.File;
+import java.io.FileOutputStream;
+import java.io.IOException;
+import java.io.OutputStream;
+import java.io.OutputStreamWriter;
+import java.io.PrintWriter;
+import java.io.Serializable;
+import java.math.BigDecimal;
+import java.nio.charset.StandardCharsets;
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.Collection;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+import java.util.Objects;
+import java.util.UUID;
+import java.util.stream.Collectors;
+
+import static org.assertj.core.api.Assertions.assertThat;
+import static org.junit.jupiter.api.Assertions.assertEquals;
+import static org.junit.jupiter.api.Assertions.assertTrue;
+
+/** MiniCluster-based integration tests CSV data format. */
+public class DataStreamCsvITCase extends TestLogger {
+
+ private static final int PARALLELISM = 4;
+
+ @TempDir File outDir;
+
+ public static final MiniClusterExtension MINI_CLUSTER_RESOURCE =
+ new MiniClusterExtension(
+ new MiniClusterResourceConfiguration.Builder()
+ .setNumberTaskManagers(1)
+ .setNumberSlotsPerTaskManager(PARALLELISM)
+ .setRpcServiceSharing(RpcServiceSharing.DEDICATED)
+ .withHaLeadershipControl()
+ .build());
+
+ @RegisterExtension
+ public static AllCallbackWrapper<MiniClusterExtension> allCallbackWrapper =
+ new AllCallbackWrapper<>(MINI_CLUSTER_RESOURCE);
+
+ // ------------------------------------------------------------------------
+ // test cases
+ // ------------------------------------------------------------------------
+ @Test
+ public void testBoundedTextFileSourceWithJackson() throws Exception {
+ writeFile(outDir, "data.csv", CSV_LINES);
+
+ final CsvReaderFormat<CityPojo> csvFormat =
CsvReaderFormat.forPojo(CityPojo.class);
+ final List<CityPojo> result = initializeSourceAndReadData(outDir,
csvFormat);
+
+ assertEquals(Arrays.asList(pojos), result);
+ }
+
+ @Test
+ public void testCustomBulkWriter() throws Exception {
+
+ final StreamExecutionEnvironment env =
StreamExecutionEnvironment.getExecutionEnvironment();
+ env.setParallelism(PARALLELISM);
+ env.setRestartStrategy(RestartStrategies.fixedDelayRestart(1, 0));
+
+ // fromCollection is not bounded, using fromSequence instead
+ final List<CityPojo> pojosList = Arrays.asList(pojos); // needs to be
Serializable
+ final DataStream<Integer> sequence =
+ env.fromSequence(0, pojos.length - 1).map(Long::intValue);
+ final DataStream<CityPojo> stream =
sequence.map(pojosList::get).returns(CityPojo.class);
+
+ FileSink<CityPojo> sink =
+ FileSink.forBulkFormat(new Path(outDir.toURI()),
factoryForPojo(CityPojo.class))
+ .withBucketAssigner(new BasePathBucketAssigner<>())
+ .build();
+
+ stream.sinkTo(sink);
+ env.execute();
+
+ String[] result = getResultsFromSinkFiles(outDir);
+
+ assertThat(result).containsExactlyInAnyOrder(CSV_LINES);
+ }
+
+ @NotNull
+ private String[] getResultsFromSinkFiles(File outDir) throws IOException {
+ final Map<File, String> contents = getFileContentByPath(outDir);
+
+ List<String> resultList =
+ contents.entrySet().stream()
+ .flatMap(e -> Arrays.stream(e.getValue().split("\n")))
+ .collect(Collectors.toList());
+
+ String[] result = resultList.toArray(new String[0]);
+ return result;
+ }
+
+ private static <T> BulkWriter.Factory<T> factoryForPojo(Class<T>
pojoClass) {
+ final Converter<T, T, Void> converter = (value, context) -> value;
+ final CsvMapper csvMapper = new CsvMapper();
+ final CsvSchema schema =
csvMapper.schemaFor(pojoClass).withoutQuoteChar();
+ return (out) -> new CsvBulkWriter<>(csvMapper, schema, converter,
null, out);
+ }
+
+ private static Map<File, String> getFileContentByPath(File directory)
throws IOException {
+ Map<File, String> contents = new HashMap<>(4);
+
+ final Collection<File> filesInBucket = FileUtils.listFiles(directory,
null, true);
+ for (File file : filesInBucket) {
+ contents.put(file, FileUtils.readFileToString(file));
+ }
+ return contents;
+ }
+
+ @JsonPropertyOrder({
+ "city",
+ "lat",
+ "lng",
+ "country",
+ "iso2",
+ "adminName",
+ "capital",
+ "population"
+ })
+ public static class CityPojo implements Serializable {
+ public String city;
+ public BigDecimal lat;
+ public BigDecimal lng;
+ public String country;
+ public String iso2;
+ public String adminName;
+ public String capital;
+ public long population;
+
+ public CityPojo() {};
+
+ public CityPojo(
+ String city,
+ BigDecimal lat,
+ BigDecimal lng,
+ String country,
+ String iso2,
+ String admin_name,
+ String capital,
+ long population) {
+ this.city = city;
+ this.lat = lat;
+ this.lng = lng;
+ this.country = country;
+ this.iso2 = iso2;
+ this.adminName = admin_name;
+ this.capital = capital;
+ this.population = population;
+ }
+
+ @Override
+ public String toString() {
+ return "CitiesPojo{"
+ + "city='"
+ + city
+ + '\''
+ + ", lat="
+ + lat
+ + ", lng="
+ + lng
+ + ", country='"
+ + country
+ + '\''
+ + ", iso2='"
+ + iso2
+ + '\''
+ + ", adminName='"
+ + adminName
+ + '\''
+ + ", capital='"
+ + capital
+ + '\''
+ + ", population="
+ + population
+ + '}';
+ }
+
+ @Override
+ public boolean equals(Object o) {
+ if (this == o) {
+ return true;
+ }
+ if (o == null || getClass() != o.getClass()) {
+ return false;
+ }
+ CityPojo that = (CityPojo) o;
+ return population == that.population
+ && Objects.equals(city, that.city)
+ && Objects.equals(lat, that.lat)
+ && Objects.equals(lng, that.lng)
+ && Objects.equals(country, that.country)
+ && Objects.equals(iso2, that.iso2)
+ && Objects.equals(adminName, that.adminName)
+ && Objects.equals(capital, that.capital);
+ }
+
+ @Override
+ public int hashCode() {
+ return Objects.hash(city, lat, lng, country, iso2, adminName,
capital, population);
+ }
+ }
+
+ private static <T> List<T> initializeSourceAndReadData(File testDir,
StreamFormat<T> csvFormat)
+ throws Exception {
+ final FileSource<T> source =
+ FileSource.forRecordStreamFormat(csvFormat,
Path.fromLocalFile(testDir)).build();
+
+ final StreamExecutionEnvironment env =
StreamExecutionEnvironment.getExecutionEnvironment();
+ env.setParallelism(PARALLELISM);
+ env.setRestartStrategy(RestartStrategies.fixedDelayRestart(1, 0));
+
+ final DataStream<T> stream =
+ env.fromSource(source, WatermarkStrategy.noWatermarks(),
"file-source");
+
+ return getResultsFromStream(stream);
+ }
+
+ @NotNull
+ private static <T> List<T> getResultsFromStream(DataStream<T> stream)
throws Exception {
+ final ClientAndIterator<T> client =
+ DataStreamUtils.collectWithClient(stream, "Bounded Results
Fetch");
+
+ final List<T> result = new ArrayList<>();
+ while (client.iterator.hasNext()) {
+ T next = client.iterator.next();
+ result.add(next);
+ }
+ return result;
+ }
+
+ // ------------------------------------------------------------------------
+ // test data
+ // ------------------------------------------------------------------------
+
+ private static final String[] CSV_LINES =
Review comment:
Can you move all static variables to the beginning of the class? The
variable declaration should be always at the top of the file that other
developers do not have to search them first. You can still generate the test
data in a method and put it here.
--
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: [email protected]
For queries about this service, please contact Infrastructure at:
[email protected]