@@ -42,6 +42,9 @@ public final class ClickHouseColumn implements Serializable {
4242 private int arrayLevel ;
4343 private ClickHouseColumn arrayBaseColumn ;
4444
45+ private boolean fixedByteLength ;
46+ private int estimatedByteLength ;
47+
4548 private static ClickHouseColumn update (ClickHouseColumn column ) {
4649 column .enumConstants = ClickHouseEnum .EMPTY ;
4750 int size = column .parameters .size ();
@@ -70,11 +73,17 @@ private static ClickHouseColumn update(ClickHouseColumn column) {
7073 if (size >= 2 ) { // same as DateTime64
7174 column .scale = Integer .parseInt (column .parameters .get (0 ));
7275 column .timeZone = TimeZone .getTimeZone (column .parameters .get (1 ).replace ("'" , "" ));
76+ if (!column .nullable ) {
77+ column .estimatedByteLength += ClickHouseDataType .DateTime64 .getByteLength ();
78+ }
7379 } else if (size == 1 ) { // same as DateTime32
7480 // unfortunately this will fall back to GMT if the time zone
7581 // cannot be resolved
7682 TimeZone tz = TimeZone .getTimeZone (column .parameters .get (0 ).replace ("'" , "" ));
7783 column .timeZone = tz ;
84+ if (!column .nullable ) {
85+ column .estimatedByteLength += ClickHouseDataType .DateTime32 .getByteLength ();
86+ }
7887 }
7988 break ;
8089 case DateTime32 :
@@ -97,6 +106,18 @@ private static ClickHouseColumn update(ClickHouseColumn column) {
97106 if (size >= 2 ) {
98107 column .precision = Integer .parseInt (column .parameters .get (0 ));
99108 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+ }
100121 }
101122 break ;
102123 case Decimal32 :
@@ -110,6 +131,15 @@ private static ClickHouseColumn update(ClickHouseColumn column) {
110131 case FixedString :
111132 if (size > 0 ) {
112133 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 ;
113143 }
114144 break ;
115145 default :
@@ -129,6 +159,9 @@ protected static int readColumn(String args, int startIndex, int len, String nam
129159 boolean lowCardinality = false ;
130160 int i = startIndex ;
131161
162+ boolean fixedLength = true ;
163+ int estimatedLength = 0 ;
164+
132165 if (args .startsWith (KEYWORD_LOW_CARDINALITY , i )) {
133166 lowCardinality = true ;
134167 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
173206 column = new ClickHouseColumn (ClickHouseDataType .valueOf (matchedKeyword ), name ,
174207 args .substring (startIndex , i ), nullable , lowCardinality , params , nestedColumns );
175208 column .aggFuncType = aggFunc ;
209+ fixedLength = false ;
210+ estimatedLength ++;
176211 } else if (args .startsWith (KEYWORD_ARRAY , i )) {
177212 int index = args .indexOf ('(' , i + KEYWORD_ARRAY .length ());
178213 if (index < i ) {
@@ -188,6 +223,8 @@ protected static int readColumn(String args, int startIndex, int len, String nam
188223 column = new ClickHouseColumn (ClickHouseDataType .Array , name , args .substring (startIndex , endIndex ),
189224 nullable , lowCardinality , null , nestedColumns );
190225 i = endIndex ;
226+ fixedLength = false ;
227+ estimatedLength ++;
191228 } else if (args .startsWith (KEYWORD_MAP , i )) {
192229 int index = args .indexOf ('(' , i + KEYWORD_MAP .length ());
193230 if (index < i ) {
@@ -210,6 +247,8 @@ protected static int readColumn(String args, int startIndex, int len, String nam
210247 column = new ClickHouseColumn (ClickHouseDataType .Map , name , args .substring (startIndex , endIndex ), nullable ,
211248 lowCardinality , null , nestedColumns );
212249 i = endIndex ;
250+ fixedLength = false ;
251+ estimatedLength ++;
213252 } else if (args .startsWith (KEYWORD_NESTED , i )) {
214253 int index = args .indexOf ('(' , i + KEYWORD_NESTED .length ());
215254 if (index < i ) {
@@ -223,6 +262,8 @@ protected static int readColumn(String args, int startIndex, int len, String nam
223262 }
224263 column = new ClickHouseColumn (ClickHouseDataType .Nested , name , originalTypeName , nullable , lowCardinality ,
225264 null , nestedColumns );
265+ fixedLength = false ;
266+ estimatedLength ++;
226267 } else if (args .startsWith (KEYWORD_TUPLE , i )) {
227268 int index = args .indexOf ('(' , i + KEYWORD_TUPLE .length ());
228269 if (index < i ) {
@@ -243,6 +284,12 @@ protected static int readColumn(String args, int startIndex, int len, String nam
243284 }
244285 column = new ClickHouseColumn (ClickHouseDataType .Tuple , name , args .substring (startIndex , endIndex ),
245286 nullable , lowCardinality , null , nestedColumns );
287+ for (ClickHouseColumn n : nestedColumns ) {
288+ estimatedLength += n .estimatedByteLength ;
289+ if (!n .fixedByteLength ) {
290+ fixedLength = false ;
291+ }
292+ }
246293 }
247294
248295 if (column == null ) {
@@ -298,6 +345,16 @@ protected static int readColumn(String args, int startIndex, int len, String nam
298345 builder .setLength (0 );
299346 }
300347
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 ;
301358 list .add (update (column ));
302359
303360 return i ;
@@ -373,11 +430,6 @@ public static List<ClickHouseColumn> parse(String args) {
373430 return Collections .unmodifiableList (c );
374431 }
375432
376- private ClickHouseColumn (String originalTypeName , String columnName ) {
377- this .originalTypeName = originalTypeName ;
378- this .columnName = columnName ;
379- }
380-
381433 private ClickHouseColumn (ClickHouseDataType dataType , String columnName , String originalTypeName , boolean nullable ,
382434 boolean lowCardinality , List <String > parameters , List <ClickHouseColumn > nestedColumns ) {
383435 this .aggFuncType = null ;
@@ -403,6 +455,9 @@ private ClickHouseColumn(ClickHouseDataType dataType, String columnName, String
403455 list .addAll (nestedColumns );
404456 this .nested = Collections .unmodifiableList (list );
405457 }
458+
459+ this .fixedByteLength = false ;
460+ this .estimatedByteLength = 0 ;
406461 }
407462
408463 public boolean isAggregateFunction () {
@@ -420,6 +475,10 @@ public boolean isEnum() {
420475 || dataType == ClickHouseDataType .Enum16 ;
421476 }
422477
478+ public boolean isFixedLength () {
479+ return fixedByteLength ;
480+ }
481+
423482 public boolean isMap () {
424483 return dataType == ClickHouseDataType .Map ;
425484 }
@@ -448,6 +507,10 @@ public ClickHouseEnum getEnumConstants() {
448507 return enumConstants ;
449508 }
450509
510+ public int getEstimatedLength () {
511+ return estimatedByteLength ;
512+ }
513+
451514 public String getOriginalTypeName () {
452515 return originalTypeName ;
453516 }
@@ -541,6 +604,8 @@ public int hashCode() {
541604 result = prime * result + precision ;
542605 result = prime * result + scale ;
543606 result = prime * result + ((timeZone == null ) ? 0 : timeZone .hashCode ());
607+ result = prime * result + (fixedByteLength ? 1231 : 1237 );
608+ result = prime * result + estimatedByteLength ;
544609 return result ;
545610 }
546611
@@ -561,7 +626,8 @@ public boolean equals(Object obj) {
561626 && Objects .equals (nested , other .nested ) && nullable == other .nullable
562627 && Objects .equals (originalTypeName , other .originalTypeName )
563628 && 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 ;
565631 }
566632
567633 @ Override
0 commit comments