|
23 | 23 | import com.google.gson.reflect.TypeToken;
|
24 | 24 |
|
25 | 25 | import java.math.BigDecimal;
|
| 26 | +import java.util.*; |
26 | 27 | import java.util.concurrent.TimeUnit;
|
27 | 28 |
|
28 | 29 | import org.apache.kafka.connect.data.Field;
|
|
35 | 36 | import java.io.IOException;
|
36 | 37 | import java.nio.charset.StandardCharsets;
|
37 | 38 | import java.util.concurrent.atomic.AtomicBoolean;
|
38 |
| -import java.util.Date; |
39 |
| -import java.util.HashMap; |
40 |
| -import java.util.List; |
41 |
| -import java.util.Map; |
42 |
| -import java.util.UUID; |
43 | 39 | import java.util.concurrent.CompletableFuture;
|
44 | 40 | import java.util.concurrent.ExecutionException;
|
45 | 41 | import java.util.stream.Collectors;
|
@@ -354,8 +350,10 @@ private void doWriteCol(Record record, Column col, ClickHousePipedOutputStream s
|
354 | 350 | BinaryStreamUtils.writeNonNull(stream);
|
355 | 351 | }
|
356 | 352 | if (!col.isNullable() && value.getObject() == null) {
|
357 |
| - // this the situation when the col is not isNullable, but the data is null here we need to drop the records |
358 |
| - throw new RuntimeException(String.format("An attempt to write null into not nullable column '%s'", col.getName())); |
| 353 | + if (colType == Type.ARRAY) |
| 354 | + BinaryStreamUtils.writeNonNull(stream); |
| 355 | + else |
| 356 | + throw new RuntimeException(String.format("An attempt to write null into not nullable column '%s'", col.getName())); |
359 | 357 | }
|
360 | 358 | switch (colType) {
|
361 | 359 | case INT8:
|
@@ -403,18 +401,23 @@ private void doWriteCol(Record record, Column col, ClickHousePipedOutputStream s
|
403 | 401 | break;
|
404 | 402 | case ARRAY:
|
405 | 403 | List<?> arrObject = (List<?>) value.getObject();
|
406 |
| - int sizeArrObject = arrObject.size(); |
407 |
| - BinaryStreamUtils.writeVarInt(stream, sizeArrObject); |
408 |
| - arrObject.forEach(v -> { |
409 |
| - try { |
410 |
| - if (col.getSubType().isNullable() && v != null) { |
411 |
| - BinaryStreamUtils.writeNonNull(stream); |
| 404 | + |
| 405 | + if (arrObject == null) { |
| 406 | + doWritePrimitive(colType, value.getFieldType(), stream, new ArrayList<>()); |
| 407 | + } else { |
| 408 | + int sizeArrObject = arrObject.size(); |
| 409 | + BinaryStreamUtils.writeVarInt(stream, sizeArrObject); |
| 410 | + arrObject.forEach(v -> { |
| 411 | + try { |
| 412 | + if (col.getSubType().isNullable() && v != null) { |
| 413 | + BinaryStreamUtils.writeNonNull(stream); |
| 414 | + } |
| 415 | + doWritePrimitive(col.getSubType().getType(), value.getFieldType(), stream, v); |
| 416 | + } catch (IOException e) { |
| 417 | + throw new RuntimeException(e); |
412 | 418 | }
|
413 |
| - doWritePrimitive(col.getSubType().getType(), value.getFieldType(), stream, v); |
414 |
| - } catch (IOException e) { |
415 |
| - throw new RuntimeException(e); |
416 |
| - } |
417 |
| - }); |
| 419 | + }); |
| 420 | + } |
418 | 421 | break;
|
419 | 422 | }
|
420 | 423 | } else {
|
|
0 commit comments