[ https://issues.apache.org/jira/browse/FLINK-9964?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=16580940#comment-16580940 ]
ASF GitHub Bot commented on FLINK-9964: --------------------------------------- buptljy commented on a change in pull request #6541: [FLINK-9964] [table] Add a CSV table format factory URL: https://github.com/apache/flink/pull/6541#discussion_r210239352 ########## File path: flink-formats/flink-csv/src/main/java/org/apache/flink/formats/csv/CsvRowDeserializationSchema.java ########## @@ -0,0 +1,228 @@ +/* + * 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.serialization.DeserializationSchema; +import org.apache.flink.api.common.typeinfo.BasicArrayTypeInfo; +import org.apache.flink.api.common.typeinfo.PrimitiveArrayTypeInfo; +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.types.Row; +import org.apache.flink.util.Preconditions; + +import com.fasterxml.jackson.databind.JsonNode; +import com.fasterxml.jackson.databind.node.ArrayNode; +import com.fasterxml.jackson.databind.node.TextNode; +import com.fasterxml.jackson.dataformat.csv.CsvMapper; +import com.fasterxml.jackson.dataformat.csv.CsvSchema; + +import java.io.IOException; +import java.io.UnsupportedEncodingException; +import java.lang.reflect.Array; +import java.math.BigDecimal; +import java.math.BigInteger; +import java.sql.Date; +import java.sql.Time; +import java.sql.Timestamp; + +/** + * Deserialization schema from CSV to Flink types. + * + * <p>Deserializes a <code>byte[]</code> message as a {@link JsonNode} and + * convert it to {@link Row}. + * + * <p>Failure during deserialization are forwarded as wrapped IOExceptions. + */ +@PublicEvolving +public class CsvRowDeserializationSchema implements DeserializationSchema<Row> { + + /** Schema describing the input csv data. */ + private CsvSchema csvSchema; + + /** Type information describing the input csv data. */ + private TypeInformation<Row> rowTypeInfo; + + /** CsvMapper used to write {@link JsonNode} into bytes. */ + private CsvMapper csvMapper = new CsvMapper(); + + /** Charset for byte[]. */ + private String charset = "UTF-8"; + + + /** + * Create a csv row DeserializationSchema with given {@link TypeInformation}. + */ + CsvRowDeserializationSchema(TypeInformation<Row> rowTypeInfo) { + Preconditions.checkNotNull(rowTypeInfo, "rowTypeInfo must not be null !"); + this.rowTypeInfo = rowTypeInfo; + this.csvSchema = CsvRowSchemaConverter.rowTypeToCsvSchema((RowTypeInfo) rowTypeInfo); + } + + @Override + public Row deserialize(byte[] message) throws IOException { + JsonNode root = csvMapper.readerFor(JsonNode.class) + .with(csvSchema).readValue(message); + return convertRow(root, (RowTypeInfo) rowTypeInfo); + } + + @Override + public boolean isEndOfStream(Row nextElement) { + return false; + } + + @Override + public TypeInformation<Row> getProducedType() { + return rowTypeInfo; + } + + /** + * + * @param root json node that contains a row's data. + * @param rowTypeInfo type information for root. + * @return result row + */ + private Row convertRow(JsonNode root, RowTypeInfo rowTypeInfo) { + String[] fields = rowTypeInfo.getFieldNames(); + TypeInformation<?>[] types = rowTypeInfo.getFieldTypes(); + Row row = new Row(fields.length); + + for (int i = 0; i < fields.length; i++) { + String columnName = fields[i]; + JsonNode node = root.get(columnName); + row.setField(i, convert(node, types[i])); + } + return row; + } + + /** + * + * @param node array node that contains a row's data. + * @param rowTypeInfo type information for node. + * @return result row + */ + private Row convertRow(ArrayNode node, RowTypeInfo rowTypeInfo) { + TypeInformation[] types = rowTypeInfo.getFieldTypes(); + String[] fields = rowTypeInfo.getFieldNames(); + Row row = new Row(fields.length); + for (int i = 0; i < fields.length; i++) { + row.setField(i, convert(node.get(i), types[i])); + } + return row; + } + + /** + * Converts json node to object with given type information. + * @param node json node to be converted. + * @param info type information for the json data. + * @return converted object + */ + private Object convert(JsonNode node, TypeInformation<?> info) { + if (info == Types.STRING) { + return node.asText(); + } else if (info == Types.LONG) { + return node.asLong(); + } else if (info == Types.INT) { + return node.asInt(); + } else if (info == Types.DOUBLE) { + return node.asDouble(); + } else if (info == Types.FLOAT) { + return Double.valueOf(node.asDouble()).floatValue(); Review comment: No, after we call ``` JsonNode root = csvMapper.readerFor(JsonNode.class).with(csvSchema).readValue(message) ``` there are only two kinds of JsonNodes left: TextNode and ArrayNode(because jackson cannot recognize the types from byte array.), and the textNode.floatValue() will return 0.0f. ---------------------------------------------------------------- This is an automated message from the Apache Git Service. To respond to the message, please log on 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 > Add a CSV table format factory > ------------------------------ > > Key: FLINK-9964 > URL: https://issues.apache.org/jira/browse/FLINK-9964 > Project: Flink > Issue Type: Sub-task > Components: Table API & SQL > Reporter: Timo Walther > Assignee: buptljy > Priority: Major > Labels: pull-request-available > > We should add a RFC 4180 compliant CSV table format factory to read and write > data into Kafka and other connectors. This requires a > {{SerializationSchemaFactory}} and {{DeserializationSchemaFactory}}. How we > want to represent all data types and nested types is still up for discussion. > For example, we could flatten and deflatten nested types as it is done > [here|http://support.gnip.com/articles/json2csv.html]. We can also have a > look how tools such as the Avro to CSV tool perform the conversion. -- This message was sent by Atlassian JIRA (v7.6.3#76005)