robobario commented on code in PR #24730: URL: https://github.com/apache/flink/pull/24730#discussion_r1618053167
########## flink-formats/flink-csv/src/test/resources/log4j2-test.properties: ########## @@ -0,0 +1,28 @@ +################################################################################ +# 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. +################################################################################ + +# Set root logger level to OFF to not flood build logs +# set manually to INFO for debugging purposes +rootLogger.level = INFO Review Comment: looking at other modules and the comment above I think this should be checked in set to `OFF` ########## flink-formats/flink-csv/src/test/java/org/apache/flink/formats/csv/CsvBulkWriterIT.java: ########## @@ -0,0 +1,205 @@ +/* + * 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.api.common.typeinfo.TypeInformation; +import org.apache.flink.connector.datagen.source.DataGeneratorSource; +import org.apache.flink.connector.datagen.source.TestDataGenerators; +import org.apache.flink.connector.file.sink.FileSink; +import org.apache.flink.core.fs.FSDataOutputStream; +import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment; + +import org.apache.commons.io.FileUtils; +import org.jetbrains.annotations.NotNull; +import org.junit.jupiter.api.Test; +import org.junit.jupiter.api.io.TempDir; + +import java.io.File; +import java.io.IOException; +import java.util.Arrays; +import java.util.Collection; +import java.util.HashMap; +import java.util.List; +import java.util.Map; +import java.util.stream.Collectors; + +import static org.assertj.core.api.Assertions.assertThat; + +public class CsvBulkWriterIT { + + @TempDir File outDir; + + @Test + public void testNoDataIsWrittenBeforeFlush() throws Exception { + + StreamExecutionEnvironment env = StreamExecutionEnvironment.getExecutionEnvironment(); + env.setParallelism(1); + env.enableCheckpointing(100); + env.setRestartStrategy(RestartStrategies.noRestart()); + + // Workaround serialization limitations + File outDirRef = new File(outDir.getAbsolutePath()); + + FileSink<Pojo> sink = + FileSink.forBulkFormat( + new org.apache.flink.core.fs.Path(outDir.getAbsolutePath()), + out -> { + FSDataOutputStreamWrapper outputStreamWrapper = + new FSDataOutputStreamWrapper(out); + return new CsvBulkWriterWrapper<>( + CsvBulkWriter.forPojo(Pojo.class, outputStreamWrapper), + outputStreamWrapper, + outDirRef); + }) + .build(); + + List<Pojo> integers = Arrays.asList(new Pojo(1), new Pojo(2)); + DataGeneratorSource<Pojo> generatorSource = + TestDataGenerators.fromDataWithSnapshotsLatch( + integers, TypeInformation.of(Pojo.class)); + env.fromSource(generatorSource, WatermarkStrategy.noWatermarks(), "").sinkTo(sink); + env.execute(); + assertThat(getResultsFromSinkFiles(outDir)).containsSequence("1", "2", "1", "2"); + } + + private static class CsvBulkWriterWrapper<T> implements BulkWriter<T> { + + private static int callCounter = 0; + + private static int unFlushedCounter = 0; Review Comment: nitpick: I wonder if it would make the logic slightly easier to read if you change this from counting the unflushed elements, to tracking the expected number of flushed elements. Eg: ```java private static int addedElements = 0; private static int expectedFlushedElements = 0; @Override public void addElement(T element) throws IOException { addedElements++; csvBulkWriter.addElement(element); assertThat(getResultsFromSinkFiles(outDir)).hasSize(expectedFlushedElements); } @Override public void flush() throws IOException { csvBulkWriter.flush(); expectedFlushedElements = addedElements; assertThat(getResultsFromSinkFiles(outDir)).hasSize(expectedFlushedElements); } ``` ########## flink-formats/flink-csv/src/test/java/org/apache/flink/formats/csv/CsvBulkWriterIT.java: ########## @@ -0,0 +1,205 @@ +/* + * 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.api.common.typeinfo.TypeInformation; +import org.apache.flink.connector.datagen.source.DataGeneratorSource; +import org.apache.flink.connector.datagen.source.TestDataGenerators; +import org.apache.flink.connector.file.sink.FileSink; +import org.apache.flink.core.fs.FSDataOutputStream; +import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment; + +import org.apache.commons.io.FileUtils; +import org.jetbrains.annotations.NotNull; +import org.junit.jupiter.api.Test; +import org.junit.jupiter.api.io.TempDir; + +import java.io.File; +import java.io.IOException; +import java.util.Arrays; +import java.util.Collection; +import java.util.HashMap; +import java.util.List; +import java.util.Map; +import java.util.stream.Collectors; + +import static org.assertj.core.api.Assertions.assertThat; + +public class CsvBulkWriterIT { + + @TempDir File outDir; + + @Test + public void testNoDataIsWrittenBeforeFlush() throws Exception { + + StreamExecutionEnvironment env = StreamExecutionEnvironment.getExecutionEnvironment(); + env.setParallelism(1); + env.enableCheckpointing(100); + env.setRestartStrategy(RestartStrategies.noRestart()); Review Comment: setRestartStrategy appears to be deprecated, suggest changing to: ``` Configuration config = new Configuration(); config.set(RestartStrategyOptions.RESTART_STRATEGY, RestartStrategyOptions.RestartStrategyType.NO_RESTART_STRATEGY.getMainValue()); StreamExecutionEnvironment env = StreamExecutionEnvironment.getExecutionEnvironment(config); ``` -- 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]
