@@ -1042,4 +1042,209 @@ public void testBindWithNestedOptionalRecord() {
1042
1042
Record boundRecordWithNull = recordBinder .bind (envelopeRecordWithNull );
1043
1043
assertNull (boundRecordWithNull .getField ("before" ));
1044
1044
}
1045
+
1046
+ // Test method for field count statistics
1047
+ @ Test
1048
+ public void testFieldCountStatistics () {
1049
+ // Test different field types and their count calculations
1050
+ String avroSchemaStr = "{\n " +
1051
+ " \" type\" : \" record\" ,\n " +
1052
+ " \" name\" : \" TestRecord\" ,\n " +
1053
+ " \" fields\" : [\n " +
1054
+ " {\" name\" : \" smallString\" , \" type\" : \" string\" },\n " +
1055
+ " {\" name\" : \" largeString\" , \" type\" : \" string\" },\n " +
1056
+ " {\" name\" : \" intField\" , \" type\" : \" int\" },\n " +
1057
+ " {\" name\" : \" binaryField\" , \" type\" : \" bytes\" }\n " +
1058
+ " ]\n " +
1059
+ "}" ;
1060
+
1061
+ Schema avroSchema = new Schema .Parser ().parse (avroSchemaStr );
1062
+ org .apache .iceberg .Schema icebergSchema = AvroSchemaUtil .toIceberg (avroSchema );
1063
+ RecordBinder recordBinder = new RecordBinder (icebergSchema , avroSchema );
1064
+
1065
+ // Create test record with different field sizes
1066
+ GenericRecord avroRecord = new GenericData .Record (avroSchema );
1067
+ avroRecord .put ("smallString" , "small" ); // 5 chars = 1 field (5+23)/24 = 1
1068
+ avroRecord .put ("largeString" , "a" .repeat (50 )); // 50 chars = 3 fields (50+23)/24 = 3
1069
+ avroRecord .put ("intField" , 42 ); // primitive = 1 field
1070
+ avroRecord .put ("binaryField" , ByteBuffer .wrap ("test" .repeat (10 ).getBytes ())); // 40 bytes = 2 fields (40+31)/32 = 2
1071
+
1072
+ // Bind record - this should trigger field counting
1073
+ Record icebergRecord = recordBinder .bind (avroRecord );
1074
+
1075
+ // Access all fields to trigger counting
1076
+ assertEquals ("small" , icebergRecord .getField ("smallString" ));
1077
+ assertEquals ("a" .repeat (50 ), icebergRecord .getField ("largeString" ));
1078
+ assertEquals (42 , icebergRecord .getField ("intField" ));
1079
+ assertEquals ("test" .repeat (10 ), new String (((ByteBuffer ) icebergRecord .getField ("binaryField" )).array ()));
1080
+
1081
+ // Check field count: 1 + 3 + 1 + 2 = 7 fields total
1082
+ long fieldCount = recordBinder .getAndResetFieldCount ();
1083
+ assertEquals (7 , fieldCount );
1084
+
1085
+ // Second call should return 0 (reset)
1086
+ assertEquals (0 , recordBinder .getAndResetFieldCount ());
1087
+
1088
+ testSendRecord (icebergSchema .asStruct ().asSchema (), icebergRecord );
1089
+ assertEquals (7 , recordBinder .getAndResetFieldCount ());
1090
+ }
1091
+
1092
+ @ Test
1093
+ public void testFieldCountWithComplexTypes () {
1094
+ // Test field counting for LIST and MAP types
1095
+ String avroSchemaStr = "{\n " +
1096
+ " \" type\" : \" record\" ,\n " +
1097
+ " \" name\" : \" ComplexRecord\" ,\n " +
1098
+ " \" fields\" : [\n " +
1099
+ " {\" name\" : \" stringList\" , \" type\" : {\" type\" : \" array\" , \" items\" : \" string\" }},\n " +
1100
+ " {\" name\" : \" stringMap\" , \" type\" : {\" type\" : \" map\" , \" values\" : \" string\" }}\n " +
1101
+ " ]\n " +
1102
+ "}" ;
1103
+
1104
+ Schema avroSchema = new Schema .Parser ().parse (avroSchemaStr );
1105
+ org .apache .iceberg .Schema icebergSchema = AvroSchemaUtil .toIceberg (avroSchema );
1106
+ RecordBinder recordBinder = new RecordBinder (icebergSchema , avroSchema );
1107
+
1108
+ GenericRecord avroRecord = new GenericData .Record (avroSchema );
1109
+ // List with 3 small strings: 1 (list itself) + 3 * 1 = 4 fields
1110
+ avroRecord .put ("stringList" , Arrays .asList ("a" , "b" , "c" ));
1111
+
1112
+ // Map with 2 entries: 1 (map itself) + 2 * (1 key + 1 value) = 5 fields
1113
+ Map <String , String > map = new HashMap <>();
1114
+ map .put ("key1" , "val1" );
1115
+ map .put ("key2" , "val2" );
1116
+ avroRecord .put ("stringMap" , map );
1117
+
1118
+ Record icebergRecord = recordBinder .bind (avroRecord );
1119
+
1120
+ // Access fields to trigger counting
1121
+ assertEquals (Arrays .asList ("a" , "b" , "c" ), icebergRecord .getField ("stringList" ));
1122
+ assertEquals (map , icebergRecord .getField ("stringMap" ));
1123
+
1124
+ // Total: 4 (list) + 5 (map) = 9 fields
1125
+ long fieldCount = recordBinder .getAndResetFieldCount ();
1126
+ assertEquals (9 , fieldCount );
1127
+
1128
+ testSendRecord (icebergSchema .asStruct ().asSchema (), icebergRecord );
1129
+ assertEquals (9 , recordBinder .getAndResetFieldCount ());
1130
+ }
1131
+
1132
+ @ Test
1133
+ public void testFieldCountWithNestedStructure () {
1134
+ // Test field counting for nested records
1135
+ String avroSchemaStr = "{\n " +
1136
+ " \" type\" : \" record\" ,\n " +
1137
+ " \" name\" : \" NestedRecord\" ,\n " +
1138
+ " \" fields\" : [\n " +
1139
+ " {\" name\" : \" simpleField\" , \" type\" : \" string\" },\n " +
1140
+ " {\n " +
1141
+ " \" name\" : \" nestedField\" ,\n " +
1142
+ " \" type\" : {\n " +
1143
+ " \" type\" : \" record\" ,\n " +
1144
+ " \" name\" : \" Nested\" ,\n " +
1145
+ " \" fields\" : [\n " +
1146
+ " {\" name\" : \" nestedString\" , \" type\" : \" string\" },\n " +
1147
+ " {\" name\" : \" nestedInt\" , \" type\" : \" int\" }\n " +
1148
+ " ]\n " +
1149
+ " }\n " +
1150
+ " }\n " +
1151
+ " ]\n " +
1152
+ "}" ;
1153
+
1154
+ Schema avroSchema = new Schema .Parser ().parse (avroSchemaStr );
1155
+ org .apache .iceberg .Schema icebergSchema = AvroSchemaUtil .toIceberg (avroSchema );
1156
+ RecordBinder recordBinder = new RecordBinder (icebergSchema , avroSchema );
1157
+
1158
+ // Create nested record
1159
+ GenericRecord nestedRecord = new GenericData .Record (avroSchema .getField ("nestedField" ).schema ());
1160
+ nestedRecord .put ("nestedString" , "nested" ); // 1 field
1161
+ nestedRecord .put ("nestedInt" , 123 ); // 1 field
1162
+
1163
+ GenericRecord mainRecord = new GenericData .Record (avroSchema );
1164
+ mainRecord .put ("simpleField" , "simple" ); // 1 field
1165
+ mainRecord .put ("nestedField" , nestedRecord ); // STRUCT fields are counted when accessed
1166
+
1167
+ Record icebergRecord = recordBinder .bind (mainRecord );
1168
+
1169
+ // Access all fields including nested ones
1170
+ assertEquals ("simple" , icebergRecord .getField ("simpleField" ));
1171
+ Record nested = (Record ) icebergRecord .getField ("nestedField" );
1172
+ assertEquals ("nested" , nested .getField ("nestedString" ));
1173
+ assertEquals (123 , nested .getField ("nestedInt" ));
1174
+
1175
+ // Total: 1 (simple) + 1(struct) + 1 (nested string) + 1 (nested int) = 4 fields
1176
+ // Note: STRUCT type itself doesn't add to count, only its leaf fields
1177
+ long fieldCount = recordBinder .getAndResetFieldCount ();
1178
+ assertEquals (4 , fieldCount );
1179
+
1180
+ testSendRecord (icebergSchema .asStruct ().asSchema (), icebergRecord );
1181
+ assertEquals (4 , recordBinder .getAndResetFieldCount ());
1182
+ }
1183
+
1184
+ @ Test
1185
+ public void testFieldCountBatchAccumulation () {
1186
+ // Test that field counts accumulate across multiple record bindings
1187
+ String avroSchemaStr = "{\n " +
1188
+ " \" type\" : \" record\" ,\n " +
1189
+ " \" name\" : \" SimpleRecord\" ,\n " +
1190
+ " \" fields\" : [\n " +
1191
+ " {\" name\" : \" stringField\" , \" type\" : \" string\" },\n " +
1192
+ " {\" name\" : \" intField\" , \" type\" : \" int\" }\n " +
1193
+ " ]\n " +
1194
+ "}" ;
1195
+
1196
+ Schema avroSchema = new Schema .Parser ().parse (avroSchemaStr );
1197
+ org .apache .iceberg .Schema icebergSchema = AvroSchemaUtil .toIceberg (avroSchema );
1198
+ RecordBinder recordBinder = new RecordBinder (icebergSchema , avroSchema );
1199
+
1200
+ // Process multiple records
1201
+ for (int i = 0 ; i < 3 ; i ++) {
1202
+ GenericRecord avroRecord = new GenericData .Record (avroSchema );
1203
+ avroRecord .put ("stringField" , "test" + i ); // 1 field each
1204
+ avroRecord .put ("intField" , i ); // 1 field each
1205
+
1206
+ Record icebergRecord = recordBinder .bind (avroRecord );
1207
+ // Access fields to trigger counting
1208
+ icebergRecord .getField ("stringField" );
1209
+ icebergRecord .getField ("intField" );
1210
+ }
1211
+
1212
+ // Total: 3 records * 2 fields each = 6 fields
1213
+ long totalFieldCount = recordBinder .getAndResetFieldCount ();
1214
+ assertEquals (6 , totalFieldCount );
1215
+ }
1216
+
1217
+ @ Test
1218
+ public void testFieldCountWithNullValues () {
1219
+ // Test that null values don't contribute to field count
1220
+ String avroSchemaStr = "{\n " +
1221
+ " \" type\" : \" record\" ,\n " +
1222
+ " \" name\" : \" NullableRecord\" ,\n " +
1223
+ " \" fields\" : [\n " +
1224
+ " {\" name\" : \" nonNullField\" , \" type\" : \" string\" },\n " +
1225
+ " {\" name\" : \" nullField\" , \" type\" : [\" null\" , \" string\" ], \" default\" : null}\n " +
1226
+ " ]\n " +
1227
+ "}" ;
1228
+
1229
+ Schema avroSchema = new Schema .Parser ().parse (avroSchemaStr );
1230
+ org .apache .iceberg .Schema icebergSchema = AvroSchemaUtil .toIceberg (avroSchema );
1231
+ RecordBinder recordBinder = new RecordBinder (icebergSchema , avroSchema );
1232
+
1233
+ GenericRecord avroRecord = new GenericData .Record (avroSchema );
1234
+ avroRecord .put ("nonNullField" , "value" ); // 1 field
1235
+ avroRecord .put ("nullField" , null ); // 0 fields
1236
+
1237
+ Record icebergRecord = recordBinder .bind (avroRecord );
1238
+
1239
+ // Access both fields
1240
+ assertEquals ("value" , icebergRecord .getField ("nonNullField" ));
1241
+ assertNull (icebergRecord .getField ("nullField" ));
1242
+
1243
+ // Only the non-null field should count
1244
+ long fieldCount = recordBinder .getAndResetFieldCount ();
1245
+ assertEquals (1 , fieldCount );
1246
+
1247
+ testSendRecord (icebergSchema .asStruct ().asSchema (), icebergRecord );
1248
+ assertEquals (1 , recordBinder .getAndResetFieldCount ());
1249
+ }
1045
1250
}
0 commit comments