Re: [PR] [FLINK-34470][Connectors/Kafka] Fix indefinite blocking by adjusting stopping condition in split reader [flink-connector-kafka]
dongwoo6kim commented on PR #100: URL: https://github.com/apache/flink-connector-kafka/pull/100#issuecomment-2164206983 Thanks for confirming @morazow, Please feel free to provide any additional advice before merging this fix. It would be also helpful if you could elaborate more on the issue you mentioned and consider adding relevant test code for it. -- This is an automated message from the Apache Git Service. To respond to the message, please log on to GitHub and use the URL above to go to the specific comment. To unsubscribe, e-mail: issues-unsubscr...@flink.apache.org For queries about this service, please contact Infrastructure at: us...@infra.apache.org
Re: [PR] [FLINK-34470][Connectors/Kafka] Fix indefinite blocking by adjusting stopping condition in split reader [flink-connector-kafka]
morazow commented on PR #100: URL: https://github.com/apache/flink-connector-kafka/pull/100#issuecomment-2159296755 Thanks @dongwoo6kim , Tests looks good from my side (_Recently I faced similar issue which maybe related, when running batch mode with setting `startingOffsets`. The change should solve that issue. But we may create issue for it_) -- This is an automated message from the Apache Git Service. To respond to the message, please log on to GitHub and use the URL above to go to the specific comment. To unsubscribe, e-mail: issues-unsubscr...@flink.apache.org For queries about this service, please contact Infrastructure at: us...@infra.apache.org
Re: [PR] [FLINK-34470][Connectors/Kafka] Fix indefinite blocking by adjusting stopping condition in split reader [flink-connector-kafka]
dongwoo6kim commented on PR #100: URL: https://github.com/apache/flink-connector-kafka/pull/100#issuecomment-2143497305 Hello @morazow I've added ITCase for this case. This case fails for current main branch due to timeout and works okay with fixed code. -- This is an automated message from the Apache Git Service. To respond to the message, please log on to GitHub and use the URL above to go to the specific comment. To unsubscribe, e-mail: issues-unsubscr...@flink.apache.org For queries about this service, please contact Infrastructure at: us...@infra.apache.org
Re: [PR] [FLINK-34470][Connectors/Kafka] Fix indefinite blocking by adjusting stopping condition in split reader [flink-connector-kafka]
dongwoo6kim commented on code in PR #100: URL: https://github.com/apache/flink-connector-kafka/pull/100#discussion_r1601015487 ## flink-connector-kafka/src/main/java/org/apache/flink/connector/kafka/source/reader/KafkaPartitionSplitReader.java: ## @@ -122,27 +122,21 @@ public RecordsWithSplitIds> fetch() throws IOExce KafkaPartitionSplitRecords recordsBySplits = new KafkaPartitionSplitRecords(consumerRecords, kafkaSourceReaderMetrics); List finishedPartitions = new ArrayList<>(); -for (TopicPartition tp : consumerRecords.partitions()) { +for (TopicPartition tp : consumer.assignment()) { long stoppingOffset = getStoppingOffset(tp); -final List> recordsFromPartition = -consumerRecords.records(tp); - -if (recordsFromPartition.size() > 0) { -final ConsumerRecord lastRecord = -recordsFromPartition.get(recordsFromPartition.size() - 1); - -// After processing a record with offset of "stoppingOffset - 1", the split reader -// should not continue fetching because the record with stoppingOffset may not -// exist. Keep polling will just block forever. -if (lastRecord.offset() >= stoppingOffset - 1) { -recordsBySplits.setPartitionStoppingOffset(tp, stoppingOffset); -finishSplitAtRecord( -tp, -stoppingOffset, -lastRecord.offset(), -finishedPartitions, -recordsBySplits); -} +long consumerPosition = consumer.position(tp); +// Stop fetching when the consumer's position reaches the stoppingOffset. +// Control messages may follow the last record; therefore, using the last record's +// offset as a stopping condition could result in indefinite blocking. +if (consumerPosition >= stoppingOffset) { +LOG.debug( +"Position of {}: {}, has reached stopping offset: {}", +tp, +consumerPosition, +stoppingOffset); +recordsBySplits.setPartitionStoppingOffset(tp, stoppingOffset); +finishSplitAtRecord( +tp, stoppingOffset, consumerPosition, finishedPartitions, recordsBySplits); } // Track this partition's record lag if it never appears before kafkaSourceReaderMetrics.maybeAddRecordsLagMetric(consumer, tp); Review Comment: @LinMingQiang, thanks for the review. I've changed to track tp, only when there is record for that tp. -- This is an automated message from the Apache Git Service. To respond to the message, please log on to GitHub and use the URL above to go to the specific comment. To unsubscribe, e-mail: issues-unsubscr...@flink.apache.org For queries about this service, please contact Infrastructure at: us...@infra.apache.org
Re: [PR] [FLINK-34470][Connectors/Kafka] Fix indefinite blocking by adjusting stopping condition in split reader [flink-connector-kafka]
dongwoo6kim commented on code in PR #100: URL: https://github.com/apache/flink-connector-kafka/pull/100#discussion_r1601015487 ## flink-connector-kafka/src/main/java/org/apache/flink/connector/kafka/source/reader/KafkaPartitionSplitReader.java: ## @@ -122,27 +122,21 @@ public RecordsWithSplitIds> fetch() throws IOExce KafkaPartitionSplitRecords recordsBySplits = new KafkaPartitionSplitRecords(consumerRecords, kafkaSourceReaderMetrics); List finishedPartitions = new ArrayList<>(); -for (TopicPartition tp : consumerRecords.partitions()) { +for (TopicPartition tp : consumer.assignment()) { long stoppingOffset = getStoppingOffset(tp); -final List> recordsFromPartition = -consumerRecords.records(tp); - -if (recordsFromPartition.size() > 0) { -final ConsumerRecord lastRecord = -recordsFromPartition.get(recordsFromPartition.size() - 1); - -// After processing a record with offset of "stoppingOffset - 1", the split reader -// should not continue fetching because the record with stoppingOffset may not -// exist. Keep polling will just block forever. -if (lastRecord.offset() >= stoppingOffset - 1) { -recordsBySplits.setPartitionStoppingOffset(tp, stoppingOffset); -finishSplitAtRecord( -tp, -stoppingOffset, -lastRecord.offset(), -finishedPartitions, -recordsBySplits); -} +long consumerPosition = consumer.position(tp); +// Stop fetching when the consumer's position reaches the stoppingOffset. +// Control messages may follow the last record; therefore, using the last record's +// offset as a stopping condition could result in indefinite blocking. +if (consumerPosition >= stoppingOffset) { +LOG.debug( +"Position of {}: {}, has reached stopping offset: {}", +tp, +consumerPosition, +stoppingOffset); +recordsBySplits.setPartitionStoppingOffset(tp, stoppingOffset); +finishSplitAtRecord( +tp, stoppingOffset, consumerPosition, finishedPartitions, recordsBySplits); } // Track this partition's record lag if it never appears before kafkaSourceReaderMetrics.maybeAddRecordsLagMetric(consumer, tp); Review Comment: @LinMingQiang, thanks for the review. I've changed to track tp, only when records for that tp is not empty. -- This is an automated message from the Apache Git Service. To respond to the message, please log on to GitHub and use the URL above to go to the specific comment. To unsubscribe, e-mail: issues-unsubscr...@flink.apache.org For queries about this service, please contact Infrastructure at: us...@infra.apache.org
Re: [PR] [FLINK-34470][Connectors/Kafka] Fix indefinite blocking by adjusting stopping condition in split reader [flink-connector-kafka]
LinMingQiang commented on code in PR #100: URL: https://github.com/apache/flink-connector-kafka/pull/100#discussion_r1600879571 ## flink-connector-kafka/src/main/java/org/apache/flink/connector/kafka/source/reader/KafkaPartitionSplitReader.java: ## @@ -122,27 +122,21 @@ public RecordsWithSplitIds> fetch() throws IOExce KafkaPartitionSplitRecords recordsBySplits = new KafkaPartitionSplitRecords(consumerRecords, kafkaSourceReaderMetrics); List finishedPartitions = new ArrayList<>(); -for (TopicPartition tp : consumerRecords.partitions()) { +for (TopicPartition tp : consumer.assignment()) { long stoppingOffset = getStoppingOffset(tp); -final List> recordsFromPartition = -consumerRecords.records(tp); - -if (recordsFromPartition.size() > 0) { -final ConsumerRecord lastRecord = -recordsFromPartition.get(recordsFromPartition.size() - 1); - -// After processing a record with offset of "stoppingOffset - 1", the split reader -// should not continue fetching because the record with stoppingOffset may not -// exist. Keep polling will just block forever. -if (lastRecord.offset() >= stoppingOffset - 1) { -recordsBySplits.setPartitionStoppingOffset(tp, stoppingOffset); -finishSplitAtRecord( -tp, -stoppingOffset, -lastRecord.offset(), -finishedPartitions, -recordsBySplits); -} +long consumerPosition = consumer.position(tp); +// Stop fetching when the consumer's position reaches the stoppingOffset. +// Control messages may follow the last record; therefore, using the last record's +// offset as a stopping condition could result in indefinite blocking. +if (consumerPosition >= stoppingOffset) { +LOG.debug( +"Position of {}: {}, has reached stopping offset: {}", +tp, +consumerPosition, +stoppingOffset); +recordsBySplits.setPartitionStoppingOffset(tp, stoppingOffset); +finishSplitAtRecord( +tp, stoppingOffset, consumerPosition, finishedPartitions, recordsBySplits); } // Track this partition's record lag if it never appears before kafkaSourceReaderMetrics.maybeAddRecordsLagMetric(consumer, tp); Review Comment: we do not need to track tp when consumerRecords is empty. -- This is an automated message from the Apache Git Service. To respond to the message, please log on to GitHub and use the URL above to go to the specific comment. To unsubscribe, e-mail: issues-unsubscr...@flink.apache.org For queries about this service, please contact Infrastructure at: us...@infra.apache.org
Re: [PR] [FLINK-34470][Connectors/Kafka] Fix indefinite blocking by adjusting stopping condition in split reader [flink-connector-kafka]
LinMingQiang commented on code in PR #100: URL: https://github.com/apache/flink-connector-kafka/pull/100#discussion_r1600878564 ## flink-connector-kafka/src/main/java/org/apache/flink/connector/kafka/source/reader/KafkaPartitionSplitReader.java: ## @@ -122,27 +122,21 @@ public RecordsWithSplitIds> fetch() throws IOExce KafkaPartitionSplitRecords recordsBySplits = new KafkaPartitionSplitRecords(consumerRecords, kafkaSourceReaderMetrics); List finishedPartitions = new ArrayList<>(); -for (TopicPartition tp : consumerRecords.partitions()) { +for (TopicPartition tp : consumer.assignment()) { long stoppingOffset = getStoppingOffset(tp); -final List> recordsFromPartition = -consumerRecords.records(tp); - -if (recordsFromPartition.size() > 0) { -final ConsumerRecord lastRecord = -recordsFromPartition.get(recordsFromPartition.size() - 1); - -// After processing a record with offset of "stoppingOffset - 1", the split reader -// should not continue fetching because the record with stoppingOffset may not -// exist. Keep polling will just block forever. -if (lastRecord.offset() >= stoppingOffset - 1) { -recordsBySplits.setPartitionStoppingOffset(tp, stoppingOffset); -finishSplitAtRecord( -tp, -stoppingOffset, -lastRecord.offset(), -finishedPartitions, -recordsBySplits); -} +long consumerPosition = consumer.position(tp); +// Stop fetching when the consumer's position reaches the stoppingOffset. +// Control messages may follow the last record; therefore, using the last record's +// offset as a stopping condition could result in indefinite blocking. +if (consumerPosition >= stoppingOffset) { +LOG.debug( +"Position of {}: {}, has reached stopping offset: {}", +tp, +consumerPosition, +stoppingOffset); +recordsBySplits.setPartitionStoppingOffset(tp, stoppingOffset); +finishSplitAtRecord( +tp, stoppingOffset, consumerPosition, finishedPartitions, recordsBySplits); } // Track this partition's record lag if it never appears before kafkaSourceReaderMetrics.maybeAddRecordsLagMetric(consumer, tp); Review Comment: consumerRecords.partitions().forEach(trackTp -> { // Track this partition's record lag if it never appears before kafkaSourceReaderMetrics.maybeAddRecordsLagMetric(consumer, trackTp); }); -- This is an automated message from the Apache Git Service. To respond to the message, please log on to GitHub and use the URL above to go to the specific comment. To unsubscribe, e-mail: issues-unsubscr...@flink.apache.org For queries about this service, please contact Infrastructure at: us...@infra.apache.org
Re: [PR] [FLINK-34470][Connectors/Kafka] Fix indefinite blocking by adjusting stopping condition in split reader [flink-connector-kafka]
LinMingQiang commented on PR #100: URL: https://github.com/apache/flink-connector-kafka/pull/100#issuecomment-2111454911 its work , i had try. -- This is an automated message from the Apache Git Service. To respond to the message, please log on to GitHub and use the URL above to go to the specific comment. To unsubscribe, e-mail: issues-unsubscr...@flink.apache.org For queries about this service, please contact Infrastructure at: us...@infra.apache.org
Re: [PR] [FLINK-34470][Connectors/Kafka] Fix indefinite blocking by adjusting stopping condition in split reader [flink-connector-kafka]
boring-cyborg[bot] commented on PR #100: URL: https://github.com/apache/flink-connector-kafka/pull/100#issuecomment-2081822026 Thanks for opening this pull request! Please check out our contributing guidelines. (https://flink.apache.org/contributing/how-to-contribute.html) -- This is an automated message from the Apache Git Service. To respond to the message, please log on to GitHub and use the URL above to go to the specific comment. To unsubscribe, e-mail: issues-unsubscr...@flink.apache.org For queries about this service, please contact Infrastructure at: us...@infra.apache.org