[
https://issues.apache.org/jira/browse/FLINK-10280?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel
]
sean.miao updated FLINK-10280:
------------------------------
Description:
h3. *problem*
data like :
{"key":"\\{\"key\":1}
"}
can be parsed correctly;
but data like :
{"key":\{"key":1}}
cannot be parsed correctly.
h3. *code position:*
JsonRowDeserializationSchema#convert
else {
// for types that were specified without JSON schema
// e.g. POJOs
try
{ return objectMapper.treeToValue(node, info.getTypeClass()); }
catch (JsonProcessingException e)
{ throw new IllegalStateException("Unsupported type information '" + info + "'
for node: " + node); }
}
h3. *error msg:*
java.lang.IllegalStateException: Unsupported type information 'String' for
node: \{"title":"aaa","test":"ttt"}
at
org.apache.flink.formats.json.JsonRowDeserializationSchema.convert(JsonRowDeserializationSchema.java:130)
at
org.apache.flink.formats.json.JsonRowDeserializationSchema.convertRow(JsonRowDeserializationSchema.java:183)
at
org.apache.flink.formats.json.JsonRowDeserializationSchema.deserialize(JsonRowDeserializationSchema.java:72)
at
org.apache.flink.formats.json.JsonRowDeserializationSchema.deserialize(JsonRowDeserializationSchema.java:44)
at
org.apache.flink.streaming.util.serialization.KeyedDeserializationSchemaWrapper.deserialize(KeyedDeserializationSchemaWrapper.java:44)
at
org.apache.flink.streaming.connectors.kafka.internal.Kafka09Fetcher.runFetchLoop(Kafka09Fetcher.java:142)
at
org.apache.flink.streaming.connectors.kafka.FlinkKafkaConsumerBase.run(FlinkKafkaConsumerBase.java:738)
at
org.apache.flink.streaming.api.operators.StreamSource.run(StreamSource.java:87)
at
org.apache.flink.streaming.api.operators.StreamSource.run(StreamSource.java:56)
at
org.apache.flink.streaming.runtime.tasks.SourceStreamTask.run(SourceStreamTask.java:99)
at
org.apache.flink.streaming.runtime.tasks.StreamTask.invoke(StreamTask.java:300)
at org.apache.flink.runtime.taskmanager.Task.run(Task.java:711)
at java.lang.Thread.run(Thread.java:748)
h3. *My solution is as follows:*
else {
// for types that were specified without JSON schema
// e.g. POJOs
try
{ return objectMapper.treeToValue(node, info.getTypeClass()); }
catch (JsonProcessingException e)
{ return node.toString }
}
was:
data like :
{"key":"\{\"key\":1}"}
can be parsed correctly;
but data like :
{"key":\{"key":1}}
cannot be parsed correctly.
code position:
JsonRowDeserializationSchema#convert
else {
// for types that were specified without JSON schema
// e.g. POJOs
try {
return objectMapper.treeToValue(node, info.getTypeClass());
} catch (JsonProcessingException e) {
throw new IllegalStateException("Unsupported type information '" + info + "'
for node: " + node);
}
}
My solution is as follows:
else {
// for types that were specified without JSON schema
// e.g. POJOs
try {
return objectMapper.treeToValue(node, info.getTypeClass());
} catch (JsonProcessingException e) {
return node.toString
}
}
> json that contains an object can not be parsed
> ----------------------------------------------
>
> Key: FLINK-10280
> URL: https://issues.apache.org/jira/browse/FLINK-10280
> Project: Flink
> Issue Type: Bug
> Components: Batch Connectors and Input/Output Formats
> Affects Versions: 1.6.0
> Reporter: sean.miao
> Priority: Major
>
> h3. *problem*
> data like :
> {"key":"\\{\"key\":1}
> "}
> can be parsed correctly;
>
> but data like :
> {"key":\{"key":1}}
> cannot be parsed correctly.
>
> h3. *code position:*
> JsonRowDeserializationSchema#convert
> else {
> // for types that were specified without JSON schema
> // e.g. POJOs
> try
> { return objectMapper.treeToValue(node, info.getTypeClass()); }
> catch (JsonProcessingException e)
> { throw new IllegalStateException("Unsupported type information '" + info +
> "' for node: " + node); }
> }
> h3. *error msg:*
> java.lang.IllegalStateException: Unsupported type information 'String' for
> node: \{"title":"aaa","test":"ttt"}
> at
> org.apache.flink.formats.json.JsonRowDeserializationSchema.convert(JsonRowDeserializationSchema.java:130)
> at
> org.apache.flink.formats.json.JsonRowDeserializationSchema.convertRow(JsonRowDeserializationSchema.java:183)
> at
> org.apache.flink.formats.json.JsonRowDeserializationSchema.deserialize(JsonRowDeserializationSchema.java:72)
> at
> org.apache.flink.formats.json.JsonRowDeserializationSchema.deserialize(JsonRowDeserializationSchema.java:44)
> at
> org.apache.flink.streaming.util.serialization.KeyedDeserializationSchemaWrapper.deserialize(KeyedDeserializationSchemaWrapper.java:44)
> at
> org.apache.flink.streaming.connectors.kafka.internal.Kafka09Fetcher.runFetchLoop(Kafka09Fetcher.java:142)
> at
> org.apache.flink.streaming.connectors.kafka.FlinkKafkaConsumerBase.run(FlinkKafkaConsumerBase.java:738)
> at
> org.apache.flink.streaming.api.operators.StreamSource.run(StreamSource.java:87)
> at
> org.apache.flink.streaming.api.operators.StreamSource.run(StreamSource.java:56)
> at
> org.apache.flink.streaming.runtime.tasks.SourceStreamTask.run(SourceStreamTask.java:99)
> at
> org.apache.flink.streaming.runtime.tasks.StreamTask.invoke(StreamTask.java:300)
> at org.apache.flink.runtime.taskmanager.Task.run(Task.java:711)
> at java.lang.Thread.run(Thread.java:748)
>
> h3. *My solution is as follows:*
> else {
> // for types that were specified without JSON schema
> // e.g. POJOs
> try
> { return objectMapper.treeToValue(node, info.getTypeClass()); }
> catch (JsonProcessingException e)
> { return node.toString }
> }
--
This message was sent by Atlassian JIRA
(v7.6.3#76005)