001/* 002 * Copyright (c) 2007-2022 The Cascading Authors. All Rights Reserved. 003 * 004 * Project and contact information: https://cascading.wensel.net/ 005 * 006 * This file is part of the Cascading project. 007 * 008 * Licensed under the Apache License, Version 2.0 (the "License"); 009 * you may not use this file except in compliance with the License. 010 * You may obtain a copy of the License at 011 * 012 * http://www.apache.org/licenses/LICENSE-2.0 013 * 014 * Unless required by applicable law or agreed to in writing, software 015 * distributed under the License is distributed on an "AS IS" BASIS, 016 * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 017 * See the License for the specific language governing permissions and 018 * limitations under the License. 019 */ 020 021package cascading.local.tap.kafka.commit; 022 023import java.util.Iterator; 024import java.util.Map; 025 026import org.apache.kafka.clients.consumer.ConsumerRecord; 027import org.apache.kafka.clients.consumer.OffsetAndMetadata; 028import org.apache.kafka.common.TopicPartition; 029 030/** 031 * 032 */ 033public class OffsetRecorderIterator<K, V> implements Iterator<ConsumerRecord<K, V>> 034 { 035 private final Map<TopicPartition, OffsetAndMetadata> currentOffsets; 036 private final Iterator<ConsumerRecord<K, V>> iterator; 037 038 private String topic; 039 private int partition; 040 private long offset; 041 042 public OffsetRecorderIterator( Map<TopicPartition, OffsetAndMetadata> currentOffsets, Iterator<ConsumerRecord<K, V>> iterator ) 043 { 044 this.currentOffsets = currentOffsets; 045 this.iterator = iterator; 046 } 047 048 @Override 049 public boolean hasNext() 050 { 051 boolean hasNext = iterator.hasNext(); 052 053 if( !hasNext ) 054 addLastOffset(); 055 056 return hasNext; 057 } 058 059 @Override 060 public ConsumerRecord<K, V> next() 061 { 062 addLastOffset(); // if the next value causes a failure, we will start with it on the next attempt 063 064 ConsumerRecord<K, V> next = iterator.next(); 065 066 topic = next.topic(); 067 partition = next.partition(); 068 offset = next.offset(); 069 070 return next; 071 } 072 073 private void addLastOffset() 074 { 075 if( topic == null ) 076 return; 077 078 currentOffsets.put( new TopicPartition( topic, partition ), new OffsetAndMetadata( offset + 1 ) ); 079 } 080 }