5
5
package org .opensearch .dataprepper .plugins .codec .parquet ;
6
6
7
7
import java .math .BigDecimal ;
8
+ import java .math .BigInteger ;
8
9
import java .nio .ByteBuffer ;
9
10
import java .nio .charset .StandardCharsets ;
10
11
import java .util .Collection ;
15
16
import java .util .function .Function ;
16
17
17
18
import org .apache .avro .LogicalType ;
19
+ import org .apache .avro .LogicalTypes ;
18
20
import org .apache .avro .Schema ;
19
21
import org .apache .avro .generic .GenericContainer ;
20
22
import org .apache .avro .generic .GenericData ;
@@ -50,7 +52,7 @@ public Function<Object, Object> getLogicalTypeConverter(LogicalType logicalType)
50
52
51
53
public String serialize (GenericRecord value ) {
52
54
StringBuilder buffer = new StringBuilder ();
53
- serialize (value , buffer , new IdentityHashMap <>(128 ) );
55
+ serialize (value , buffer , new IdentityHashMap <>(128 ), null );
54
56
String result = buffer .toString ();
55
57
return result ;
56
58
}
@@ -59,7 +61,9 @@ public String serialize(GenericRecord value) {
59
61
" \" >>> CIRCULAR REFERENCE CANNOT BE PUT IN JSON STRING, ABORTING RECURSION <<<\" " ;
60
62
61
63
/** Renders a Java datum as <a href="http://www.json.org/">JSON</a>. */
62
- private void serialize (final Object datum , final StringBuilder buffer , final IdentityHashMap <Object , Object > seenObjects ) {
64
+ private void serialize (final Object datum , final StringBuilder buffer ,
65
+ final IdentityHashMap <Object , Object > seenObjects ,
66
+ final Integer decimalScale ) {
63
67
if (isRecord (datum )) {
64
68
if (seenObjects .containsKey (datum )) {
65
69
buffer .append (TOSTRING_CIRCULAR_REFERENCE_ERROR_TEXT );
@@ -70,10 +74,32 @@ private void serialize(final Object datum, final StringBuilder buffer, final Ide
70
74
int count = 0 ;
71
75
Schema schema = getRecordSchema (datum );
72
76
for (Schema .Field f : schema .getFields ()) {
73
- serialize (f .name (), buffer , seenObjects );
77
+ serialize (f .name (), buffer , seenObjects , null ); // field name
74
78
buffer .append (": " );
79
+
75
80
Function <Object , Object > logicalTypeConverter = getLogicalTypeConverter (f );
76
- serialize (logicalTypeConverter .apply (getField (datum , f .name (), f .pos ())), buffer , seenObjects );
81
+
82
+ boolean serializedDecimal = false ;
83
+ Schema fieldSchema = f .schema ();
84
+ if (fieldSchema .getType () == Schema .Type .UNION ) {
85
+ for (Schema s : fieldSchema .getTypes ()) {
86
+ if (s .getType () != Schema .Type .NULL ) {
87
+ if (s .getType () == Schema .Type .BYTES &&
88
+ s .getLogicalType () instanceof LogicalTypes .Decimal ) {
89
+ serialize (logicalTypeConverter .apply (getField (datum , f .name (), f .pos ())), buffer , seenObjects , ((LogicalTypes .Decimal ) s .getLogicalType ()).getScale ());
90
+ serializedDecimal = true ;
91
+ break ;
92
+ }
93
+ }
94
+ }
95
+ } else if (fieldSchema .getLogicalType () instanceof LogicalTypes .Decimal ) {
96
+ serialize (logicalTypeConverter .apply (getField (datum , f .name (), f .pos ())), buffer , seenObjects , ((LogicalTypes .Decimal ) fieldSchema .getLogicalType ()).getScale ());
97
+ serializedDecimal = true ;
98
+ }
99
+
100
+ if (!serializedDecimal ) {
101
+ serialize (logicalTypeConverter .apply (getField (datum , f .name (), f .pos ())), buffer , seenObjects , null );
102
+ }
77
103
if (++count < schema .getFields ().size ())
78
104
buffer .append (", " );
79
105
}
@@ -90,7 +116,7 @@ private void serialize(final Object datum, final StringBuilder buffer, final Ide
90
116
long last = array .size ()-1 ;
91
117
int i = 0 ;
92
118
for (Object element : array ) {
93
- serialize (element , buffer , seenObjects );
119
+ serialize (element , buffer , seenObjects , null );
94
120
if (i ++ < last )
95
121
buffer .append (", " );
96
122
}
@@ -107,9 +133,9 @@ private void serialize(final Object datum, final StringBuilder buffer, final Ide
107
133
@ SuppressWarnings (value ="unchecked" )
108
134
Map <Object ,Object > map = (Map <Object ,Object >)datum ;
109
135
for (Map .Entry <Object ,Object > entry : map .entrySet ()) {
110
- serialize (entry .getKey (), buffer , seenObjects );
136
+ serialize (entry .getKey (), buffer , seenObjects , null );
111
137
buffer .append (": " );
112
- serialize (entry .getValue (), buffer , seenObjects );
138
+ serialize (entry .getValue (), buffer , seenObjects , null );
113
139
if (++count < map .size ())
114
140
buffer .append (", " );
115
141
}
@@ -120,15 +146,24 @@ private void serialize(final Object datum, final StringBuilder buffer, final Ide
120
146
writeEscapedString (datum .toString (), buffer );
121
147
buffer .append ("\" " );
122
148
} else if (isBytes (datum )) {
123
- final String bytesAsString = StandardCharsets .UTF_8 .decode ((ByteBuffer ) datum ).toString ();
124
- final Optional <BigDecimal > bytesAsBigDecimal = getBigDecimal (bytesAsString );
125
- if (bytesAsBigDecimal .isPresent ()) {
126
- buffer .append (bytesAsBigDecimal .get ().doubleValue ());
149
+ if (decimalScale != null ) {
150
+ ByteBuffer sourceBuffer = (ByteBuffer ) datum ;
151
+ byte [] bytesArray = new byte [sourceBuffer .remaining ()];
152
+ sourceBuffer .duplicate ().get (bytesArray );
153
+ BigInteger unscaledValue = new BigInteger (bytesArray );
154
+ BigDecimal decimal = new BigDecimal (unscaledValue , decimalScale );
155
+ buffer .append (decimal .doubleValue ());
127
156
} else {
128
- buffer .append ("{\" bytes\" : \" " );
129
- ByteBuffer bytes = ((ByteBuffer ) datum ).duplicate ();
130
- writeEscapedString (new String (bytes .array (), StandardCharsets .ISO_8859_1 ), buffer );
131
- buffer .append ("\" }" );
157
+ final String bytesAsString = StandardCharsets .UTF_8 .decode ((ByteBuffer ) datum ).toString ();
158
+ final Optional <BigDecimal > bytesAsBigDecimal = getBigDecimal (bytesAsString );
159
+ if (bytesAsBigDecimal .isPresent ()) {
160
+ buffer .append (bytesAsBigDecimal .get ().doubleValue ());
161
+ } else {
162
+ buffer .append ("{\" bytes\" : \" " );
163
+ ByteBuffer bytes = ((ByteBuffer ) datum ).duplicate ();
164
+ writeEscapedString (new String (bytes .array (), StandardCharsets .ISO_8859_1 ), buffer );
165
+ buffer .append ("\" }" );
166
+ }
132
167
}
133
168
} else if (((datum instanceof Float ) && // quote Nan & Infinity
134
169
(((Float )datum ).isInfinite () || ((Float )datum ).isNaN ()))
@@ -143,7 +178,7 @@ private void serialize(final Object datum, final StringBuilder buffer, final Ide
143
178
return ;
144
179
}
145
180
seenObjects .put (datum , datum );
146
- serialize (datum , buffer , seenObjects );
181
+ serialize (datum , buffer , seenObjects , null );
147
182
seenObjects .remove (datum );
148
183
} else {
149
184
// This fallback is the reason why GenericRecord toString does not
0 commit comments