@@ -42,6 +42,9 @@ public final class ClickHouseColumn implements Serializable {
42
42
private int arrayLevel ;
43
43
private ClickHouseColumn arrayBaseColumn ;
44
44
45
+ private boolean fixedByteLength ;
46
+ private int estimatedByteLength ;
47
+
45
48
private static ClickHouseColumn update (ClickHouseColumn column ) {
46
49
column .enumConstants = ClickHouseEnum .EMPTY ;
47
50
int size = column .parameters .size ();
@@ -70,11 +73,17 @@ private static ClickHouseColumn update(ClickHouseColumn column) {
70
73
if (size >= 2 ) { // same as DateTime64
71
74
column .scale = Integer .parseInt (column .parameters .get (0 ));
72
75
column .timeZone = TimeZone .getTimeZone (column .parameters .get (1 ).replace ("'" , "" ));
76
+ if (!column .nullable ) {
77
+ column .estimatedByteLength += ClickHouseDataType .DateTime64 .getByteLength ();
78
+ }
73
79
} else if (size == 1 ) { // same as DateTime32
74
80
// unfortunately this will fall back to GMT if the time zone
75
81
// cannot be resolved
76
82
TimeZone tz = TimeZone .getTimeZone (column .parameters .get (0 ).replace ("'" , "" ));
77
83
column .timeZone = tz ;
84
+ if (!column .nullable ) {
85
+ column .estimatedByteLength += ClickHouseDataType .DateTime32 .getByteLength ();
86
+ }
78
87
}
79
88
break ;
80
89
case DateTime32 :
@@ -97,6 +106,18 @@ private static ClickHouseColumn update(ClickHouseColumn column) {
97
106
if (size >= 2 ) {
98
107
column .precision = Integer .parseInt (column .parameters .get (0 ));
99
108
column .scale = Integer .parseInt (column .parameters .get (1 ));
109
+
110
+ if (!column .nullable ) {
111
+ if (column .precision > ClickHouseDataType .Decimal128 .getMaxScale ()) {
112
+ column .estimatedByteLength += ClickHouseDataType .Decimal256 .getByteLength ();
113
+ } else if (column .precision > ClickHouseDataType .Decimal64 .getMaxScale ()) {
114
+ column .estimatedByteLength += ClickHouseDataType .Decimal128 .getByteLength ();
115
+ } else if (column .precision > ClickHouseDataType .Decimal32 .getMaxScale ()) {
116
+ column .estimatedByteLength += ClickHouseDataType .Decimal64 .getByteLength ();
117
+ } else {
118
+ column .estimatedByteLength += ClickHouseDataType .Decimal32 .getByteLength ();
119
+ }
120
+ }
100
121
}
101
122
break ;
102
123
case Decimal32 :
@@ -110,6 +131,15 @@ private static ClickHouseColumn update(ClickHouseColumn column) {
110
131
case FixedString :
111
132
if (size > 0 ) {
112
133
column .precision = Integer .parseInt (column .parameters .get (0 ));
134
+ if (!column .nullable ) {
135
+ column .estimatedByteLength += column .precision ;
136
+ }
137
+ }
138
+ break ;
139
+ case String :
140
+ column .fixedByteLength = false ;
141
+ if (!column .nullable ) {
142
+ column .estimatedByteLength += 1 ;
113
143
}
114
144
break ;
115
145
default :
@@ -129,6 +159,9 @@ protected static int readColumn(String args, int startIndex, int len, String nam
129
159
boolean lowCardinality = false ;
130
160
int i = startIndex ;
131
161
162
+ boolean fixedLength = true ;
163
+ int estimatedLength = 0 ;
164
+
132
165
if (args .startsWith (KEYWORD_LOW_CARDINALITY , i )) {
133
166
lowCardinality = true ;
134
167
int index = args .indexOf ('(' , i + KEYWORD_LOW_CARDINALITY .length ());
@@ -173,6 +206,8 @@ protected static int readColumn(String args, int startIndex, int len, String nam
173
206
column = new ClickHouseColumn (ClickHouseDataType .valueOf (matchedKeyword ), name ,
174
207
args .substring (startIndex , i ), nullable , lowCardinality , params , nestedColumns );
175
208
column .aggFuncType = aggFunc ;
209
+ fixedLength = false ;
210
+ estimatedLength ++;
176
211
} else if (args .startsWith (KEYWORD_ARRAY , i )) {
177
212
int index = args .indexOf ('(' , i + KEYWORD_ARRAY .length ());
178
213
if (index < i ) {
@@ -188,6 +223,8 @@ protected static int readColumn(String args, int startIndex, int len, String nam
188
223
column = new ClickHouseColumn (ClickHouseDataType .Array , name , args .substring (startIndex , endIndex ),
189
224
nullable , lowCardinality , null , nestedColumns );
190
225
i = endIndex ;
226
+ fixedLength = false ;
227
+ estimatedLength ++;
191
228
} else if (args .startsWith (KEYWORD_MAP , i )) {
192
229
int index = args .indexOf ('(' , i + KEYWORD_MAP .length ());
193
230
if (index < i ) {
@@ -210,6 +247,8 @@ protected static int readColumn(String args, int startIndex, int len, String nam
210
247
column = new ClickHouseColumn (ClickHouseDataType .Map , name , args .substring (startIndex , endIndex ), nullable ,
211
248
lowCardinality , null , nestedColumns );
212
249
i = endIndex ;
250
+ fixedLength = false ;
251
+ estimatedLength ++;
213
252
} else if (args .startsWith (KEYWORD_NESTED , i )) {
214
253
int index = args .indexOf ('(' , i + KEYWORD_NESTED .length ());
215
254
if (index < i ) {
@@ -223,6 +262,8 @@ protected static int readColumn(String args, int startIndex, int len, String nam
223
262
}
224
263
column = new ClickHouseColumn (ClickHouseDataType .Nested , name , originalTypeName , nullable , lowCardinality ,
225
264
null , nestedColumns );
265
+ fixedLength = false ;
266
+ estimatedLength ++;
226
267
} else if (args .startsWith (KEYWORD_TUPLE , i )) {
227
268
int index = args .indexOf ('(' , i + KEYWORD_TUPLE .length ());
228
269
if (index < i ) {
@@ -243,6 +284,12 @@ protected static int readColumn(String args, int startIndex, int len, String nam
243
284
}
244
285
column = new ClickHouseColumn (ClickHouseDataType .Tuple , name , args .substring (startIndex , endIndex ),
245
286
nullable , lowCardinality , null , nestedColumns );
287
+ for (ClickHouseColumn n : nestedColumns ) {
288
+ estimatedLength += n .estimatedByteLength ;
289
+ if (!n .fixedByteLength ) {
290
+ fixedLength = false ;
291
+ }
292
+ }
246
293
}
247
294
248
295
if (column == null ) {
@@ -298,6 +345,16 @@ protected static int readColumn(String args, int startIndex, int len, String nam
298
345
builder .setLength (0 );
299
346
}
300
347
348
+ if (nullable ) {
349
+ fixedLength = false ;
350
+ estimatedLength ++;
351
+ } else if (column .dataType .getByteLength () == 0 ) {
352
+ fixedLength = false ;
353
+ } else {
354
+ estimatedLength += column .dataType .getByteLength ();
355
+ }
356
+ column .fixedByteLength = fixedLength ;
357
+ column .estimatedByteLength = estimatedLength ;
301
358
list .add (update (column ));
302
359
303
360
return i ;
@@ -373,11 +430,6 @@ public static List<ClickHouseColumn> parse(String args) {
373
430
return Collections .unmodifiableList (c );
374
431
}
375
432
376
- private ClickHouseColumn (String originalTypeName , String columnName ) {
377
- this .originalTypeName = originalTypeName ;
378
- this .columnName = columnName ;
379
- }
380
-
381
433
private ClickHouseColumn (ClickHouseDataType dataType , String columnName , String originalTypeName , boolean nullable ,
382
434
boolean lowCardinality , List <String > parameters , List <ClickHouseColumn > nestedColumns ) {
383
435
this .aggFuncType = null ;
@@ -403,6 +455,9 @@ private ClickHouseColumn(ClickHouseDataType dataType, String columnName, String
403
455
list .addAll (nestedColumns );
404
456
this .nested = Collections .unmodifiableList (list );
405
457
}
458
+
459
+ this .fixedByteLength = false ;
460
+ this .estimatedByteLength = 0 ;
406
461
}
407
462
408
463
public boolean isAggregateFunction () {
@@ -420,6 +475,10 @@ public boolean isEnum() {
420
475
|| dataType == ClickHouseDataType .Enum16 ;
421
476
}
422
477
478
+ public boolean isFixedLength () {
479
+ return fixedByteLength ;
480
+ }
481
+
423
482
public boolean isMap () {
424
483
return dataType == ClickHouseDataType .Map ;
425
484
}
@@ -448,6 +507,10 @@ public ClickHouseEnum getEnumConstants() {
448
507
return enumConstants ;
449
508
}
450
509
510
+ public int getEstimatedLength () {
511
+ return estimatedByteLength ;
512
+ }
513
+
451
514
public String getOriginalTypeName () {
452
515
return originalTypeName ;
453
516
}
@@ -541,6 +604,8 @@ public int hashCode() {
541
604
result = prime * result + precision ;
542
605
result = prime * result + scale ;
543
606
result = prime * result + ((timeZone == null ) ? 0 : timeZone .hashCode ());
607
+ result = prime * result + (fixedByteLength ? 1231 : 1237 );
608
+ result = prime * result + estimatedByteLength ;
544
609
return result ;
545
610
}
546
611
@@ -561,7 +626,8 @@ public boolean equals(Object obj) {
561
626
&& Objects .equals (nested , other .nested ) && nullable == other .nullable
562
627
&& Objects .equals (originalTypeName , other .originalTypeName )
563
628
&& Objects .equals (parameters , other .parameters ) && precision == other .precision && scale == other .scale
564
- && Objects .equals (timeZone , other .timeZone );
629
+ && Objects .equals (timeZone , other .timeZone ) && fixedByteLength == other .fixedByteLength
630
+ && estimatedByteLength == other .estimatedByteLength ;
565
631
}
566
632
567
633
@ Override
0 commit comments