[
https://issues.apache.org/jira/browse/FLINK-8630?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=16365450#comment-16365450
]
ASF GitHub Bot commented on FLINK-8630:
---------------------------------------
Github user xccui commented on a diff in the pull request:
https://github.com/apache/flink/pull/5491#discussion_r168449398
--- Diff:
flink-formats/flink-json/src/main/java/org/apache/flink/formats/json/JsonRowSerializationSchema.java
---
@@ -0,0 +1,208 @@
+/*
+ * 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.json;
+
+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.ObjectArrayTypeInfo;
+import org.apache.flink.api.java.typeutils.RowTypeInfo;
+import org.apache.flink.types.Row;
+import org.apache.flink.util.Preconditions;
+
+import
org.apache.flink.shaded.jackson2.com.fasterxml.jackson.databind.JsonNode;
+import
org.apache.flink.shaded.jackson2.com.fasterxml.jackson.databind.ObjectMapper;
+import
org.apache.flink.shaded.jackson2.com.fasterxml.jackson.databind.node.ArrayNode;
+import
org.apache.flink.shaded.jackson2.com.fasterxml.jackson.databind.node.ContainerNode;
+import
org.apache.flink.shaded.jackson2.com.fasterxml.jackson.databind.node.ObjectNode;
+
+import java.math.BigDecimal;
+import java.math.BigInteger;
+import java.sql.Time;
+import java.sql.Timestamp;
+import java.text.SimpleDateFormat;
+
+/**
+ * Serialization schema that serializes an object of Flink types into a
JSON bytes.
+ *
+ * <p>Serializes the input Flink object into a JSON string and
+ * converts it into <code>byte[]</code>.
+ *
+ * <p>Result <code>byte[]</code> messages can be deserialized using {@link
JsonRowDeserializationSchema}.
+ */
+@PublicEvolving
+public class JsonRowSerializationSchema implements
SerializationSchema<Row> {
+
+ private static final long serialVersionUID = -2885556750743978636L;
+
+ /** Type information describing the input type. */
+ private final TypeInformation<Row> typeInfo;
+
+ /** Object mapper that is used to create output JSON objects. */
+ private final ObjectMapper mapper = new ObjectMapper();
+
+ /** Formatter for RFC 3339-compliant string representation of a time
value (with UTC timezone, without milliseconds). */
+ private SimpleDateFormat timeFormat = new
SimpleDateFormat("HH:mm:ss'Z'");
+
+ /** Formatter for RFC 3339-compliant string representation of a time
value (with UTC timezone). */
+ private SimpleDateFormat timeFormatWithMillis = new
SimpleDateFormat("HH:mm:ss.SSS'Z'");
+
+ /** Formatter for RFC 3339-compliant string representation of a
timestamp value (with UTC timezone). */
+ private SimpleDateFormat timestampFormat = new
SimpleDateFormat("yyyy-MM-dd'T'HH:mm:ss.SSS'Z'");
+
+ /** Reusable object node. */
+ private transient ObjectNode node;
+
+ /**
+ * Creates a JSON serialization schema for the given type information.
+ *
+ * @param typeInfo The field names of {@link Row} are used to map to
JSON properties.
+ */
+ public JsonRowSerializationSchema(TypeInformation<Row> typeInfo) {
+ Preconditions.checkNotNull(typeInfo, "Type information");
+ this.typeInfo = typeInfo;
+ }
+
+ /**
+ * Creates a JSON serialization schema for the given JSON schema.
+ *
+ * @param jsonSchema JSON schema describing the result type
+ *
+ * @see <a href="http://json-schema.org/">http://json-schema.org/</a>
+ */
+ public JsonRowSerializationSchema(String jsonSchema) {
+ this(JsonSchemaConverter.convert(jsonSchema));
+ }
+
+ @Override
+ public byte[] serialize(Row row) {
--- End diff --
The `serialize()` method is also not thread-safe since it invokes the
method such as `SimpleDateFromat.format()`. Not sure if it matters.
> To support JSON schema to TypeInformation conversion
> -----------------------------------------------------
>
> Key: FLINK-8630
> URL: https://issues.apache.org/jira/browse/FLINK-8630
> Project: Flink
> Issue Type: Sub-task
> Components: Streaming Connectors
> Reporter: Xingcan Cui
> Assignee: Timo Walther
> Priority: Blocker
> Fix For: 1.5.0
>
>
> To support (FLINK-8558), we need to generate a \{{TypeInformation}} from a
> standard [JSON schema|http://json-schema.org/] (and maybe vice verse). There
> are some problems to be discussed, e.g., how to handle JSON {{Number}} type.
> One option would be we always return a specific type, which can be configured
> to be double or BigDecimal, for it.
--
This message was sent by Atlassian JIRA
(v7.6.3#76005)