@Override public UnboundedReader<Pair<byte[], byte[]>, Long> openReader() throws IOException { final Consumer<byte[], byte[]> c = KafkaUtils.newConsumer(brokerList, null, config); final List<TopicPartition> partitionList = Collections.singletonList(topicPartition); c.assign(partitionList); if (startOffset > 0) { c.seek(topicPartition, startOffset); } else if (startOffset == 0) { c.seekToBeginning(partitionList); } return new ConsumerReader(c, topicPartition, stopReadingAtStamp); }
@VisibleForTesting Consumer<byte[], byte[]> newConsumer( String brokerList, @Nullable String groupId, @Nullable Settings config) { return KafkaUtils.newConsumer( brokerList, groupId, config); }
static Producer<byte [], byte []> newProducer(String brokerList, Settings config) { final Properties ps = toProperties(config); ps.setProperty(ProducerConfig.BOOTSTRAP_SERVERS_CONFIG, brokerList); ps.setProperty(ProducerConfig.KEY_SERIALIZER_CLASS_CONFIG, ByteArraySerializer.class.getName()); ps.setProperty(ProducerConfig.VALUE_SERIALIZER_CLASS_CONFIG, ByteArraySerializer.class.getName()); if (ps.getProperty(ProducerConfig.ACKS_CONFIG) == null) { ps.setProperty(ProducerConfig.ACKS_CONFIG, "1"); } return new KafkaProducer<>(ps); }
public void tryGetPartitions(List<PartitionInfo> partitions) { KafkaSource source = mock(KafkaSource.class); Consumer<byte[], byte[]> consumer = mock(Consumer.class); when(consumer.partitionsFor(any(String.class))).thenReturn(partitions); when(source.newConsumer(any(), any(), any())).thenReturn(consumer); when(source.getPartitions()).thenCallRealMethod(); source.getPartitions(); } }
private List<UnboundedPartition<Pair<byte[], byte[]>, Long>> getPartitions( long startTimestamp, long endTimestamp) { try (Consumer<byte[], byte[]> consumer = newConsumer( brokerList, "euphoria.partition-probe-" + UUID.randomUUID().toString(), config)) { : startTimestamp; return new KafkaPartition( brokerList, tp,
case "kafka": { Dataset<Pair<byte[], byte[]>> input = flow.createInput( new KafkaSource(uri.getAuthority(), uri.getPath().substring(1), settings)); return FlatMap.of(input)
@Override public Writer<Pair<byte[], byte[]>> openWriter(int partitionId) { String cacheKey = brokers; Producer<byte[], byte[]> producer = PRODUCERS.get(cacheKey); if (producer == null) { // ~ ok, let's create a new producer (this may take some time) final Producer<byte[], byte[]> p = KafkaUtils.newProducer(brokers, config); // ~ now, let's try to store it in our global cache final Producer<byte[], byte[]> p1 = PRODUCERS.putIfAbsent(cacheKey, p); if (p1 == null) { producer = p; } else { // ~ looks like somebody managed to create concurrently a new // producer in between and store it quicker into the global cache producer = p1; // ~ must close the created one to avoid leaking resources! p.close(); } } final List<PartitionInfo> partitions = producer.partitionsFor(topic); return new ProducerWriter(producer, topic, partitionId % partitions.size()); }
@Override public List<UnboundedPartition<Pair<byte[], byte[]>, Long>> getPartitions() { long offsetTimestamp = -1L; long stopReadingAtStamp = Long.MAX_VALUE; if (config != null) { offsetTimestamp = config.getLong(CFG_RESET_OFFSET_TIMESTAMP_MILLIS, -1L); if (offsetTimestamp > 0) { LOG.info("Resetting offset of kafka topic {} to {}", topicId, offsetTimestamp); } else if (offsetTimestamp == 0) { LOG.info("Going to read the whole contents of kafka topic {}", topicId); } stopReadingAtStamp = config.getLong( CFG_STOP_AT_TIMESTAMP_MILLIS, stopReadingAtStamp); if (stopReadingAtStamp < Long.MAX_VALUE) { LOG.info("Will stop polling kafka topic at current timestamp {}", stopReadingAtStamp); } } return getPartitions(offsetTimestamp, stopReadingAtStamp); }
/** * Test if where there are no partitions * an IllegalStateException is expected */ @Test(expected = IllegalStateException.class) @SuppressWarnings("unchecked") public void testNoPartitions() { tryGetPartitions(Collections.emptyList()); }
@Override public void close() throws IOException { // ~ wait for all pending futures to finish if (LOG.isDebugEnabled()) { final int nItems = fs.size(); long start = System.nanoTime(); waitPendingConfirms(); long end = System.nanoTime(); LOG.debug("Finished waiting for confirmation of {} items in {}ms", nItems, TimeUnit.NANOSECONDS.toMillis(end - start)); } else { waitPendingConfirms(); } }
@Override protected Pair<byte[], byte[]> computeNext() { while (next == null || !next.hasNext()) { if (Thread.currentThread().isInterrupted()) { LOG.info("Terminating polling on topic due to thread interruption"); endOfData(); return null; } ConsumerRecords<byte[], byte[]> polled = c.poll(500); next = polled.iterator(); } ConsumerRecord<byte[], byte[]> r = this.next.next(); if (stopReadingAtStamp > 0) { long messageStamp = r.timestamp(); if (messageStamp > stopReadingAtStamp) { LOG.info( "Terminating polling of topic, passed initial timestamp {} with value {}", stopReadingAtStamp, messageStamp); endOfData(); return null; } } offset = r.offset(); return Pair.of(r.key(), r.value()); }
static DataSink<byte[]> getSink(URI output, Configuration conf) { switch (output.getScheme()) { case "hdfs": case "file": return DataSinks.mapping( SequenceFileSink .of(ImmutableBytesWritable.class, ImmutableBytesWritable.class) .outputPath(output.toString()) .withConfiguration(conf) .build(), b -> Pair.of(new ImmutableBytesWritable(), new ImmutableBytesWritable(b))); case "kafka": return DataSinks.mapping( new KafkaSink(output.getAuthority(), getPath(output), toSettings(conf)), b -> Pair.of(new byte[0], b)); } throw new IllegalArgumentException("Unknown scheme in " + output); }
settings.setInt(KafkaSource.CFG_RESET_OFFSET_TIMESTAMP_MILLIS, 0); Dataset<Pair<byte[], byte[]>> raw = flow.createInput( new KafkaSource(input.getAuthority(), getPath(input), settings));
@SuppressWarnings("unchecked") public void testPartitions() { Node leaderNode = new Node(1, "localhost", 3333); PartitionInfo pi = new PartitionInfo("topic", 0, leaderNode, null, null); tryGetPartitions(Collections.singletonList(pi)); }
static Consumer<byte[], byte[]> newConsumer( String brokerList, @Nullable String groupId, @Nullable Settings config) { Properties ps = toProperties(config); ps.setProperty(ConsumerConfig.BOOTSTRAP_SERVERS_CONFIG, brokerList); ps.setProperty(ConsumerConfig.KEY_DESERIALIZER_CLASS_CONFIG, StringDeserializer.class.getName()); ps.setProperty(ConsumerConfig.VALUE_DESERIALIZER_CLASS_CONFIG, ByteArrayDeserializer.class.getName()); if (groupId != null) { ps.setProperty(ConsumerConfig.GROUP_ID_CONFIG, groupId); } if (ps.getProperty(ConsumerConfig.CLIENT_ID_CONFIG) == null) { final String name = "euphoria.client-id-" + UUID.randomUUID().toString(); LOG.warn("Autogenerating name of consumer's {} to {}", ConsumerConfig.CLIENT_ID_CONFIG, name); ps.setProperty(ConsumerConfig.CLIENT_ID_CONFIG, name); } return new KafkaConsumer<>(ps); }
@Test(expected = IllegalStateException.class) @SuppressWarnings("unchecked") public void testNoLeader() { PartitionInfo pi = new PartitionInfo("topic", 0, Node.noNode(), null, null); tryGetPartitions(Collections.singletonList(pi)); }