001/*
002 * Copyright (c) 2017-2019 Chris K Wensel <chris@wensel.net>. All Rights Reserved.
003 *
004 * Project and contact information: http://www.cascading.org/
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 six fields: {@link #TOPIC_FIELDS} typed String, {@link #OFFSET_FIELDS} typed long,
047 * {@link #KEY_FIELDS} typed String, and {@link #VALUE_FIELDS} typed String,
048 * {@link #TIMESTAMP_FIELDS} typed long, {@link #TIMESTAMP_TYPE_FIELDS} typed String
049 * <p>
050 * If alternate source fields are given, any type information will be honored.
051 * <p>
052 * As a sink, the first field encountered will be used as the topic key, and the second field encountered will be
053 * used as the value.
054 */
055public class TextKafkaScheme extends KafkaScheme<String, String, TextKafkaScheme.Context, TextKafkaScheme.Context>
056  {
057  /** Field TOPIC_FIELDS */
058  public static final Fields TOPIC_FIELDS = new Fields( "topic", String.class );
059  /** Field OFFSET_FIELDS */
060  public static final Fields OFFSET_FIELDS = new Fields( "offset", long.class );
061  /** Field KEY_FIELDS */
062  public static final Fields KEY_FIELDS = new Fields( "key", String.class );
063  /** Field VALUE_FIELDS */
064  public static final Fields VALUE_FIELDS = new Fields( "value", String.class );
065  /** Field TIMESTAMP_FIELDS */
066  public static final Fields TIMESTAMP_FIELDS = new Fields( "timestamp", long.class );
067  /** Field TIMESTAMP_TYPE_FIELDS */
068  public static final Fields TIMESTAMP_TYPE_FIELDS = new Fields( "timestampType", String.class );
069  /** Field DEFAULT_SOURCE_FIELDS */
070  public static final Fields DEFAULT_SOURCE_FIELDS = TOPIC_FIELDS.append( OFFSET_FIELDS ).append( KEY_FIELDS ).append( VALUE_FIELDS ).append( TIMESTAMP_FIELDS ).append( TIMESTAMP_TYPE_FIELDS );
071
072  class Context
073    {
074    String[] topics;
075
076    public Context( String[] topics )
077      {
078      this.topics = topics;
079      }
080    }
081
082  /**
083   * Constructor TextKafkaScheme creates a new TextKafkaScheme instance.
084   */
085  public TextKafkaScheme()
086    {
087    super( DEFAULT_SOURCE_FIELDS );
088    }
089
090  /**
091   * Constructor TextKafkaScheme creates a new TextKafkaScheme instance.
092   *
093   * @param sourceFields of Fields
094   */
095  public TextKafkaScheme( Fields sourceFields )
096    {
097    super( sourceFields );
098
099    if( sourceFields.size() != 6 )
100      throw new IllegalArgumentException( "wrong number of source fields, requires 6, got: " + sourceFields );
101    }
102
103  @Override
104  public void sourceConfInit( FlowProcess<? extends Properties> flowProcess, Tap<Properties, Iterator<ConsumerRecord<String, String>>, Producer<String, String>> tap, Properties conf )
105    {
106    conf.setProperty( ConsumerConfig.KEY_DESERIALIZER_CLASS_CONFIG, StringDeserializer.class.getCanonicalName() );
107    conf.setProperty( ConsumerConfig.VALUE_DESERIALIZER_CLASS_CONFIG, StringDeserializer.class.getCanonicalName() );
108    }
109
110  @Override
111  public void sinkConfInit( FlowProcess<? extends Properties> flowProcess, Tap<Properties, Iterator<ConsumerRecord<String, String>>, Producer<String, String>> tap, Properties conf )
112    {
113    conf.setProperty( ProducerConfig.KEY_SERIALIZER_CLASS_CONFIG, StringSerializer.class.getCanonicalName() );
114    conf.setProperty( ProducerConfig.VALUE_SERIALIZER_CLASS_CONFIG, StringSerializer.class.getCanonicalName() );
115    }
116
117  @Override
118  public void sourcePrepare( FlowProcess<? extends Properties> flowProcess, SourceCall<TextKafkaScheme.Context, Iterator<ConsumerRecord<String, String>>> sourceCall ) throws IOException
119    {
120    sourceCall.setContext( new Context( ( (KafkaTap) sourceCall.getTap() ).getTopics() ) );
121    }
122
123  @Override
124  public void sinkPrepare( FlowProcess<? extends Properties> flowProcess, SinkCall<TextKafkaScheme.Context, Producer<String, String>> sinkCall ) throws IOException
125    {
126    sinkCall.setContext( new Context( ( (KafkaTap) sinkCall.getTap() ).getTopics() ) );
127    }
128
129  @Override
130  public boolean source( FlowProcess<? extends Properties> flowProcess, SourceCall<TextKafkaScheme.Context, Iterator<ConsumerRecord<String, String>>> sourceCall ) throws IOException
131    {
132    Iterator<ConsumerRecord<String, String>> input = sourceCall.getInput();
133
134    if( !input.hasNext() )
135      return false;
136
137    ConsumerRecord<String, String> record = input.next();
138    TupleEntry incomingEntry = sourceCall.getIncomingEntry();
139
140    // honor declared type information via #setObject()
141    incomingEntry.setObject( 0, record.topic() );
142    incomingEntry.setObject( 1, record.offset() );
143    incomingEntry.setObject( 2, record.key() );
144    incomingEntry.setObject( 3, record.value() );
145    incomingEntry.setObject( 4, record.timestamp() );
146    incomingEntry.setObject( 5, record.timestampType() );
147
148    return true;
149    }
150
151  @Override
152  public void sink( FlowProcess<? extends Properties> flowProcess, SinkCall<TextKafkaScheme.Context, Producer<String, String>> sinkCall ) throws IOException
153    {
154    // consider tap only providing bytes consumer and going ot byte here
155    String key = sinkCall.getOutgoingEntry().getString( 0 );
156    String value = sinkCall.getOutgoingEntry().getString( 1 );
157
158    for( String topic : sinkCall.getContext().topics )
159      sinkCall.getOutput().send( new ProducerRecord<>( topic, key, value ) );
160    }
161  }