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; 022 023import java.io.IOException; 024import java.util.Iterator; 025import java.util.Properties; 026 027import cascading.flow.FlowProcess; 028import cascading.scheme.SinkCall; 029import cascading.scheme.SourceCall; 030import cascading.tap.Tap; 031import cascading.tuple.Fields; 032import cascading.tuple.TupleEntry; 033import org.apache.kafka.clients.consumer.ConsumerConfig; 034import org.apache.kafka.clients.consumer.ConsumerRecord; 035import org.apache.kafka.clients.producer.Producer; 036import org.apache.kafka.clients.producer.ProducerConfig; 037import org.apache.kafka.clients.producer.ProducerRecord; 038import org.apache.kafka.common.serialization.StringDeserializer; 039import org.apache.kafka.common.serialization.StringSerializer; 040 041/** 042 * Class TextKafkaScheme is a sub-class of the {@link KafkaScheme} for use with a {@link KafkaTap} instance. 043 * <p> 044 * It consumes and produces text/string based keys and values. 045 * <p> 046 * As a source, it produces seven fields: {@link #TOPIC_FIELDS} typed String, {@link #PARTITION_FIELDS} typed int, 047 * {@link #OFFSET_FIELDS} typed long, 048 * {@link #KEY_FIELDS} typed String, and {@link #VALUE_FIELDS} typed String, 049 * {@link #TIMESTAMP_FIELDS} typed long, {@link #TIMESTAMP_TYPE_FIELDS} typed String 050 * <p> 051 * If alternate source fields are given, any type information will be honored. 052 * <p> 053 * As a sink, the first field encountered will be used as the topic key, and the second field encountered will be 054 * used as the value. 055 */ 056public class TextKafkaScheme extends KafkaScheme<String, String, TextKafkaScheme.Context, TextKafkaScheme.Context> 057 { 058 /** Field TOPIC_FIELDS */ 059 public static final Fields TOPIC_FIELDS = new Fields( "topic", String.class ); 060 /** Field PARTITION_FIELDS */ 061 public static final Fields PARTITION_FIELDS = new Fields( "partition", int.class ); 062 /** Field OFFSET_FIELDS */ 063 public static final Fields OFFSET_FIELDS = new Fields( "offset", long.class ); 064 /** Field KEY_FIELDS */ 065 public static final Fields KEY_FIELDS = new Fields( "key", String.class ); 066 /** Field VALUE_FIELDS */ 067 public static final Fields VALUE_FIELDS = new Fields( "value", String.class ); 068 /** Field TIMESTAMP_FIELDS */ 069 public static final Fields TIMESTAMP_FIELDS = new Fields( "timestamp", long.class ); 070 /** Field TIMESTAMP_TYPE_FIELDS */ 071 public static final Fields TIMESTAMP_TYPE_FIELDS = new Fields( "timestampType", String.class ); 072 /** Field DEFAULT_SOURCE_FIELDS */ 073 public static final Fields DEFAULT_SOURCE_FIELDS = TOPIC_FIELDS.append( PARTITION_FIELDS ).append( OFFSET_FIELDS ).append( KEY_FIELDS ).append( VALUE_FIELDS ).append( TIMESTAMP_FIELDS ).append( TIMESTAMP_TYPE_FIELDS ); 074 075 class Context 076 { 077 String[] topics; 078 079 public Context( String[] topics ) 080 { 081 this.topics = topics; 082 } 083 } 084 085 /** 086 * Constructor TextKafkaScheme creates a new TextKafkaScheme instance. 087 */ 088 public TextKafkaScheme() 089 { 090 super( DEFAULT_SOURCE_FIELDS ); 091 } 092 093 /** 094 * Constructor TextKafkaScheme creates a new TextKafkaScheme instance. 095 * 096 * @param sourceFields of Fields 097 */ 098 public TextKafkaScheme( Fields sourceFields ) 099 { 100 super( sourceFields ); 101 102 if( sourceFields.size() != 7 ) 103 throw new IllegalArgumentException( "wrong number of source fields, requires 6, got: " + sourceFields ); 104 } 105 106 @Override 107 public void sourceConfInit( FlowProcess<? extends Properties> flowProcess, Tap<Properties, KafkaConsumerRecordIterator<String, String>, Producer<String, String>> tap, Properties conf ) 108 { 109 conf.setProperty( ConsumerConfig.KEY_DESERIALIZER_CLASS_CONFIG, StringDeserializer.class.getCanonicalName() ); 110 conf.setProperty( ConsumerConfig.VALUE_DESERIALIZER_CLASS_CONFIG, StringDeserializer.class.getCanonicalName() ); 111 } 112 113 @Override 114 public void sinkConfInit( FlowProcess<? extends Properties> flowProcess, Tap<Properties, KafkaConsumerRecordIterator<String, String>, Producer<String, String>> tap, Properties conf ) 115 { 116 conf.setProperty( ProducerConfig.KEY_SERIALIZER_CLASS_CONFIG, StringSerializer.class.getCanonicalName() ); 117 conf.setProperty( ProducerConfig.VALUE_SERIALIZER_CLASS_CONFIG, StringSerializer.class.getCanonicalName() ); 118 } 119 120 @Override 121 public void sourcePrepare( FlowProcess<? extends Properties> flowProcess, SourceCall<TextKafkaScheme.Context, KafkaConsumerRecordIterator<String, String>> sourceCall ) throws IOException 122 { 123 sourceCall.setContext( new Context( ( (KafkaTap) sourceCall.getTap() ).getTopics() ) ); 124 } 125 126 @Override 127 public void sinkPrepare( FlowProcess<? extends Properties> flowProcess, SinkCall<TextKafkaScheme.Context, Producer<String, String>> sinkCall ) throws IOException 128 { 129 sinkCall.setContext( new Context( ( (KafkaTap) sinkCall.getTap() ).getTopics() ) ); 130 } 131 132 @Override 133 public boolean source( FlowProcess<? extends Properties> flowProcess, SourceCall<TextKafkaScheme.Context, KafkaConsumerRecordIterator<String, String>> sourceCall ) throws IOException 134 { 135 Iterator<ConsumerRecord<String, String>> input = sourceCall.getInput(); 136 137 if( !input.hasNext() ) 138 return false; 139 140 ConsumerRecord<String, String> record = input.next(); 141 TupleEntry incomingEntry = sourceCall.getIncomingEntry(); 142 143 // honor declared type information via #setObject() 144 incomingEntry.setObject( 0, record.topic() ); 145 incomingEntry.setObject( 1, record.partition() ); 146 incomingEntry.setObject( 2, record.offset() ); 147 incomingEntry.setObject( 3, record.key() ); 148 incomingEntry.setObject( 4, record.value() ); 149 incomingEntry.setObject( 5, record.timestamp() ); 150 incomingEntry.setObject( 6, record.timestampType() ); 151 152 return true; 153 } 154 155 @Override 156 public void sink( FlowProcess<? extends Properties> flowProcess, SinkCall<TextKafkaScheme.Context, Producer<String, String>> sinkCall ) throws IOException 157 { 158 // consider tap only providing bytes consumer and going ot byte here 159 String key = sinkCall.getOutgoingEntry().getString( 0 ); 160 String value = sinkCall.getOutgoingEntry().getString( 1 ); 161 162 for( String topic : sinkCall.getContext().topics ) 163 sinkCall.getOutput().send( new ProducerRecord<>( topic, key, value ) ); 164 } 165 }