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.Collection; 024import java.util.HashMap; 025import java.util.Map; 026 027import cascading.local.tap.kafka.decorator.ForwardingConsumerRebalanceListener; 028import org.apache.kafka.clients.consumer.Consumer; 029import org.apache.kafka.clients.consumer.ConsumerRebalanceListener; 030import org.apache.kafka.clients.consumer.OffsetAndMetadata; 031import org.apache.kafka.common.TopicPartition; 032 033/** 034 * 035 */ 036public class CommittingRebalanceListener<K, V> extends ForwardingConsumerRebalanceListener 037 { 038 Consumer<K, V> consumer; 039 CommitListener commitListener; 040 Map<TopicPartition, OffsetAndMetadata> currentOffsets; 041 042 public CommittingRebalanceListener( Consumer<K, V> consumer, CommitListener commitListener, Map<TopicPartition, OffsetAndMetadata> currentOffsets ) 043 { 044 this.consumer = consumer; 045 this.commitListener = commitListener; 046 this.currentOffsets = currentOffsets; 047 } 048 049 public CommittingRebalanceListener( ConsumerRebalanceListener listener, Consumer<K, V> consumer, CommitListener commitListener, Map<TopicPartition, OffsetAndMetadata> currentOffsets ) 050 { 051 super( listener ); 052 this.consumer = consumer; 053 this.commitListener = commitListener; 054 this.currentOffsets = currentOffsets; 055 } 056 057 @Override 058 public void onPartitionsRevoked( Collection<TopicPartition> collection ) 059 { 060 super.onPartitionsRevoked( collection ); 061 062 // only commit those revoked and clean up current offsets to prevent them being committed again when 063 // not currently assigned to the consumer 064 Map<TopicPartition, OffsetAndMetadata> revoked = new HashMap<>(); 065 066 for( TopicPartition topicPartition : collection ) 067 { 068 OffsetAndMetadata removed = currentOffsets.remove( topicPartition ); 069 070 if( removed != null ) 071 revoked.put( topicPartition, removed ); 072 } 073 074 commitListener.onRevoke( consumer, revoked ); 075 076 try 077 { 078 consumer.commitSync( revoked ); 079 } 080 catch( RuntimeException exception ) 081 { 082 if( commitListener.onFail( consumer, exception, revoked ) ) 083 throw exception; 084 085 currentOffsets.putAll( revoked ); // return offsets so we can try later 086 } 087 } 088 }