2323
2424import com .dtstack .flink .sql .source .AbsDeserialization ;
2525import com .dtstack .flink .sql .source .kafka .metric .KafkaTopicPartitionLagMetric ;
26+ import com .google .common .collect .Maps ;
27+ import org .apache .flink .api .common .typeinfo .BasicArrayTypeInfo ;
28+ import org .apache .flink .api .common .typeinfo .PrimitiveArrayTypeInfo ;
2629import org .apache .flink .api .common .typeinfo .TypeInformation ;
30+ import org .apache .flink .api .common .typeinfo .Types ;
31+ import org .apache .flink .api .java .typeutils .ObjectArrayTypeInfo ;
2732import org .apache .flink .api .java .typeutils .RowTypeInfo ;
2833import org .apache .flink .metrics .MetricGroup ;
34+ import org .apache .flink .shaded .jackson2 .com .fasterxml .jackson .core .JsonProcessingException ;
2935import org .apache .flink .shaded .jackson2 .com .fasterxml .jackson .databind .JsonNode ;
3036import org .apache .flink .shaded .jackson2 .com .fasterxml .jackson .databind .ObjectMapper ;
3137import org .apache .flink .streaming .connectors .kafka .internal .KafkaConsumerThread ;
3945
4046import java .io .IOException ;
4147import java .lang .reflect .Field ;
48+ import java .sql .Date ;
49+ import java .sql .Time ;
50+ import java .sql .Timestamp ;
4251import java .util .Iterator ;
52+ import java .util .Map ;
4353import java .util .Set ;
4454
4555import static com .dtstack .flink .sql .metric .MetricConstant .DT_PARTITION_GROUP ;
@@ -77,16 +87,22 @@ public class CustomerJsonDeserialization extends AbsDeserialization<Row> {
7787
7888 private boolean firstMsg = true ;
7989
80- public CustomerJsonDeserialization (TypeInformation <Row > typeInfo ){
90+ private Map <String , String > rowAndFieldMapping = Maps .newHashMap ();
91+
92+ private Map <String , JsonNode > nodeAndJsonnodeMapping = Maps .newHashMap ();
93+
94+ public CustomerJsonDeserialization (TypeInformation <Row > typeInfo ,Map <String , String > rowAndFieldMapping ){
8195 this .typeInfo = typeInfo ;
8296
8397 this .fieldNames = ((RowTypeInfo ) typeInfo ).getFieldNames ();
8498
8599 this .fieldTypes = ((RowTypeInfo ) typeInfo ).getFieldTypes ();
100+
101+ this .rowAndFieldMapping =rowAndFieldMapping ;
86102 }
87103
88104 @ Override
89- public Row deserialize (byte [] message ) throws IOException {
105+ public Row deserialize (byte [] message ) {
90106
91107 if (firstMsg ){
92108 try {
@@ -103,7 +119,10 @@ public Row deserialize(byte[] message) throws IOException {
103119 numInBytes .inc (message .length );
104120
105121 JsonNode root = objectMapper .readTree (message );
122+ parseTree (root ,"" );
123+
106124 Row row = new Row (fieldNames .length );
125+
107126 for (int i = 0 ; i < fieldNames .length ; i ++) {
108127 JsonNode node = getIgnoreCase (root , fieldNames [i ]);
109128
@@ -115,9 +134,7 @@ public Row deserialize(byte[] message) throws IOException {
115134 row .setField (i , null );
116135 }
117136 } else {
118- // Read the value as specified type
119- Object value = objectMapper .treeToValue (node , fieldTypes [i ].getTypeClass ());
120- row .setField (i , value );
137+ row .setField (i , convert (node , fieldTypes [i ]));
121138 }
122139 }
123140
@@ -131,22 +148,29 @@ public Row deserialize(byte[] message) throws IOException {
131148 }
132149 }
133150
151+ public void parseTree (JsonNode jsonNode , String prefix ){
152+ nodeAndJsonnodeMapping .clear ();
153+
154+ Iterator <String > iterator = jsonNode .fieldNames ();
155+ while (iterator .hasNext ()){
156+ String next = iterator .next ();
157+ JsonNode child = jsonNode .get (next );
158+ if (child .isObject ()){
159+ parseTree (child ,next +"." +prefix );
160+ }else {
161+ nodeAndJsonnodeMapping .put (prefix +next ,child );
162+ }
163+ }
164+ }
165+
134166 public void setFailOnMissingField (boolean failOnMissingField ) {
135167 this .failOnMissingField = failOnMissingField ;
136168 }
137169
138- public JsonNode getIgnoreCase (JsonNode jsonNode , String key ) {
139-
140- Iterator <String > iter = jsonNode .fieldNames ();
141- while (iter .hasNext ()) {
142- String key1 = iter .next ();
143- if (key1 .equalsIgnoreCase (key )) {
144- return jsonNode .get (key1 );
145- }
146- }
147-
148- return null ;
149170
171+ public JsonNode getIgnoreCase (JsonNode jsonNode , String key ) {
172+ String nodeMappingKey = rowAndFieldMapping .get (key );
173+ return nodeAndJsonnodeMapping .get (nodeMappingKey );
150174 }
151175
152176 public void setFetcher (AbstractFetcher <Row , ?> fetcher ) {
@@ -192,4 +216,65 @@ protected void registerPtMetric(AbstractFetcher<Row, ?> fetcher) throws Exceptio
192216 private static String partitionLagMetricName (TopicPartition tp ) {
193217 return tp + ".records-lag" ;
194218 }
219+
220+ // --------------------------------------------------------------------------------------------
221+
222+ private Object convert (JsonNode node , TypeInformation <?> info ) {
223+ if (info .getTypeClass ().equals (Types .BOOLEAN .getTypeClass ())) {
224+ return node .asBoolean ();
225+ } else if (info .getTypeClass ().equals (Types .STRING .getTypeClass ())) {
226+ return node .asText ();
227+ } else if (info .getTypeClass ().equals (Types .BIG_DEC .getTypeClass ())) {
228+ return node .decimalValue ();
229+ } else if (info .getTypeClass ().equals (Types .BIG_INT .getTypeClass ())) {
230+ return node .bigIntegerValue ();
231+ } else if (info .getTypeClass ().equals (Types .SQL_DATE .getTypeClass ())) {
232+ return Date .valueOf (node .asText ());
233+ } else if (info .getTypeClass ().equals (Types .SQL_TIME .getTypeClass ())) {
234+ // according to RFC 3339 every full-time must have a timezone;
235+ // until we have full timezone support, we only support UTC;
236+ // users can parse their time as string as a workaround
237+ final String time = node .asText ();
238+ if (time .indexOf ('Z' ) < 0 || time .indexOf ('.' ) >= 0 ) {
239+ throw new IllegalStateException (
240+ "Invalid time format. Only a time in UTC timezone without milliseconds is supported yet. " +
241+ "Format: HH:mm:ss'Z'" );
242+ }
243+ return Time .valueOf (time .substring (0 , time .length () - 1 ));
244+ } else if (info .getTypeClass ().equals (Types .SQL_TIMESTAMP .getTypeClass ())) {
245+ // according to RFC 3339 every date-time must have a timezone;
246+ // until we have full timezone support, we only support UTC;
247+ // users can parse their time as string as a workaround
248+ final String timestamp = node .asText ();
249+ if (timestamp .indexOf ('Z' ) < 0 ) {
250+ throw new IllegalStateException (
251+ "Invalid timestamp format. Only a timestamp in UTC timezone is supported yet. " +
252+ "Format: yyyy-MM-dd'T'HH:mm:ss.SSS'Z'" );
253+ }
254+ return Timestamp .valueOf (timestamp .substring (0 , timestamp .length () - 1 ).replace ('T' , ' ' ));
255+ } else if (info instanceof ObjectArrayTypeInfo ) {
256+ throw new IllegalStateException ("Unsupported type information '" + info + "' for node: " + node );
257+ } else if (info instanceof BasicArrayTypeInfo ) {
258+ throw new IllegalStateException ("Unsupported type information '" + info + "' for node: " + node );
259+ } else if (info instanceof PrimitiveArrayTypeInfo &&
260+ ((PrimitiveArrayTypeInfo ) info ).getComponentType () == Types .BYTE ) {
261+ return convertByteArray (node );
262+ } else {
263+ // for types that were specified without JSON schema
264+ // e.g. POJOs
265+ try {
266+ return objectMapper .treeToValue (node , info .getTypeClass ());
267+ } catch (JsonProcessingException e ) {
268+ throw new IllegalStateException ("Unsupported type information '" + info + "' for node: " + node );
269+ }
270+ }
271+ }
272+
273+ private Object convertByteArray (JsonNode node ) {
274+ try {
275+ return node .binaryValue ();
276+ } catch (IOException e ) {
277+ throw new RuntimeException ("Unable to deserialize byte array." , e );
278+ }
279+ }
195280}
0 commit comments