[ https://issues.apache.org/jira/browse/FLINK-9964?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=16578399#comment-16578399 ]
ASF GitHub Bot commented on FLINK-9964: --------------------------------------- twalthr 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_r209632669 ########## File path: flink-formats/flink-csv/src/main/java/org/apache/flink/formats/csv/CsvRowSerializationSchema.java ########## @@ -0,0 +1,234 @@ +/* + * 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.SerializationSchema; +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.core.JsonProcessingException; +import com.fasterxml.jackson.databind.JsonNode; +import com.fasterxml.jackson.databind.node.ArrayNode; +import com.fasterxml.jackson.databind.node.ContainerNode; +import com.fasterxml.jackson.databind.node.ObjectNode; +import com.fasterxml.jackson.dataformat.csv.CsvMapper; +import com.fasterxml.jackson.dataformat.csv.CsvSchema; + +import java.io.UnsupportedEncodingException; +import java.math.BigDecimal; +import java.math.BigInteger; +import java.sql.Date; +import java.sql.Time; +import java.sql.Timestamp; + +/** + * Serialization schema that serializes an object of Flink types into a CSV bytes. + * + * <p>Serializes the input row into a {@link ObjectNode} and + * converts it into <code>byte[]</code>. + * + * <p>Result <code>byte[]</code> messages can be deserialized using {@link CsvRowDeserializationSchema}. + */ +@PublicEvolving +public class CsvRowSerializationSchema implements SerializationSchema<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(); + + /** Reusable object node. */ + private ObjectNode root; + + /** Charset for byte[]. */ + private String charset = "UTF-8"; + + /** + * Create a {@link CsvRowSerializationSchema} with given {@link TypeInformation}. + * @param rowTypeInfo type information used to create schem. + */ + CsvRowSerializationSchema(TypeInformation<Row> rowTypeInfo) { + Preconditions.checkNotNull(rowTypeInfo, "rowTypeInfo must not be null !"); + this.rowTypeInfo = rowTypeInfo; + this.csvSchema = CsvRowSchemaConverter.rowTypeToCsvSchema((RowTypeInfo) rowTypeInfo); + } + + @Override + public byte[] serialize(Row row) { + if (root == null) { + root = csvMapper.createObjectNode(); + } + try { + convertRow(root, row, (RowTypeInfo) rowTypeInfo); + return csvMapper.writer(csvSchema).writeValueAsBytes(root); + } catch (JsonProcessingException e) { + throw new RuntimeException("Could not serialize row '" + row + "'. " + + "Make sure that the schema matches the input.", e); + } + } + + private void convertRow(ObjectNode reuse, Row row, RowTypeInfo rowTypeInfo) { + if (reuse == null) { + reuse = csvMapper.createObjectNode(); + } + if (row.getArity() != rowTypeInfo.getFieldNames().length) { + throw new IllegalStateException(String.format( + "Number of elements in the row '%s' is different from number of field names: %d", + row, rowTypeInfo.getFieldNames().length)); + } + TypeInformation[] types = rowTypeInfo.getFieldTypes(); + String[] fields = rowTypeInfo.getFieldNames(); + for (int i = 0; i < types.length; i++) { + String columnName = fields[i]; + Object obj = row.getField(i); + reuse.set(columnName, convert(reuse, obj, types[i], false)); + } + } + + /** + * Converts an object to a JsonNode. + * @param container {@link ContainerNode} that creates {@link JsonNode}. + * @param obj Object that used to {@link JsonNode}. + * @param info Type infomation that decides the type of {@link JsonNode}. + * @param nested variable that indicates whether the obj is in a nested structure + * like a string in an array. + * @return result after converting. + */ + private JsonNode convert(ContainerNode<?> container, Object obj, TypeInformation info, Boolean nested) { Review comment: This nested flag means that only one level of nesting is supported right? ---------------------------------------------------------------- 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)