slinkydeveloper commented on a change in pull request #17598:
URL: https://github.com/apache/flink/pull/17598#discussion_r768823822



##########
File path: 
flink-formats/flink-csv/src/main/java/org/apache/flink/formats/csv/CsvSerDeSchemaFactory.java
##########
@@ -0,0 +1,192 @@
+/*
+ * 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.Internal;
+import org.apache.flink.api.common.serialization.DeserializationSchema;
+import org.apache.flink.api.common.serialization.SerializationSchema;
+import org.apache.flink.api.common.typeinfo.TypeInformation;
+import org.apache.flink.configuration.ConfigOption;
+import org.apache.flink.configuration.ReadableConfig;
+import org.apache.flink.table.connector.ChangelogMode;
+import org.apache.flink.table.connector.format.DecodingFormat;
+import org.apache.flink.table.connector.format.EncodingFormat;
+import org.apache.flink.table.connector.sink.DynamicTableSink;
+import org.apache.flink.table.connector.source.DynamicTableSource;
+import org.apache.flink.table.data.RowData;
+import org.apache.flink.table.factories.DeserializationFormatFactory;
+import org.apache.flink.table.factories.DynamicTableFactory;
+import org.apache.flink.table.factories.FactoryUtil;
+import org.apache.flink.table.factories.SerializationFormatFactory;
+import org.apache.flink.table.types.DataType;
+import org.apache.flink.table.types.logical.RowType;
+
+import org.apache.commons.lang3.StringEscapeUtils;
+
+import java.util.Collections;
+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;
+
+/**
+ * Format factory for providing configured instances of CSV to RowData {@link 
SerializationSchema}
+ * and {@link DeserializationSchema}.
+ */
+@Internal
+public final class CsvSerDeSchemaFactory
+        implements DeserializationFormatFactory, SerializationFormatFactory {
+
+    @Override
+    public DecodingFormat<DeserializationSchema<RowData>> createDecodingFormat(
+            DynamicTableFactory.Context context, ReadableConfig formatOptions) 
{
+        FactoryUtil.validateFactoryOptions(this, formatOptions);
+        CsvCommons.validateFormatOptions(formatOptions);
+
+        return new DecodingFormat<DeserializationSchema<RowData>>() {
+            @Override
+            public DeserializationSchema<RowData> createRuntimeDecoder(

Review comment:
       I wonder if this one can be a projectable decoding format as well, given 
it roughly uses the same code for deserializing.

##########
File path: 
flink-formats/flink-csv/src/test/java/org/apache/flink/formats/csv/CsvFormatITCase.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.unixTimestampToLocalDateTime;
+
+/** Tests for the CSV file format. */
+public class CsvFormatITCase extends JsonPlanTestBase {
+
+    @Test
+    public void testProjectPushDown() throws Exception {
+        List<String> data = Arrays.asList("1,1,hi", "2,1,hello", "3,2,hello 
world");
+        createSourceTable("MyTable", data, "a bigint", "b int not null", "c 
varchar");
+        File sinkPath = createSinkTable("MySink", "a bigint", "c varchar");
+
+        String jsonPlan = tableEnv.getJsonPlan("insert into MySink select a, c 
from MyTable");
+        tableEnv.executeJsonPlan(jsonPlan).await();
+
+        assertResult(Arrays.asList("1,hi", "2,hello", "3,hello world"), 
sinkPath);
+    }
+
+    @Test
+    public void testReadingMetadata() throws Exception {
+        createTestValuesSourceTable(
+                "MyTable",
+                JavaScalaConversionUtil.toJava(TestData.smallData3()),
+                new String[] {"a int", "b bigint", "m varchar metadata"},
+                new HashMap<String, String>() {
+                    {
+                        put("readable-metadata", "m:STRING");
+                    }
+                });
+
+        File sinkPath = createSinkTable("MySink", "a bigint", "m varchar");
+
+        String jsonPlan = tableEnv.getJsonPlan("insert into MySink select a, m 
from MyTable");
+        tableEnv.executeJsonPlan(jsonPlan).await();
+
+        assertResult(Arrays.asList("1,Hi", "2,Hello", "3,Hello world"), 
sinkPath);
+    }
+
+    @Test
+    public void testFilterPushDown() throws Exception {
+        List<String> data = Arrays.asList("1,1,hi", "2,1,hello", "3,2,hello 
world");
+        createSourceTable("MyTable", data, "a bigint", "b int not null", "c 
varchar");
+        File sinkPath = createSinkTable("MySink", "a bigint", "b int", "c 
varchar");
+
+        String jsonPlan =
+                tableEnv.getJsonPlan("insert into MySink select * from MyTable 
where a > 1");
+        tableEnv.executeJsonPlan(jsonPlan).await();
+
+        assertResult(Arrays.asList("2,1,hello", "3,2,hello world"), sinkPath);
+    }
+
+    @Test
+    public void testPartitionPushDown() throws Exception {
+        createTestValuesSourceTable(
+                "MyTable",
+                JavaScalaConversionUtil.toJava(TestData.smallData3()),
+                new String[] {"a int", "p bigint", "c varchar"},
+                "p",
+                new HashMap<String, String>() {
+                    {
+                        put("partition-list", "p:1;p:2");
+                    }
+                });
+        File sinkPath = createSinkTable("MySink", "a int", "p bigint", "c 
varchar");
+
+        String jsonPlan =
+                tableEnv.getJsonPlan("insert into MySink select * from MyTable 
where p = 2");
+        tableEnv.executeJsonPlan(jsonPlan).await();
+
+        assertResult(Arrays.asList("2,2,Hello", "3,2,Hello world"), sinkPath);
+    }
+
+    @Test
+    public void testWatermarkPushDown() throws Exception {
+        createTestValuesSourceTable(
+                "MyTable",
+                JavaScalaConversionUtil.toJava(TestData.data3WithTimestamp()),
+                new String[] {
+                    "a int",
+                    "b bigint",
+                    "c varchar",
+                    "ts timestamp(3)",
+                    "watermark for ts as ts - interval '5' second"
+                },
+                new HashMap<String, String>() {
+                    {
+                        put("enable-watermark-push-down", "true");
+                    }
+                });
+
+        File sinkPath = createSinkTable("MySink", "a int", "b bigint", "ts 
timestamp(3)");
+
+        String jsonPlan =
+                tableEnv.getJsonPlan("insert into MySink select a, b, ts from 
MyTable where b = 3");
+        tableEnv.executeJsonPlan(jsonPlan).await();
+
+        assertResult(
+                Arrays.asList(
+                        "4,3," + formatSqlTimestamp(4000L),
+                        "5,3," + formatSqlTimestamp(5000L),
+                        "6,3," + formatSqlTimestamp(6000L)),
+                sinkPath);
+    }
+
+    @Test
+    public void testPushDowns() throws Exception {
+        createTestValuesSourceTable(
+                "MyTable",
+                JavaScalaConversionUtil.toJava(TestData.data3WithTimestamp()),
+                new String[] {
+                    "a int",
+                    "b bigint",
+                    "c varchar",
+                    "ts timestamp(3)",
+                    "watermark for ts as ts - interval '5' second"
+                },
+                "b",
+                new HashMap<String, String>() {
+                    {
+                        put("readable-metadata", "a:INT");
+                        put("filterable-fields", "a");
+                        put("enable-watermark-push-down", "true");
+                        put("partition-list", "b:1;b:2;b:3;b:4;b:5;b:6");
+                    }
+                });
+
+        File sinkPath = createSinkTable("MySink", "a int", "ts timestamp(3)");
+
+        String jsonPlan =
+                tableEnv.getJsonPlan(
+                        "insert into MySink select a, ts from MyTable where b 
= 3 and a > 4");
+        tableEnv.executeJsonPlan(jsonPlan).await();
+
+        assertResult(
+                Arrays.asList("5," + formatSqlTimestamp(5000L), "6," + 
formatSqlTimestamp(6000L)),
+                sinkPath);
+    }
+
+    private static String formatSqlTimestamp(long timestamp) {
+        return 
TimeFormats.SQL_TIMESTAMP_FORMAT.format(unixTimestampToLocalDateTime(timestamp));
+    }

Review comment:
       Use `formatUnixTimestamp(long unixtime, TimeZone tz)` instead, as it 
also makes sure you use a fixed timezone and you don't get weird time zone 
issues when running on other machines :)

##########
File path: 
flink-table/flink-table-planner/src/test/java/org/apache/flink/table/planner/utils/JsonPlanTestBase.java
##########
@@ -92,17 +92,27 @@ protected void createTestValuesSourceTable(
             @Nullable String partitionFields,
             Map<String, String> extraProperties) {
         checkArgument(fieldNameAndTypes.length > 0);
-        String partitionedBy =
-                StringUtils.isNullOrWhitespaceOnly(partitionFields)
-                        ? ""
-                        : "\n partitioned by (" + partitionFields + ") \n";
+
         String dataId = TestValuesTableFactory.registerData(data);
         Map<String, String> properties = new HashMap<>();
         properties.put("connector", "values");
         properties.put("data-id", dataId);
         properties.put("bounded", "true");
         properties.put("disable-lookup", "true");
         properties.putAll(extraProperties);
+        createTestSourceTable(tableName, fieldNameAndTypes, partitionFields, 
properties);
+    }
+
+    protected void createTestSourceTable(
+            String tableName,
+            String[] fieldNameAndTypes,
+            @Nullable String partitionFields,
+            Map<String, String> properties) {
+        checkArgument(fieldNameAndTypes.length > 0);
+        String partitionedBy =
+                StringUtils.isNullOrWhitespaceOnly(partitionFields)
+                        ? ""
+                        : "\n partitioned by (" + partitionFields + ") \n";

Review comment:
       please remove this function usage and rewrite it using `Stream`s instead

##########
File path: 
flink-formats/flink-csv/src/main/java/org/apache/flink/formats/csv/CsvFormatFactory.java
##########
@@ -125,128 +70,88 @@ public String factoryIdentifier() {
 
     @Override
     public Set<ConfigOption<?>> optionalOptions() {
-        Set<ConfigOption<?>> options = new HashSet<>();
-        options.add(FIELD_DELIMITER);
-        options.add(DISABLE_QUOTE_CHARACTER);
-        options.add(QUOTE_CHARACTER);
-        options.add(ALLOW_COMMENTS);
-        options.add(IGNORE_PARSE_ERRORS);
-        options.add(ARRAY_ELEMENT_DELIMITER);
-        options.add(ESCAPE_CHARACTER);
-        options.add(NULL_LITERAL);
-        return options;
-    }
-
-    // ------------------------------------------------------------------------
-    //  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);
+        return CsvCommons.optionalOptions();
     }
 
-    /** Validates the option {@code option} value must be a Character. */
-    private static void validateCharacterVal(
-            ReadableConfig tableOptions, ConfigOption<String> option) {
-        validateCharacterVal(tableOptions, option, false);
-    }
+    @Override
+    public BulkDecodingFormat<RowData> createDecodingFormat(
+            DynamicTableFactory.Context context, ReadableConfig formatOptions) 
{
 
-    /**
-     * 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()) {
-            final String value =
-                    unescape
-                            ? 
StringEscapeUtils.unescapeJava(tableOptions.get(option))
-                            : tableOptions.get(option);
-            if (value.length() != 1) {
-                throw new ValidationException(
-                        String.format(
-                                "Option '%s.%s' must be a string with single 
character, but was: %s",
-                                IDENTIFIER, option.key(), 
tableOptions.get(option)));
-            }
-        }
+        return new CsvBulkDecodingFormat(formatOptions);
     }
 
-    // ------------------------------------------------------------------------
-    //  Utilities
-    // ------------------------------------------------------------------------
+    private static class CsvBulkDecodingFormat
+            implements BulkDecodingFormat<RowData>,
+                    ProjectableDecodingFormat<BulkFormat<RowData, 
FileSourceSplit>> {
 
-    private static void configureDeserializationSchema(
-            ReadableConfig formatOptions, 
CsvRowDataDeserializationSchema.Builder schemaBuilder) {
-        formatOptions
-                .getOptional(FIELD_DELIMITER)
-                .map(delimiter -> 
StringEscapeUtils.unescapeJava(delimiter).charAt(0))
-                .ifPresent(schemaBuilder::setFieldDelimiter);
+        private final ReadableConfig formatOptions;
 
-        if (formatOptions.get(DISABLE_QUOTE_CHARACTER)) {
-            schemaBuilder.disableQuoteCharacter();
-        } else {
-            formatOptions
-                    .getOptional(QUOTE_CHARACTER)
-                    .map(quote -> quote.charAt(0))
-                    .ifPresent(schemaBuilder::setQuoteCharacter);
+        public CsvBulkDecodingFormat(ReadableConfig formatOptions) {
+            this.formatOptions = formatOptions;
         }
 
-        
formatOptions.getOptional(ALLOW_COMMENTS).ifPresent(schemaBuilder::setAllowComments);
-
-        formatOptions
-                .getOptional(IGNORE_PARSE_ERRORS)
-                .ifPresent(schemaBuilder::setIgnoreParseErrors);
-
-        formatOptions
-                .getOptional(ARRAY_ELEMENT_DELIMITER)
-                .ifPresent(schemaBuilder::setArrayElementDelimiter);
-
-        formatOptions
-                .getOptional(ESCAPE_CHARACTER)
-                .map(escape -> escape.charAt(0))
-                .ifPresent(schemaBuilder::setEscapeCharacter);
+        @Override
+        // TODO: is it possible to avoid the cast with a reasonable effort?
+        @SuppressWarnings({"unchecked", "rawtypes"})
+        public BulkFormat<RowData, FileSourceSplit> createRuntimeDecoder(
+                Context context, DataType physicalDataType, int[][] 
projections) {
+
+            final DataType projectedDataDype = 
Projection.of(projections).project(physicalDataType);
+            final RowType rowTypeProjected = (RowType) 
projectedDataDype.getLogicalType();
+
+            final RowType rowType = (RowType) 
physicalDataType.getLogicalType();

Review comment:
       Name it `physicalRowType` so it makes the distinction with the above 
variable more clear




-- 
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: issues-unsubscr...@flink.apache.org

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


Reply via email to