From issues-return-184116-archive-asf-public=cust-asf.ponee.io@flink.apache.org Mon Aug 13 16:56:27 2018 Return-Path: X-Original-To: archive-asf-public@cust-asf.ponee.io Delivered-To: archive-asf-public@cust-asf.ponee.io Received: from mail.apache.org (hermes.apache.org [140.211.11.3]) by mx-eu-01.ponee.io (Postfix) with SMTP id 22A8E1807AA for ; Mon, 13 Aug 2018 16:56:25 +0200 (CEST) Received: (qmail 35848 invoked by uid 500); 13 Aug 2018 14:56:25 -0000 Mailing-List: contact issues-help@flink.apache.org; run by ezmlm Precedence: bulk List-Help: List-Unsubscribe: List-Post: List-Id: Reply-To: dev@flink.apache.org Delivered-To: mailing list issues@flink.apache.org Received: (qmail 35297 invoked by uid 99); 13 Aug 2018 14:56:24 -0000 Received: from ec2-52-202-80-70.compute-1.amazonaws.com (HELO gitbox.apache.org) (52.202.80.70) by apache.org (qpsmtpd/0.29) with ESMTP; Mon, 13 Aug 2018 14:56:24 +0000 From: GitBox To: issues@flink.apache.org Subject: [GitHub] twalthr commented on a change in pull request #6541: [FLINK-9964] [table] Add a CSV table format factory Message-ID: <153417218437.1732.2504756434588960906.gitbox@gitbox.apache.org> Date: Mon, 13 Aug 2018 14:56:24 -0000 Content-Type: text/plain; charset=utf-8 Content-Transfer-Encoding: 8bit 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_r209611971 ########## 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. + * + *

Deserializes a byte[] message as a {@link JsonNode} and + * convert it to {@link Row}. + * + *

Failure during deserialization are forwarded as wrapped IOExceptions. + */ +@PublicEvolving +public class CsvRowDeserializationSchema implements DeserializationSchema { + + /** Schema describing the input csv data. */ + private CsvSchema csvSchema; + + /** Type information describing the input csv data. */ + private TypeInformation 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 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 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(); + } else if (info == Types.BIG_DEC) { + return BigDecimal.valueOf(node.asDouble()); + } else if (info == Types.BIG_INT) { + return BigInteger.valueOf(node.asLong()); + } else if (info == Types.SQL_DATE) { + return Date.valueOf(node.asText()); + } else if (info == Types.SQL_TIME) { + return Time.valueOf(node.asText()); + } else if (info == Types.SQL_TIMESTAMP) { + return Timestamp.valueOf((String) node.asText()); + } else if (info == Types.BOOLEAN) { + return node.asBoolean(); + } else if (info instanceof RowTypeInfo) { + return convertRow((ArrayNode) node, (RowTypeInfo) info); + } else if (info instanceof BasicArrayTypeInfo) { + return convertArray((ArrayNode) node, ((BasicArrayTypeInfo) info).getComponentInfo()); + } else if (info instanceof PrimitiveArrayTypeInfo && + ((PrimitiveArrayTypeInfo) info).getComponentType() == Types.BYTE) { + return convertByteArray((TextNode) node); + } else { + throw new RuntimeException("Unable to support type " + info.toString() + " yet"); + } + } + + /** + * @param node array node used to convert array. + * @param elementType type information of array elements. + * @return result array + */ + private Object[] convertArray(ArrayNode node, TypeInformation elementType) { + final Object[] array = (Object[]) Array.newInstance(elementType.getTypeClass(), node.size()); + for (int i = 0; i < node.size(); i++) { + array[i] = convert(node.get(i), elementType); + } + return array; + } + + private byte[] convertByteArray(TextNode node) { + try { + return node.asText().getBytes(charset); + } catch (UnsupportedEncodingException e) { + throw new RuntimeException("Unsupport encoding charset" + charset, e); + } + } + + public void setCharset(String charset) { + this.charset = charset; + } + + public void setFieldDelimiter(String s) { + if (s.length() != 1) { + throw new RuntimeException("FieldDelimiter's length must be one !"); + } + this.csvSchema = this.csvSchema.rebuild().setColumnSeparator(s.charAt(0)).build(); + } + + public void setArrayElementDelimiter(String s) { + this.csvSchema = this.csvSchema.rebuild().setArrayElementSeparator(s).build(); + } + + public void setQuoteCharacter(char c) { + this.csvSchema = this.csvSchema.rebuild().setQuoteChar(c).build(); + } + + public void setEscapeCharacter(char c) { Review comment: Add the possibility to configure a null value and comments as well? ---------------------------------------------------------------- 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: users@infra.apache.org With regards, Apache Git Services