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.Iterator; 026import java.util.Map; 027import java.util.Properties; 028import java.util.concurrent.TimeUnit; 029import java.util.regex.Pattern; 030 031import cascading.local.tap.kafka.decorator.ForwardingConsumer; 032import cascading.local.tap.kafka.decorator.ForwardingConsumerRecords; 033import org.apache.kafka.clients.consumer.Consumer; 034import org.apache.kafka.clients.consumer.ConsumerConfig; 035import org.apache.kafka.clients.consumer.ConsumerRebalanceListener; 036import org.apache.kafka.clients.consumer.ConsumerRecord; 037import org.apache.kafka.clients.consumer.ConsumerRecords; 038import org.apache.kafka.clients.consumer.KafkaConsumer; 039import org.apache.kafka.clients.consumer.OffsetAndMetadata; 040import org.apache.kafka.common.TopicPartition; 041import org.slf4j.Logger; 042import org.slf4j.LoggerFactory; 043 044/** 045 * 046 */ 047public class CommittingConsumer<K, V> extends ForwardingConsumer<K, V> 048 { 049 private static final Logger LOG = LoggerFactory.getLogger( CommittingConsumer.class ); 050 051 Map<TopicPartition, OffsetAndMetadata> currentOffsets = new HashMap<>(); 052 053 CommitListener listener = new CommitListener() 054 { 055 @Override 056 public void onClose( Consumer consumer, Map<TopicPartition, OffsetAndMetadata> offsets ) 057 { 058 LOG.info( "committing offsets on close" ); 059 } 060 061 @Override 062 public void onRevoke( Consumer consumer, Map<TopicPartition, OffsetAndMetadata> offsets ) 063 { 064 LOG.info( "committing offsets on partition revoke" ); 065 } 066 067 @Override 068 public boolean onFail( Consumer consumer, RuntimeException exception, Map<TopicPartition, OffsetAndMetadata> offsets ) 069 { 070 LOG.error( "failed committing offsets", exception ); 071 072 return true; 073 } 074 }; 075 076 public CommittingConsumer( Properties properties, CommitListener listener ) 077 { 078 super( properties ); 079 this.listener = listener; 080 } 081 082 public CommittingConsumer( Properties properties ) 083 { 084 super( properties ); 085 } 086 087 @Override 088 protected KafkaConsumer<K, V> createKafkaConsumerInstance( Properties properties ) 089 { 090 boolean autoCommitEnabled = Boolean.parseBoolean( properties.getProperty( ConsumerConfig.ENABLE_AUTO_COMMIT_CONFIG ) ); 091 092 if( autoCommitEnabled ) 093 LOG.info( "disabling kafka auto-commit" ); 094 095 properties.setProperty( ConsumerConfig.ENABLE_AUTO_COMMIT_CONFIG, "false" ); 096 097 return super.createKafkaConsumerInstance( properties ); 098 } 099 100 @Override 101 public void subscribe( Collection<String> collection ) 102 { 103 super.subscribe( collection, new CommittingRebalanceListener<>( getConsumer(), listener, currentOffsets ) ); 104 } 105 106 @Override 107 public void subscribe( Collection<String> collection, ConsumerRebalanceListener consumerRebalanceListener ) 108 { 109 super.subscribe( collection, new CommittingRebalanceListener<>( consumerRebalanceListener, getConsumer(), listener, currentOffsets ) ); 110 } 111 112 @Override 113 public void subscribe( Pattern pattern, ConsumerRebalanceListener consumerRebalanceListener ) 114 { 115 super.subscribe( pattern, new CommittingRebalanceListener<>( consumerRebalanceListener, getConsumer(), listener, currentOffsets ) ); 116 } 117 118 @Override 119 public ConsumerRecords<K, V> poll( long l ) 120 { 121 return new ForwardingConsumerRecords<K, V>( super.poll( l ) ) 122 { 123 @Override 124 public Iterator<ConsumerRecord<K, V>> iterator() 125 { 126 return new OffsetRecorderIterator<>( currentOffsets, super.iterator() ); 127 } 128 }; 129 } 130 131 @Override 132 public void close() 133 { 134 try 135 { 136 listener.onClose( getConsumer(), currentOffsets ); 137 getConsumer().commitSync( currentOffsets ); 138 } 139 catch( RuntimeException exception ) 140 { 141 if( listener.onFail( getConsumer(), exception, currentOffsets ) ) 142 throw exception; 143 } 144 finally 145 { 146 super.close(); 147 } 148 } 149 150 @Override 151 public void close( long l, TimeUnit timeUnit ) 152 { 153 try 154 { 155 listener.onClose( getConsumer(), currentOffsets ); 156 getConsumer().commitSync( currentOffsets ); 157 } 158 catch( RuntimeException exception ) 159 { 160 if( listener.onFail( getConsumer(), exception, currentOffsets ) ) 161 throw exception; 162 } 163 finally 164 { 165 super.close( l, timeUnit ); 166 } 167 } 168 }