Skip to content

Commit 917ef41

Browse files
committed
Fix indexOutOfBoundException
1 parent ecc4fe4 commit 917ef41

File tree

3 files changed

+94
-53
lines changed

3 files changed

+94
-53
lines changed

src/main/java/com/snowflake/kafka/connector/internal/streaming/TopicPartitionChannel.java

Lines changed: 18 additions & 13 deletions
Original file line numberDiff line numberDiff line change
@@ -654,10 +654,10 @@ public InsertRowsResponse get() throws Throwable {
654654
"Invoking insertRows API for channel:{}, streamingBuffer:{}",
655655
this.channel.getFullyQualifiedName(),
656656
this.insertRowsStreamingBuffer);
657-
Pair<List<Map<String, Object>>, List<Long>> recordsAndOffsets =
657+
Pair<List<Map<String, Object>>, List<SinkRecord>> recordsAndOriginalSinkRecords =
658658
this.insertRowsStreamingBuffer.getData();
659-
List<Map<String, Object>> records = recordsAndOffsets.getKey();
660-
List<Long> offsets = recordsAndOffsets.getValue();
659+
List<Map<String, Object>> records = recordsAndOriginalSinkRecords.getKey();
660+
List<SinkRecord> originalSinkRecords = recordsAndOriginalSinkRecords.getValue();
661661
InsertValidationResponse finalResponse = new InsertValidationResponse();
662662
boolean needToResetOffset = false;
663663
if (!enableSchemaEvolution) {
@@ -671,14 +671,18 @@ public InsertRowsResponse get() throws Throwable {
671671
// For schema evolution, we need to call the insertRows API row by row in order to
672672
// preserve the original order, for anything after the first schema mismatch error we will
673673
// retry after the evolution
674-
InsertValidationResponse response =
675-
this.channel.insertRow(records.get(idx), Long.toString(offsets.get(idx)));
674+
SinkRecord originalSinkRecord = originalSinkRecords.get(idx);
675+
InsertValidationResponse response = this.channel.insertRow(
676+
records.get(idx), Long.toString(originalSinkRecord.kafkaOffset())
677+
);
676678
if (response.hasErrors()) {
677679
InsertValidationResponse.InsertError insertError = response.getInsertErrors().get(0);
678680
List<String> extraColNames = insertError.getExtraColNames();
679681
List<String> nonNullableColumns = insertError.getMissingNotNullColNames();
680-
long originalSinkRecordIdx =
681-
offsets.get(idx) - this.insertRowsStreamingBuffer.getFirstOffset();
682+
683+
// TODO : originalSinkRecordIdx can be replaced by idx
684+
long originalSinkRecordIdx = originalSinkRecord.kafkaOffset()
685+
- this.insertRowsStreamingBuffer.getFirstOffset();
682686
if (extraColNames == null && nonNullableColumns == null) {
683687
InsertValidationResponse.InsertError newInsertError =
684688
new InsertValidationResponse.InsertError(
@@ -694,7 +698,8 @@ public InsertRowsResponse get() throws Throwable {
694698
this.channel.getTableName(),
695699
nonNullableColumns,
696700
extraColNames,
697-
this.insertRowsStreamingBuffer.getSinkRecord(originalSinkRecordIdx));
701+
originalSinkRecord
702+
);
698703
// Offset reset needed since it's possible that we successfully ingested partial batch
699704
needToResetOffset = true;
700705
break;
@@ -1251,7 +1256,7 @@ protected long getApproxSizeOfRecordInBytes(SinkRecord kafkaSinkRecord) {
12511256
*/
12521257
@VisibleForTesting
12531258
protected class StreamingBuffer
1254-
extends PartitionBuffer<Pair<List<Map<String, Object>>, List<Long>>> {
1259+
extends PartitionBuffer<Pair<List<Map<String, Object>>, List<SinkRecord>>> {
12551260
// Records coming from Kafka
12561261
private final List<SinkRecord> sinkRecords;
12571262

@@ -1285,9 +1290,9 @@ public void insert(SinkRecord kafkaSinkRecord) {
12851290
* @return A pair that contains the records and their corresponding offsets
12861291
*/
12871292
@Override
1288-
public Pair<List<Map<String, Object>>, List<Long>> getData() {
1293+
public Pair<List<Map<String, Object>>, List<SinkRecord>> getData() {
12891294
final List<Map<String, Object>> records = new ArrayList<>();
1290-
final List<Long> offsets = new ArrayList<>();
1295+
final List<SinkRecord> filteredOriginalSinkRecords = new ArrayList<>();
12911296
for (SinkRecord kafkaSinkRecord : sinkRecords) {
12921297
SinkRecord snowflakeRecord = getSnowflakeSinkRecordFromKafkaRecord(kafkaSinkRecord);
12931298

@@ -1313,7 +1318,7 @@ public Pair<List<Map<String, Object>>, List<Long>> getData() {
13131318
Map<String, Object> tableRow =
13141319
recordService.getProcessedRecordForStreamingIngest(snowflakeRecord);
13151320
records.add(tableRow);
1316-
offsets.add(snowflakeRecord.kafkaOffset());
1321+
filteredOriginalSinkRecords.add(kafkaSinkRecord);
13171322
} catch (JsonProcessingException e) {
13181323
LOGGER.warn(
13191324
"Record has JsonProcessingException offset:{}, topic:{}",
@@ -1329,7 +1334,7 @@ public Pair<List<Map<String, Object>>, List<Long>> getData() {
13291334
getBufferSizeBytes(),
13301335
getFirstOffset(),
13311336
getLastOffset());
1332-
return new Pair<>(records, offsets);
1337+
return new Pair<>(records, filteredOriginalSinkRecords);
13331338
}
13341339

13351340
@Override

src/test/java/com/snowflake/kafka/connector/internal/TestUtils.java

Lines changed: 31 additions & 7 deletions
Original file line numberDiff line numberDiff line change
@@ -649,17 +649,41 @@ public static List<SinkRecord> createNativeJsonSinkRecords(
649649
final long startOffset,
650650
final long noOfRecords,
651651
final String topicName,
652-
final int partitionNo) {
653-
ArrayList<SinkRecord> records = new ArrayList<>();
652+
final int partitionNo
653+
) {
654+
return createJsonRecords(
655+
startOffset, noOfRecords, topicName, partitionNo,
656+
TestUtils.JSON_WITH_SCHEMA.getBytes(StandardCharsets.UTF_8),
657+
Collections.singletonMap("schemas.enable", Boolean.toString(true))
658+
);
659+
}
660+
661+
/* Generate (noOfRecords - startOffset) blank records for a given topic and partition. */
662+
public static List<SinkRecord> createBlankJsonSinkRecords(
663+
final long startOffset,
664+
final long noOfRecords,
665+
final String topicName,
666+
final int partitionNo
667+
) {
668+
return createJsonRecords(
669+
startOffset, noOfRecords, topicName, partitionNo, null,
670+
Collections.singletonMap("schemas.enable", Boolean.toString(false))
671+
);
672+
}
654673

674+
private static List<SinkRecord> createJsonRecords(
675+
final long startOffset,
676+
final long noOfRecords,
677+
final String topicName,
678+
final int partitionNo,
679+
byte[] value,
680+
Map<String, String> converterConfig
681+
) {
655682
JsonConverter converter = new JsonConverter();
656-
HashMap<String, String> converterConfig = new HashMap<>();
657-
converterConfig.put("schemas.enable", "true");
658683
converter.configure(converterConfig, false);
659-
SchemaAndValue schemaInputValue =
660-
converter.toConnectData(
661-
"test", TestUtils.JSON_WITH_SCHEMA.getBytes(StandardCharsets.UTF_8));
684+
SchemaAndValue schemaInputValue = converter.toConnectData("test", value);
662685

686+
ArrayList<SinkRecord> records = new ArrayList<>();
663687
for (long i = startOffset; i < startOffset + noOfRecords; ++i) {
664688
records.add(
665689
new SinkRecord(

src/test/java/com/snowflake/kafka/connector/internal/streaming/TopicPartitionChannelIT.java

Lines changed: 45 additions & 33 deletions
Original file line numberDiff line numberDiff line change
@@ -709,6 +709,30 @@ public void testChannelMigrateOffsetTokenSystemFunction_NullOffsetTokenInFormatV
709709
service.closeAll();
710710
}
711711

712+
@Test
713+
public void testInsertRowsWithGaps_schematization_largeBufferSize_largeGap() throws Exception {
714+
SnowflakeSinkService service = setupSnowflakeService(true, 4);
715+
716+
// insert blank records that do not evolve schema: 0, 1
717+
List<SinkRecord> records = TestUtils.createBlankJsonSinkRecords(0, 2, topic, PARTITION);
718+
719+
// add records with change in schema with extreme gap in offsets.
720+
records.addAll(TestUtils.createNativeJsonSinkRecords(300, 2, topic, PARTITION));
721+
722+
// records' offsets -> [0, 1, 300, 301]
723+
service.insert(records);
724+
// With schematization, we need to resend a new batch should succeed even if there is an offset
725+
// gap from the previous committed offset
726+
service.insert(records);
727+
728+
TestUtils.assertWithRetry(
729+
() -> service.getOffset(new TopicPartition(topic, PARTITION)) == 302, 10, 10);
730+
731+
assert TestUtils.tableSize(testTableName) == 4
732+
: "expected: " + 4 + " actual: " + TestUtils.tableSize(testTableName);
733+
service.closeAll();
734+
}
735+
712736
@Test
713737
public void testInsertRowsWithGaps_schematization() throws Exception {
714738
testInsertRowsWithGaps(true);
@@ -720,41 +744,10 @@ public void testInsertRowsWithGaps_nonSchematization() throws Exception {
720744
}
721745

722746
private void testInsertRowsWithGaps(boolean withSchematization) throws Exception {
723-
// setup
724-
Map<String, String> config = TestUtils.getConfForStreaming();
725-
SnowflakeSinkConnectorConfig.setDefaultValues(config);
726-
config.put(
727-
SnowflakeSinkConnectorConfig.ENABLE_SCHEMATIZATION_CONFIG,
728-
Boolean.toString(withSchematization));
729-
730-
// create tpChannel
731-
SnowflakeSinkService service =
732-
SnowflakeSinkServiceFactory.builder(conn, IngestionMethodConfig.SNOWPIPE_STREAMING, config)
733-
.setRecordNumber(1)
734-
.setErrorReporter(new InMemoryKafkaRecordErrorReporter())
735-
.setSinkTaskContext(new InMemorySinkTaskContext(Collections.singleton(topicPartition)))
736-
.addTask(testTableName, topicPartition)
737-
.build();
747+
SnowflakeSinkService service = setupSnowflakeService(withSchematization, 1);
738748

739749
// insert blank records that do not evolve schema: 0, 1
740-
JsonConverter converter = new JsonConverter();
741-
HashMap<String, String> converterConfig = new HashMap<>();
742-
converterConfig.put("schemas.enable", "false");
743-
converter.configure(converterConfig, false);
744-
SchemaAndValue schemaInputValue = converter.toConnectData("test", null);
745-
List<SinkRecord> blankRecords = new ArrayList<>();
746-
for (int i = 0; i < 2; i++) {
747-
blankRecords.add(
748-
new SinkRecord(
749-
topic,
750-
PARTITION,
751-
Schema.STRING_SCHEMA,
752-
"test",
753-
schemaInputValue.schema(),
754-
schemaInputValue.value(),
755-
i));
756-
}
757-
750+
List<SinkRecord> blankRecords = TestUtils.createBlankJsonSinkRecords(0, 2, topic, PARTITION);
758751
service.insert(blankRecords);
759752
TestUtils.assertWithRetry(
760753
() -> service.getOffset(new TopicPartition(topic, PARTITION)) == 2, 20, 5);
@@ -777,4 +770,23 @@ private void testInsertRowsWithGaps(boolean withSchematization) throws Exception
777770
: "expected: " + 4 + " actual: " + TestUtils.tableSize(testTableName);
778771
service.closeAll();
779772
}
773+
774+
private SnowflakeSinkService setupSnowflakeService(boolean withSchematization, int recordNumber) {
775+
// setup
776+
Map<String, String> config = TestUtils.getConfForStreaming();
777+
SnowflakeSinkConnectorConfig.setDefaultValues(config);
778+
config.put(
779+
SnowflakeSinkConnectorConfig.ENABLE_SCHEMATIZATION_CONFIG,
780+
Boolean.toString(withSchematization)
781+
);
782+
783+
// create tpChannel
784+
return SnowflakeSinkServiceFactory
785+
.builder(conn, IngestionMethodConfig.SNOWPIPE_STREAMING, config)
786+
.setRecordNumber(recordNumber)
787+
.setErrorReporter(new InMemoryKafkaRecordErrorReporter())
788+
.setSinkTaskContext(new InMemorySinkTaskContext(Collections.singleton(topicPartition)))
789+
.addTask(testTableName, topicPartition)
790+
.build();
791+
}
780792
}

0 commit comments

Comments
 (0)