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.net.URI; 025import java.net.URISyntaxException; 026import java.util.ArrayList; 027import java.util.Arrays; 028import java.util.Iterator; 029import java.util.List; 030import java.util.Map; 031import java.util.NoSuchElementException; 032import java.util.Properties; 033import java.util.Set; 034import java.util.concurrent.ExecutionException; 035import java.util.function.Supplier; 036import java.util.regex.Pattern; 037 038import cascading.flow.FlowProcess; 039import cascading.local.tap.kafka.decorator.ForwardingConsumer; 040import cascading.property.PropertyUtil; 041import cascading.scheme.SourceCall; 042import cascading.tap.SinkMode; 043import cascading.tap.Tap; 044import cascading.tuple.TupleEntryCollector; 045import cascading.tuple.TupleEntryIterator; 046import cascading.tuple.TupleEntrySchemeCollector; 047import cascading.tuple.TupleEntrySchemeIterator; 048import cascading.util.CloseableIterator; 049import cascading.util.Util; 050import org.apache.kafka.clients.admin.AdminClient; 051import org.apache.kafka.clients.admin.CreateTopicsResult; 052import org.apache.kafka.clients.admin.DeleteTopicsResult; 053import org.apache.kafka.clients.admin.DescribeTopicsResult; 054import org.apache.kafka.clients.admin.NewTopic; 055import org.apache.kafka.clients.admin.TopicDescription; 056import org.apache.kafka.clients.consumer.Consumer; 057import org.apache.kafka.clients.consumer.ConsumerConfig; 058import org.apache.kafka.clients.consumer.ConsumerRebalanceListener; 059import org.apache.kafka.clients.consumer.ConsumerRecord; 060import org.apache.kafka.clients.consumer.ConsumerRecords; 061import org.apache.kafka.clients.consumer.internals.NoOpConsumerRebalanceListener; 062import org.apache.kafka.clients.producer.KafkaProducer; 063import org.apache.kafka.clients.producer.Producer; 064import org.apache.kafka.clients.producer.ProducerConfig; 065import org.apache.kafka.common.KafkaFuture; 066import org.slf4j.Logger; 067import org.slf4j.LoggerFactory; 068 069/** 070 * Class KafkaTap is a Cascading local mode Tap providing read and write access to data stored in Apache Kafka topics. 071 * <p> 072 * This Tap is not intended to be used with any of the other Cascading planners unless they specify they are local mode 073 * compatible. 074 * <p> 075 * The Kafka producer and consumer interfaces are not Input/OutputStream based, subsequently the 076 * KafkaTap must be used with a {@link KafkaScheme} sub-class which encapsulates the mechanics of reading/writing 077 * from/to a Kafka topic. 078 * <p> 079 * Further Kafka tends to treat a topic as data vs a location, causing further coupling between the KafkaTap and 080 * KafkaScheme. This may be Kafka's actual intent, so this class may need to be revised, or an alternate implementation 081 * implemented. 082 * <p> 083 * Subsequently, the KafkaTap can be instantiated with a {@link URI} instance of the format: 084 * {@code kafka://[hostname]<:port>/[topic]<,topic>}. 085 * <p> 086 * Where hostname and at least one topic is required. 087 * <p> 088 * If the first topic is wrapped by slash ({@code /}), e.g. {@code /some-.*-topic/}, the string within the slashes 089 * will be considered a regular-expression. Any subsequent topics will be ignored. 090 * <p> 091 * Note on read, the KafkaTap will continue to retrieve data until the {@code pollTimeout} is reached, where the 092 * default is 10 seconds. 093 * <p> 094 * Use the {@code defaultProperties} argument to set Kafka Consumer/Producer specific properties. 095 * <p> 096 * By default, {@link #CONSUME_AUTO_COMMIT_EARLIEST} with values: 097 * <ul> 098 * <li>{@code ConsumerConfig.ENABLE_AUTO_COMMIT_CONFIG} is {@code true }</li> 099 * <li>{@code ConsumerConfig.AUTO_COMMIT_INTERVAL_MS_CONFIG} is {@code 1000 }</li> 100 * <li>{@code ConsumerConfig.AUTO_OFFSET_RESET_CONFIG} is {@code "earliest" }</li> 101 * </ul> 102 * <p> 103 * and {@link #PRODUCE_ACK_ALL_NO_RETRY} with values: 104 * <ul> 105 * <li>{@code ProducerConfig.ACKS_CONFIG} is {@code "all }</li> 106 * <li>{@code ProducerConfig.RETRIES_CONFIG} is {@code 0 }</li> 107 * </ul> 108 * Note, calling @{code close()} on the ConsumerRecord Iterator from the {@link KafkaScheme#source(FlowProcess, SourceCall)} 109 * method will close the Kafka consumer and notify the parent {@link TupleEntrySchemeIterator} to stop providing 110 * new values. 111 * 112 * @see #CONSUME_AUTO_COMMIT_EARLIEST 113 * @see #CONSUME_AUTO_COMMIT_LATEST 114 * @see #PRODUCE_ACK_ALL_NO_RETRY 115 * @see PropertyUtil#merge(Properties...) for conveniently merging Property instances 116 */ 117public class KafkaTap<K, V> extends Tap<Properties, KafkaConsumerRecordIterator<K, V>, Producer<K, V>> 118 { 119 /** Field LOG */ 120 private static final Logger LOG = LoggerFactory.getLogger( KafkaTap.class ); 121 122 /** Field CONSUME_AUTO_COMMIT_LATEST */ 123 public static final Properties CONSUME_AUTO_COMMIT_LATEST = new Properties() 124 { 125 { 126 setProperty( ConsumerConfig.ENABLE_AUTO_COMMIT_CONFIG, "true" ); 127 setProperty( ConsumerConfig.AUTO_COMMIT_INTERVAL_MS_CONFIG, "1000" ); 128 setProperty( ConsumerConfig.AUTO_OFFSET_RESET_CONFIG, "latest" ); 129 } 130 }; 131 132 /** Field CONSUME_AUTO_COMMIT_EARLIEST */ 133 public static final Properties CONSUME_AUTO_COMMIT_EARLIEST = new Properties() 134 { 135 { 136 setProperty( ConsumerConfig.ENABLE_AUTO_COMMIT_CONFIG, "true" ); 137 setProperty( ConsumerConfig.AUTO_COMMIT_INTERVAL_MS_CONFIG, "1000" ); 138 setProperty( ConsumerConfig.AUTO_OFFSET_RESET_CONFIG, "earliest" ); 139 } 140 }; 141 142 /** Field PRODUCE_ACK_ALL_NO_RETRY */ 143 public static final Properties PRODUCE_ACK_ALL_NO_RETRY = new Properties() 144 { 145 { 146 setProperty( ProducerConfig.ACKS_CONFIG, "all" ); 147 setProperty( ProducerConfig.RETRIES_CONFIG, "0" ); 148 } 149 }; 150 151 /** Field DEFAULT_POLL_TIMEOUT */ 152 public static final long DEFAULT_POLL_TIMEOUT = 10_000L; 153 /** Field DEFAULT_REPLICATION_FACTOR */ 154 public static final short DEFAULT_REPLICATION_FACTOR = 1; 155 /** Field DEFAULT_NUM_PARTITIONS */ 156 public static final int DEFAULT_NUM_PARTITIONS = 1; 157 158 /** Field defaultProperties */ 159 Properties defaultProperties = PropertyUtil.merge( CONSUME_AUTO_COMMIT_EARLIEST, PRODUCE_ACK_ALL_NO_RETRY ); 160 /** Field hostname */ 161 String hostname; 162 /** Field topics */ 163 String[] topics; 164 /** Field topicIsPattern */ 165 boolean isTopicPattern = false; 166 /** Field numPartitions */ 167 int numPartitions = DEFAULT_NUM_PARTITIONS; 168 /** Field replicationFactor */ 169 short replicationFactor = DEFAULT_REPLICATION_FACTOR; 170 /** Field clientID */ 171 String clientID = null; 172 /** Field groupID */ 173 String groupID = Tap.id( this ); 174 /** Field pollTimeout */ 175 long pollTimeout = DEFAULT_POLL_TIMEOUT; 176 177 /** 178 * Method makeURI creates a kafka URI for use with the KafkaTap. 179 * 180 * @param hostname hostname and optionally port information to connect too 181 * @param topics one more topics to connect too 182 * @return a URI instance 183 */ 184 public static URI makeURI( String hostname, String... topics ) 185 { 186 if( hostname == null ) 187 throw new IllegalArgumentException( "hostname may not be null" ); 188 189 Arrays.sort( topics ); 190 191 try 192 { 193 return new URI( "kafka", hostname, "/" + Util.join( ",", topics ), null, null ); 194 } 195 catch( URISyntaxException exception ) 196 { 197 throw new IllegalArgumentException( exception.getMessage(), exception ); 198 } 199 } 200 201 /** 202 * Constructor KafkaTap creates a new KafkaTap instance. 203 * 204 * @param defaultProperties of Properties 205 * @param scheme of KafkaScheme 206 * @param identifier of URI 207 */ 208 public KafkaTap( Properties defaultProperties, KafkaScheme<K, V, ?, ?> scheme, URI identifier ) 209 { 210 this( defaultProperties, scheme, identifier, DEFAULT_POLL_TIMEOUT, DEFAULT_NUM_PARTITIONS, DEFAULT_REPLICATION_FACTOR ); 211 } 212 213 /** 214 * Constructor KafkaTap creates a new KafkaTap instance. 215 * 216 * @param scheme of KafkaScheme 217 * @param identifier of URI 218 * @param pollTimeout of long 219 */ 220 public KafkaTap( KafkaScheme<K, V, ?, ?> scheme, URI identifier, long pollTimeout ) 221 { 222 this( scheme, identifier, pollTimeout, DEFAULT_NUM_PARTITIONS, DEFAULT_REPLICATION_FACTOR ); 223 } 224 225 /** 226 * Constructor KafkaTap creates a new KafkaTap instance. 227 * 228 * @param scheme of KafkaScheme 229 * @param identifier of URI 230 * @param numPartitions of int 231 * @param replicationFactor of short 232 */ 233 public KafkaTap( KafkaScheme<K, V, ?, ?> scheme, URI identifier, int numPartitions, short replicationFactor ) 234 { 235 this( scheme, identifier, DEFAULT_POLL_TIMEOUT, numPartitions, replicationFactor ); 236 } 237 238 /** 239 * Constructor KafkaTap creates a new KafkaTap instance. 240 * 241 * @param scheme of KafkaScheme 242 * @param identifier of URI 243 * @param pollTimeout of int 244 * @param numPartitions of int 245 * @param replicationFactor of short 246 */ 247 public KafkaTap( KafkaScheme<K, V, ?, ?> scheme, URI identifier, long pollTimeout, int numPartitions, short replicationFactor ) 248 { 249 this( null, scheme, identifier, pollTimeout, numPartitions, replicationFactor ); 250 } 251 252 /** 253 * Constructor KafkaTap creates a new KafkaTap instance. 254 * 255 * @param scheme of KafkaScheme 256 * @param identifier of URI 257 */ 258 public KafkaTap( KafkaScheme<K, V, ?, ?> scheme, URI identifier ) 259 { 260 this( scheme, identifier, DEFAULT_POLL_TIMEOUT, DEFAULT_NUM_PARTITIONS, DEFAULT_REPLICATION_FACTOR ); 261 } 262 263 /** 264 * Constructor KafkaTap creates a new KafkaTap instance. 265 * 266 * @param defaultProperties of Properties 267 * @param scheme of KafkaScheme 268 * @param identifier of URI 269 * @param pollTimeout of int 270 */ 271 public KafkaTap( Properties defaultProperties, KafkaScheme<K, V, ?, ?> scheme, URI identifier, long pollTimeout ) 272 { 273 this( defaultProperties, scheme, identifier, pollTimeout, DEFAULT_NUM_PARTITIONS, DEFAULT_REPLICATION_FACTOR ); 274 } 275 276 /** 277 * Constructor KafkaTap creates a new KafkaTap instance. 278 * 279 * @param defaultProperties of Properties 280 * @param scheme of KafkaScheme 281 * @param identifier of URI 282 * @param numPartitions of int 283 * @param replicationFactor of short 284 */ 285 public KafkaTap( Properties defaultProperties, KafkaScheme<K, V, ?, ?> scheme, URI identifier, int numPartitions, short replicationFactor ) 286 { 287 this( defaultProperties, scheme, identifier, DEFAULT_POLL_TIMEOUT, numPartitions, replicationFactor ); 288 } 289 290 /** 291 * Constructor KafkaTap creates a new KafkaTap instance. 292 * 293 * @param defaultProperties of Properties 294 * @param scheme of KafkaScheme 295 * @param identifier of URI 296 * @param pollTimeout of int 297 * @param numPartitions of int 298 * @param replicationFactor of short 299 */ 300 public KafkaTap( Properties defaultProperties, KafkaScheme<K, V, ?, ?> scheme, URI identifier, long pollTimeout, int numPartitions, short replicationFactor ) 301 { 302 this( defaultProperties, scheme, identifier, null, pollTimeout, numPartitions, replicationFactor ); 303 } 304 305 /** 306 * Constructor KafkaTap creates a new KafkaTap instance. 307 * 308 * @param defaultProperties of Properties 309 * @param scheme of KafkaScheme 310 * @param identifier of URI 311 * @param clientID of String 312 */ 313 public KafkaTap( Properties defaultProperties, KafkaScheme<K, V, ?, ?> scheme, URI identifier, String clientID ) 314 { 315 this( defaultProperties, scheme, identifier, clientID, DEFAULT_POLL_TIMEOUT, DEFAULT_NUM_PARTITIONS, DEFAULT_REPLICATION_FACTOR ); 316 } 317 318 /** 319 * Constructor KafkaTap creates a new KafkaTap instance. 320 * 321 * @param defaultProperties of Properties 322 * @param scheme of KafkaScheme 323 * @param identifier of URI 324 * @param clientID of String 325 * @param groupID of String 326 */ 327 public KafkaTap( Properties defaultProperties, KafkaScheme<K, V, ?, ?> scheme, URI identifier, String clientID, String groupID ) 328 { 329 this( defaultProperties, scheme, identifier, clientID, groupID, DEFAULT_POLL_TIMEOUT, DEFAULT_NUM_PARTITIONS, DEFAULT_REPLICATION_FACTOR ); 330 } 331 332 /** 333 * Constructor KafkaTap creates a new KafkaTap instance. 334 * 335 * @param scheme of KafkaScheme 336 * @param identifier of URI 337 * @param clientID of String 338 * @param pollTimeout of long 339 */ 340 public KafkaTap( KafkaScheme<K, V, ?, ?> scheme, URI identifier, String clientID, long pollTimeout ) 341 { 342 this( scheme, identifier, clientID, pollTimeout, DEFAULT_NUM_PARTITIONS, DEFAULT_REPLICATION_FACTOR ); 343 } 344 345 /** 346 * Constructor KafkaTap creates a new KafkaTap instance. 347 * 348 * @param scheme of KafkaScheme 349 * @param identifier of URI 350 * @param clientID of String 351 * @param numPartitions of int 352 * @param replicationFactor of short 353 */ 354 public KafkaTap( KafkaScheme<K, V, ?, ?> scheme, URI identifier, String clientID, int numPartitions, short replicationFactor ) 355 { 356 this( scheme, identifier, clientID, DEFAULT_POLL_TIMEOUT, numPartitions, replicationFactor ); 357 } 358 359 /** 360 * Constructor KafkaTap creates a new KafkaTap instance. 361 * 362 * @param scheme of KafkaScheme 363 * @param identifier of URI 364 * @param clientID of String 365 * @param pollTimeout of int 366 * @param numPartitions of int 367 * @param replicationFactor of short 368 */ 369 public KafkaTap( KafkaScheme<K, V, ?, ?> scheme, URI identifier, String clientID, long pollTimeout, int numPartitions, short replicationFactor ) 370 { 371 this( null, scheme, identifier, clientID, pollTimeout, numPartitions, replicationFactor ); 372 } 373 374 /** 375 * Constructor KafkaTap creates a new KafkaTap instance. 376 * 377 * @param scheme of KafkaScheme 378 * @param identifier of URI 379 * @param clientID of String 380 */ 381 public KafkaTap( KafkaScheme<K, V, ?, ?> scheme, URI identifier, String clientID ) 382 { 383 this( scheme, identifier, clientID, DEFAULT_POLL_TIMEOUT, DEFAULT_NUM_PARTITIONS, DEFAULT_REPLICATION_FACTOR ); 384 } 385 386 /** 387 * Constructor KafkaTap creates a new KafkaTap instance. 388 * 389 * @param scheme of KafkaScheme 390 * @param identifier of URI 391 * @param clientID of String 392 * @param groupID of String 393 */ 394 public KafkaTap( KafkaScheme<K, V, ?, ?> scheme, URI identifier, String clientID, String groupID ) 395 { 396 this( null, scheme, identifier, clientID, groupID, DEFAULT_POLL_TIMEOUT, DEFAULT_NUM_PARTITIONS, DEFAULT_REPLICATION_FACTOR ); 397 } 398 399 /** 400 * Constructor KafkaTap creates a new KafkaTap instance. 401 * 402 * @param defaultProperties of Properties 403 * @param scheme of KafkaScheme 404 * @param identifier of URI 405 * @param clientID of String 406 * @param pollTimeout of int 407 */ 408 public KafkaTap( Properties defaultProperties, KafkaScheme<K, V, ?, ?> scheme, URI identifier, String clientID, long pollTimeout ) 409 { 410 this( defaultProperties, scheme, identifier, clientID, pollTimeout, DEFAULT_NUM_PARTITIONS, DEFAULT_REPLICATION_FACTOR ); 411 } 412 413 /** 414 * Constructor KafkaTap creates a new KafkaTap instance. 415 * 416 * @param defaultProperties of Properties 417 * @param scheme of KafkaScheme 418 * @param identifier of URI 419 * @param clientID of String 420 * @param numPartitions of int 421 * @param replicationFactor of short 422 */ 423 public KafkaTap( Properties defaultProperties, KafkaScheme<K, V, ?, ?> scheme, URI identifier, String clientID, int numPartitions, short replicationFactor ) 424 { 425 this( defaultProperties, scheme, identifier, clientID, DEFAULT_POLL_TIMEOUT, numPartitions, replicationFactor ); 426 } 427 428 /** 429 * Constructor KafkaTap creates a new KafkaTap instance. 430 * 431 * @param defaultProperties of Properties 432 * @param scheme of KafkaScheme 433 * @param identifier of URI 434 * @param clientID of String 435 * @param pollTimeout of int 436 * @param numPartitions of int 437 * @param replicationFactor of short 438 */ 439 public KafkaTap( Properties defaultProperties, KafkaScheme<K, V, ?, ?> scheme, URI identifier, String clientID, long pollTimeout, int numPartitions, short replicationFactor ) 440 { 441 this( defaultProperties, scheme, identifier, clientID, null, pollTimeout, numPartitions, replicationFactor ); 442 } 443 444 /** 445 * Constructor KafkaTap creates a new KafkaTap instance. 446 * 447 * @param defaultProperties of Properties 448 * @param scheme of KafkaScheme 449 * @param identifier of URI 450 * @param clientID of String 451 * @param groupID of String 452 * @param pollTimeout of int 453 * @param numPartitions of int 454 * @param replicationFactor of short 455 */ 456 public KafkaTap( Properties defaultProperties, KafkaScheme<K, V, ?, ?> scheme, URI identifier, String clientID, String groupID, long pollTimeout, int numPartitions, short replicationFactor ) 457 { 458 super( scheme, SinkMode.UPDATE ); 459 460 if( defaultProperties != null ) 461 this.defaultProperties = new Properties( defaultProperties ); 462 463 if( identifier == null ) 464 throw new IllegalArgumentException( "identifier may not be null" ); 465 466 if( !identifier.getScheme().equalsIgnoreCase( "kafka" ) ) 467 throw new IllegalArgumentException( "identifier does not have kafka scheme" ); 468 469 this.hostname = identifier.getHost(); 470 471 if( identifier.getPort() != -1 ) 472 this.hostname += ":" + identifier.getPort(); 473 474 if( identifier.getQuery() == null ) 475 throw new IllegalArgumentException( "must have at least one topic in the query part of the URI" ); 476 477 if( clientID != null ) 478 this.clientID = clientID; 479 480 if( groupID != null ) 481 this.groupID = groupID; 482 483 this.pollTimeout = pollTimeout; 484 this.numPartitions = numPartitions; 485 this.replicationFactor = replicationFactor; 486 487 applyTopics( identifier.getQuery().split( "," ) ); 488 } 489 490 /** 491 * Constructor KafkaTap creates a new KafkaTap instance. 492 * 493 * @param scheme of KafkaScheme 494 * @param hostname of String 495 * @param pollTimeout of long 496 * @param topics of String... 497 */ 498 public KafkaTap( KafkaScheme<K, V, ?, ?> scheme, String hostname, long pollTimeout, String... topics ) 499 { 500 this( scheme, hostname, pollTimeout, DEFAULT_NUM_PARTITIONS, DEFAULT_REPLICATION_FACTOR, topics ); 501 } 502 503 /** 504 * Constructor KafkaTap creates a new KafkaTap instance. 505 * 506 * @param scheme of KafkaScheme 507 * @param hostname of String 508 * @param pollTimeout of int 509 * @param numPartitions of int 510 * @param replicationFactor of short 511 * @param topics of String... 512 */ 513 public KafkaTap( KafkaScheme<K, V, ?, ?> scheme, String hostname, long pollTimeout, int numPartitions, short replicationFactor, String... topics ) 514 { 515 this( null, scheme, hostname, pollTimeout, numPartitions, replicationFactor, topics ); 516 } 517 518 /** 519 * Constructor KafkaTap creates a new KafkaTap instance. 520 * 521 * @param defaultProperties of Properties 522 * @param scheme of KafkaScheme 523 * @param hostname of String 524 * @param numPartitions of int 525 * @param replicationFactor of short 526 * @param topics of String... 527 */ 528 public KafkaTap( Properties defaultProperties, KafkaScheme<K, V, ?, ?> scheme, String hostname, int numPartitions, short replicationFactor, String... topics ) 529 { 530 this( defaultProperties, scheme, hostname, DEFAULT_POLL_TIMEOUT, numPartitions, replicationFactor, topics ); 531 } 532 533 /** 534 * Constructor KafkaTap creates a new KafkaTap instance. 535 * 536 * @param defaultProperties of Properties 537 * @param scheme of KafkaScheme 538 * @param hostname of String 539 * @param topics of String... 540 */ 541 public KafkaTap( Properties defaultProperties, KafkaScheme<K, V, ?, ?> scheme, String hostname, String... topics ) 542 { 543 this( defaultProperties, scheme, hostname, DEFAULT_POLL_TIMEOUT, DEFAULT_NUM_PARTITIONS, DEFAULT_REPLICATION_FACTOR, topics ); 544 } 545 546 /** 547 * Constructor KafkaTap creates a new KafkaTap instance. 548 * 549 * @param defaultProperties of Properties 550 * @param scheme of KafkaScheme 551 * @param hostname of String 552 * @param pollTimeout of int 553 * @param topics of String... 554 */ 555 public KafkaTap( Properties defaultProperties, KafkaScheme<K, V, ?, ?> scheme, String hostname, long pollTimeout, String... topics ) 556 { 557 this( defaultProperties, scheme, hostname, pollTimeout, DEFAULT_NUM_PARTITIONS, DEFAULT_REPLICATION_FACTOR, topics ); 558 } 559 560 /** 561 * Constructor KafkaTap creates a new KafkaTap instance. 562 * 563 * @param defaultProperties of Properties 564 * @param scheme of KafkaScheme 565 * @param hostname of String 566 * @param pollTimeout of int 567 * @param numPartitions of int 568 * @param replicationFactor of short 569 * @param topics of String... 570 */ 571 public KafkaTap( Properties defaultProperties, KafkaScheme<K, V, ?, ?> scheme, String hostname, long pollTimeout, int numPartitions, short replicationFactor, String... topics ) 572 { 573 this( defaultProperties, scheme, hostname, null, pollTimeout, numPartitions, replicationFactor, topics ); 574 } 575 576 /** 577 * Constructor KafkaTap creates a new KafkaTap instance. 578 * 579 * @param scheme of KafkaScheme 580 * @param hostname of String 581 * @param clientID of String 582 * @param topics of String... 583 */ 584 public KafkaTap( KafkaScheme<K, V, ?, ?> scheme, String hostname, String clientID, String... topics ) 585 { 586 this( scheme, hostname, clientID, DEFAULT_POLL_TIMEOUT, DEFAULT_NUM_PARTITIONS, DEFAULT_REPLICATION_FACTOR, topics ); 587 } 588 589 /** 590 * Constructor KafkaTap creates a new KafkaTap instance. 591 * 592 * @param scheme of KafkaScheme 593 * @param hostname of String 594 * @param clientID of String 595 * @param pollTimeout of long 596 * @param topics of String... 597 */ 598 public KafkaTap( KafkaScheme<K, V, ?, ?> scheme, String hostname, String clientID, long pollTimeout, String... topics ) 599 { 600 this( scheme, hostname, clientID, pollTimeout, DEFAULT_NUM_PARTITIONS, DEFAULT_REPLICATION_FACTOR, topics ); 601 } 602 603 /** 604 * Constructor KafkaTap creates a new KafkaTap instance. 605 * 606 * @param scheme of KafkaScheme 607 * @param hostname of String 608 * @param clientID of String 609 * @param pollTimeout of int 610 * @param numPartitions of int 611 * @param replicationFactor of short 612 * @param topics of String... 613 */ 614 public KafkaTap( KafkaScheme<K, V, ?, ?> scheme, String hostname, String clientID, long pollTimeout, int numPartitions, short replicationFactor, String... topics ) 615 { 616 this( null, scheme, hostname, clientID, pollTimeout, numPartitions, replicationFactor, topics ); 617 } 618 619 /** 620 * Constructor KafkaTap creates a new KafkaTap instance. 621 * 622 * @param defaultProperties of Properties 623 * @param scheme of KafkaScheme 624 * @param hostname of String 625 * @param clientID of String 626 * @param numPartitions of int 627 * @param replicationFactor of short 628 * @param topics of String... 629 */ 630 public KafkaTap( Properties defaultProperties, KafkaScheme<K, V, ?, ?> scheme, String hostname, String clientID, int numPartitions, short replicationFactor, String... topics ) 631 { 632 this( defaultProperties, scheme, hostname, clientID, DEFAULT_POLL_TIMEOUT, numPartitions, replicationFactor, topics ); 633 } 634 635 /** 636 * Constructor KafkaTap creates a new KafkaTap instance. 637 * 638 * @param defaultProperties of Properties 639 * @param scheme of KafkaScheme 640 * @param hostname of String 641 * @param clientID of String 642 * @param topics of String... 643 */ 644 public KafkaTap( Properties defaultProperties, KafkaScheme<K, V, ?, ?> scheme, String hostname, String clientID, String... topics ) 645 { 646 this( defaultProperties, scheme, hostname, clientID, DEFAULT_POLL_TIMEOUT, DEFAULT_NUM_PARTITIONS, DEFAULT_REPLICATION_FACTOR, topics ); 647 } 648 649 /** 650 * Constructor KafkaTap creates a new KafkaTap instance. 651 * 652 * @param defaultProperties of Properties 653 * @param scheme of KafkaScheme 654 * @param hostname of String 655 * @param clientID of String 656 * @param pollTimeout of int 657 * @param topics of String... 658 */ 659 public KafkaTap( Properties defaultProperties, KafkaScheme<K, V, ?, ?> scheme, String hostname, String clientID, long pollTimeout, String... topics ) 660 { 661 this( defaultProperties, scheme, hostname, clientID, pollTimeout, DEFAULT_NUM_PARTITIONS, DEFAULT_REPLICATION_FACTOR, topics ); 662 } 663 664 /** 665 * Constructor KafkaTap creates a new KafkaTap instance. 666 * 667 * @param defaultProperties of Properties 668 * @param scheme of KafkaScheme 669 * @param hostname of String 670 * @param clientID of String 671 * @param pollTimeout of int 672 * @param numPartitions of int 673 * @param replicationFactor of short 674 * @param topics of String... 675 */ 676 public KafkaTap( Properties defaultProperties, KafkaScheme<K, V, ?, ?> scheme, String hostname, String clientID, long pollTimeout, int numPartitions, short replicationFactor, String... topics ) 677 { 678 this( defaultProperties, scheme, hostname, clientID, null, pollTimeout, numPartitions, replicationFactor, topics ); 679 } 680 681 public KafkaTap( Properties defaultProperties, KafkaScheme<K, V, ?, ?> scheme, String hostname, String clientID, String groupID, long pollTimeout, int numPartitions, short replicationFactor, String... topics ) 682 { 683 super( scheme, SinkMode.UPDATE ); 684 685 if( defaultProperties != null ) 686 this.defaultProperties = new Properties( defaultProperties ); 687 688 this.hostname = hostname; 689 690 if( clientID != null ) 691 this.clientID = clientID; 692 693 if( groupID != null ) 694 this.groupID = groupID; 695 696 this.pollTimeout = pollTimeout; 697 this.numPartitions = numPartitions; 698 this.replicationFactor = replicationFactor; 699 700 applyTopics( topics ); 701 } 702 703 protected void applyTopics( String[] topics ) 704 { 705 if( topics[ 0 ].matches( "^/([^/]|//)*/$" ) ) 706 { 707 this.topics = new String[]{topics[ 0 ].substring( 1, topics[ 0 ].length() - 1 )}; 708 this.isTopicPattern = true; 709 } 710 else 711 { 712 this.topics = new String[ topics.length ]; 713 System.arraycopy( topics, 0, this.topics, 0, topics.length ); 714 } 715 } 716 717 /** 718 * Method getHostname returns the hostname of this KafkaTap object. 719 * 720 * @return the hostname (type String) of this KafkaTap object. 721 */ 722 public String getHostname() 723 { 724 return hostname; 725 } 726 727 /** 728 * Method getClientID returns the clientID of this KafkaTap object. 729 * 730 * @return the clientID (type String) of this KafkaTap object. 731 */ 732 public String getClientID() 733 { 734 return clientID; 735 } 736 737 /** 738 * Method getGroupID returns the groupID of this KafkaTap object. 739 * 740 * @return the groupID (type String) of this KafkaTap object. 741 */ 742 public String getGroupID() 743 { 744 return groupID; 745 } 746 747 /** 748 * Method getTopics returns the topics of this KafkaTap object. 749 * 750 * @return the topics (type String[]) of this KafkaTap object. 751 */ 752 public String[] getTopics() 753 { 754 return topics; 755 } 756 757 /** 758 * Method isTopicPattern returns true if the topic is a regular expression. 759 * 760 * @return true if the topic is a regular expression. 761 */ 762 public boolean isTopicPattern() 763 { 764 return isTopicPattern; 765 } 766 767 @Override 768 public String getIdentifier() 769 { 770 return makeURI( hostname, topics ).toString(); 771 } 772 773 protected Consumer<K, V> createKafkaConsumer( Properties properties ) 774 { 775 return new ForwardingConsumer<>( properties ); 776 } 777 778 @Override 779 public TupleEntryIterator openForRead( FlowProcess<? extends Properties> flowProcess, KafkaConsumerRecordIterator<K, V> consumerRecord ) throws IOException 780 { 781 Properties props = PropertyUtil.merge( flowProcess.getConfig(), defaultProperties ); 782 783 props.setProperty( ConsumerConfig.BOOTSTRAP_SERVERS_CONFIG, hostname ); 784 785 Set<String> keys = props.stringPropertyNames(); 786 787 if( clientID != null && !keys.contains( ConsumerConfig.CLIENT_ID_CONFIG ) ) 788 props.setProperty( ConsumerConfig.CLIENT_ID_CONFIG, clientID ); 789 790 // allows all calls from this instance to share a group 791 if( !keys.contains( ConsumerConfig.GROUP_ID_CONFIG ) ) 792 props.setProperty( ConsumerConfig.GROUP_ID_CONFIG, groupID ); 793 794 sourceConfInit( flowProcess, props ); 795 796 Properties consumerProperties = PropertyUtil.retain( props, ConsumerConfig.configNames() ); 797 Consumer<K, V> consumer = createKafkaConsumer( consumerProperties ); 798 799 preConsumerSubscribe( consumer ); 800 801 if( isTopicPattern ) 802 consumer.subscribe( Pattern.compile( topics[ 0 ] ), getConsumerRebalanceListener( consumer ) ); 803 else 804 consumer.subscribe( Arrays.asList( getTopics() ), getConsumerRebalanceListener( consumer ) ); 805 806 postConsumerSubscribe( consumer ); 807 808 CloseableIterator<Iterator<ConsumerRecord<K, V>>> iterator = new CloseableIterator<Iterator<ConsumerRecord<K, V>>>() 809 { 810 boolean completed = false; 811 ConsumerRecords<K, V> records; 812 813 @Override 814 public boolean hasNext() 815 { 816 if( records != null ) 817 return true; 818 819 if( completed ) 820 return false; 821 822 records = consumer.poll( pollTimeout ); 823 824 if( LOG.isDebugEnabled() ) 825 LOG.debug( "kafka records polled: {}", records.count() ); 826 827 if( records.isEmpty() ) 828 { 829 completed = true; 830 records = null; 831 } 832 833 return records != null; 834 } 835 836 @Override 837 public Iterator<ConsumerRecord<K, V>> next() 838 { 839 if( !hasNext() ) 840 throw new NoSuchElementException( "no more elements" ); 841 842 try 843 { 844 CloseableIterator<Iterator<ConsumerRecord<K, V>>> parent = this; 845 846 return new KafkaConsumerRecordIterator<K, V>() 847 { 848 Iterator<ConsumerRecord<K, V>> delegate = records.iterator(); 849 Supplier<Boolean> hasNext = () -> delegate.hasNext(); 850 851 @Override 852 protected Consumer<K, V> getConsumer() 853 { 854 return consumer; 855 } 856 857 @Override 858 public void close() throws IOException 859 { 860 hasNext = () -> false; 861 parent.close(); 862 } 863 864 @Override 865 public boolean hasNext() 866 { 867 return hasNext.get(); 868 } 869 870 @Override 871 public ConsumerRecord<K, V> next() 872 { 873 return delegate.next(); 874 } 875 }; 876 } 877 finally 878 { 879 records = null; 880 } 881 } 882 883 @Override 884 public void close() 885 { 886 try 887 { 888 try 889 { 890 consumer.close(); 891 } 892 catch( IllegalStateException exception ) 893 { 894 LOG.error( "ignoring exception on closing", exception ); 895 } 896 } 897 finally 898 { 899 completed = true; 900 } 901 } 902 }; 903 904 return new TupleEntrySchemeIterator<Properties, Iterator<ConsumerRecord<K, V>>>( flowProcess, this, getScheme(), iterator ); 905 } 906 907 /** 908 * Prepare {@link Consumer} prior to any topic subscription. 909 * 910 * @param consumer the current Consumer 911 */ 912 protected void preConsumerSubscribe( Consumer<K, V> consumer ) 913 { 914 915 } 916 917 /** 918 * Prepare {@link Consumer} post to any topic subscription and prior to any {@link Consumer#poll(long)} request. 919 * 920 * @param consumer the current Consumer 921 */ 922 protected void postConsumerSubscribe( Consumer<K, V> consumer ) 923 { 924 925 } 926 927 /** 928 * Returns a {@link NoOpConsumerRebalanceListener} instance. 929 * <p> 930 * Override to supply a customer listener. 931 * 932 * @param consumer 933 * @return a NoOpConsumerRebalanceListener instance. 934 */ 935 protected ConsumerRebalanceListener getConsumerRebalanceListener( Consumer<K, V> consumer ) 936 { 937 return new NoOpConsumerRebalanceListener(); 938 } 939 940 @Override 941 public TupleEntryCollector openForWrite( FlowProcess<? extends Properties> flowProcess, Producer<K, V> producer ) throws IOException 942 { 943 Properties props = PropertyUtil.merge( flowProcess.getConfig(), defaultProperties ); 944 945 props.setProperty( ProducerConfig.BOOTSTRAP_SERVERS_CONFIG, hostname ); 946 947 sinkConfInit( flowProcess, props ); 948 949 producer = new KafkaProducer<>( PropertyUtil.retain( props, ProducerConfig.configNames() ) ); 950 951 return new TupleEntrySchemeCollector<Properties, Producer<?, ?>>( flowProcess, this, getScheme(), producer ); 952 } 953 954 protected AdminClient createAdminClient( Properties conf ) 955 { 956 Properties props = new Properties( conf ); 957 958 props.setProperty( ProducerConfig.BOOTSTRAP_SERVERS_CONFIG, hostname ); 959 960 return AdminClient.create( props ); 961 } 962 963 @Override 964 public boolean createResource( Properties conf ) 965 { 966 AdminClient client = createAdminClient( conf ); 967 968 List<NewTopic> topics = new ArrayList<>( getTopics().length ); 969 970 for( String topic : getTopics() ) 971 topics.add( new NewTopic( topic, numPartitions, replicationFactor ) ); 972 973 CreateTopicsResult result = client.createTopics( topics ); 974 975 KafkaFuture<Void> all = result.all(); 976 977 try 978 { 979 all.get(); 980 } 981 catch( InterruptedException | ExecutionException exception ) 982 { 983 LOG.info( "unable to create topics" ); 984 985 return false; 986 } 987 988 return true; 989 } 990 991 @Override 992 public boolean deleteResource( Properties conf ) 993 { 994 AdminClient client = createAdminClient( conf ); 995 996 DeleteTopicsResult result = client.deleteTopics( Arrays.asList( getTopics() ) ); 997 998 KafkaFuture<Void> all = result.all(); 999 1000 try 1001 { 1002 all.get(); 1003 } 1004 catch( InterruptedException | ExecutionException exception ) 1005 { 1006 LOG.info( "unable to create topics" ); 1007 1008 return false; 1009 } 1010 1011 return true; 1012 } 1013 1014 @Override 1015 public boolean resourceExists( Properties conf ) 1016 { 1017 AdminClient client = createAdminClient( conf ); 1018 1019 DescribeTopicsResult result = client.describeTopics( Arrays.asList( getTopics() ) ); 1020 1021 KafkaFuture<Map<String, TopicDescription>> all = result.all(); 1022 1023 try 1024 { 1025 Map<String, TopicDescription> map = all.get(); 1026 1027 return map.size() == getTopics().length; 1028 } 1029 catch( InterruptedException | ExecutionException exception ) 1030 { 1031 LOG.info( "unable to create topics" ); 1032 1033 return false; 1034 } 1035 } 1036 1037 @Override 1038 public long getModifiedTime( Properties conf ) throws IOException 1039 { 1040 if( resourceExists( conf ) ) 1041 return Long.MAX_VALUE; 1042 else 1043 return 0L; 1044 } 1045 }