Skip to content
Open
Show file tree
Hide file tree
Changes from all commits
Commits
File filter

Filter by extension

Filter by extension


Conversations
Failed to load comments.
Loading
Jump to
Jump to file
Failed to load files.
Loading
Diff view
Diff view
7 changes: 6 additions & 1 deletion itests/qtest-druid/pom.xml
Original file line number Diff line number Diff line change
Expand Up @@ -36,7 +36,7 @@
<druid.derby.version>10.11.1.1</druid.derby.version>
<druid.guava.version>16.0.1</druid.guava.version>
<druid.guice.version>4.1.0</druid.guice.version>
<kafka.test.version>2.5.0</kafka.test.version>
<kafka.test.version>3.9.1</kafka.test.version>
<druid.guice.version>4.1.0</druid.guice.version>
<slf4j.version>1.7.30</slf4j.version>
</properties>
Expand Down Expand Up @@ -226,6 +226,11 @@
<artifactId>kafka-clients</artifactId>
<version>${kafka.test.version}</version>
</dependency>
<dependency>
<groupId>org.apache.kafka</groupId>
<artifactId>kafka-server</artifactId>
<version>${kafka.test.version}</version>
</dependency>
<dependency>
<groupId>org.slf4j</groupId>
<artifactId>slf4j-api</artifactId>
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -19,7 +19,7 @@
package org.apache.hive.kafka;

import kafka.server.KafkaConfig;
import kafka.server.KafkaServerStartable;
import kafka.server.KafkaServer;

import org.apache.commons.io.FileUtils;
import org.apache.hadoop.service.AbstractService;
Expand All @@ -29,6 +29,7 @@
import org.apache.kafka.clients.producer.ProducerRecord;
import org.apache.kafka.common.serialization.ByteArraySerializer;
import org.apache.kafka.common.serialization.StringSerializer;
import org.apache.kafka.common.utils.Time;

import com.google.common.base.Throwables;
import com.google.common.io.Files;
Expand All @@ -43,6 +44,7 @@
import java.util.List;
import java.util.Properties;
import java.util.stream.IntStream;
import scala.Option;

/**
* This class has the hooks to start and stop single node kafka cluster.
Expand All @@ -54,7 +56,7 @@ public class SingleNodeKafkaCluster extends AbstractService {
private static final String LOCALHOST = "localhost";


private final KafkaServerStartable serverStartable;
private final KafkaServer server;
private final int brokerPort;
private final String kafkaServer;

Expand Down Expand Up @@ -94,21 +96,21 @@ public SingleNodeKafkaCluster(String name, String logDir, Integer zkPort, Intege
properties.setProperty("transaction.state.log.min.isr", String.valueOf(1));
properties.setProperty("log.cleaner.dedupe.buffer.size", "1048577");

this.serverStartable = new KafkaServerStartable(KafkaConfig.fromProps(properties));
this.server = new KafkaServer(KafkaConfig.fromProps(properties), Time.SYSTEM, Option.empty(), false);
}


@Override
protected void serviceStart() throws Exception {
serverStartable.startup();
server.startup();
log.info("Kafka Server Started on port {}", brokerPort);

}

@Override
protected void serviceStop() throws Exception {
log.info("Stopping Kafka Server");
serverStartable.shutdown();
server.shutdown();
log.info("Kafka Server Stopped");
}

Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -33,6 +33,7 @@
import org.apache.kafka.common.PartitionInfo;
import org.apache.kafka.common.TopicPartition;
import org.apache.kafka.common.errors.ProducerFencedException;
import org.apache.kafka.common.Uuid;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;

Expand All @@ -44,6 +45,7 @@
import java.util.List;
import java.util.Map;
import java.util.Properties;
import java.util.Set;
import java.util.concurrent.Future;

/**
Expand All @@ -67,6 +69,11 @@ class HiveKafkaProducer<K, V> implements Producer<K, V> {
kafkaProducer = new KafkaProducer<>(properties);
}

@Override
public Uuid clientInstanceId(Duration timeout) {
throw new UnsupportedOperationException();
}

@Override public void initTransactions() {
kafkaProducer.initTransactions();
}
Expand Down Expand Up @@ -138,11 +145,11 @@ synchronized void resumeTransaction(long producerId, short epoch) {

Object transactionManager = getValue(kafkaProducer, "transactionManager");

Object topicPartitionBookkeeper = getValue(transactionManager, "topicPartitionBookkeeper");
Object txnPartitionMap = getValue(transactionManager, "txnPartitionMap");
invoke(transactionManager,
"transitionTo",
getEnum("org.apache.kafka.clients.producer.internals.TransactionManager$State.INITIALIZING"));
invoke(topicPartitionBookkeeper, "reset");
invoke(txnPartitionMap, "reset");
Object producerIdAndEpoch = getValue(transactionManager, "producerIdAndEpoch");
setValue(producerIdAndEpoch, "producerId", producerId);
setValue(producerIdAndEpoch, "epoch", epoch);
Expand Down Expand Up @@ -181,10 +188,15 @@ short getEpoch() {
*/
private void flushNewPartitions() {
LOG.info("Flushing new partitions");
TransactionalRequestResult result = enqueueNewPartitions();
Object sender = getValue(kafkaProducer, "sender");
invoke(sender, "wakeup");
result.await();
Object transactionManager = getValue(kafkaProducer, "transactionManager");
Set<TopicPartition> newPartitionsInTransaction =
(Set<TopicPartition>) getValue(transactionManager, "newPartitionsInTransaction");
if (!newPartitionsInTransaction.isEmpty()) {
TransactionalRequestResult result = enqueueNewPartitions();
Object sender = getValue(kafkaProducer, "sender");
invoke(sender, "wakeup");
result.await();
}
}

private synchronized TransactionalRequestResult enqueueNewPartitions() {
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -140,7 +140,7 @@ class KafkaRecordIterator implements Iterator<ConsumerRecord<byte[], byte[]>> {
}
} else {
// case seek to beginning of stream
consumer.seekToBeginning(Collections.singleton(topicPartition));
consumer.seekToBeginning(topicPartitionList);
// seekToBeginning is lazy thus need to call position() or poll(0)
this.startOffset = consumer.position(topicPartition);
LOG.info("Consumer at beginning of topic partition [{}], current start offset [{}]",
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -25,6 +25,7 @@
import org.apache.kafka.clients.consumer.KafkaConsumer;
import org.apache.kafka.clients.producer.ProducerConfig;
import org.apache.kafka.clients.producer.ProducerRecord;
import org.apache.kafka.clients.consumer.OffsetAndMetadata;
import org.apache.kafka.common.TopicPartition;
import org.apache.kafka.common.serialization.ByteArrayDeserializer;
import org.apache.kafka.common.serialization.ByteArraySerializer;
Expand Down Expand Up @@ -158,7 +159,9 @@
@Test(expected = org.apache.kafka.common.KafkaException.class) public void testWrongEpochAndId() {
HiveKafkaProducer secondProducer = new HiveKafkaProducer(producerProperties);
secondProducer.resumeTransaction(3434L, (short) 12);
secondProducer.sendOffsetsToTransaction(ImmutableMap.of(), "__dummy_consumer_group");
secondProducer.sendOffsetsToTransaction(Collections.singletonMap(
new TopicPartition("dummy_topic", 0),
new OffsetAndMetadata(0L)), "__dummy_consumer_group");
secondProducer.close(Duration.ZERO);
}

Expand All @@ -169,7 +172,9 @@
producer.close(Duration.ZERO);
HiveKafkaProducer secondProducer = new HiveKafkaProducer(producerProperties);
secondProducer.resumeTransaction(pid, (short) 12);
secondProducer.sendOffsetsToTransaction(ImmutableMap.of(), "__dummy_consumer_group");
secondProducer.sendOffsetsToTransaction(Collections.singletonMap(
new TopicPartition("dummy_topic", 0),
new OffsetAndMetadata(0L)), "__dummy_consumer_group");
secondProducer.close(Duration.ZERO);
}

Expand All @@ -180,7 +185,9 @@
producer.close(Duration.ZERO);
HiveKafkaProducer secondProducer = new HiveKafkaProducer(producerProperties);
secondProducer.resumeTransaction(45L, epoch);
secondProducer.sendOffsetsToTransaction(ImmutableMap.of(), "__dummy_consumer_group");
secondProducer.sendOffsetsToTransaction(Collections.singletonMap(
new TopicPartition("dummy_topic", 0),
new OffsetAndMetadata(0L)), "__dummy_consumer_group");
secondProducer.close(Duration.ZERO);
}
}
Original file line number Diff line number Diff line change
Expand Up @@ -26,7 +26,7 @@
import kafka.zk.EmbeddedZookeeper;
import org.apache.commons.io.FileUtils;
import org.apache.hadoop.hive.common.IPStackUtils;
import org.apache.kafka.common.network.Mode;
import org.apache.kafka.common.network.ConnectionMode;
import org.apache.kafka.common.utils.Time;
import org.apache.kafka.test.TestSslUtils;
import org.junit.rules.ExternalResource;
Expand All @@ -41,6 +41,7 @@
import java.util.Map;
import java.util.Properties;
import java.util.stream.Collectors;
import scala.Option;

/**
* Test Helper Class to start and stop a kafka broker.
Expand Down Expand Up @@ -106,7 +107,8 @@ KafkaBrokerResource enableSASL(String principal, String keytab) {
brokerProps.setProperty("listener.name.l2.gssapi.sasl.jaas.config", jaasConfig);
brokerProps.setProperty("listener.name.l3.gssapi.sasl.jaas.config", jaasConfig);
truststoreFile = File.createTempFile("kafka_truststore", "jks");
brokerProps.putAll(new TestSslUtils.SslConfigsBuilder(Mode.SERVER).createNewTrustStore(truststoreFile).build());
brokerProps.putAll(new TestSslUtils.SslConfigsBuilder(ConnectionMode.SERVER)
.createNewTrustStore(truststoreFile).build());
brokerProps.setProperty("delegation.token.master.key", "AnyValueShouldDoHereItDoesntMatter");
}
brokerProps.setProperty("offsets.topic.replication.factor", "1");
Expand All @@ -116,9 +118,9 @@ KafkaBrokerResource enableSASL(String principal, String keytab) {
kafkaServer = TestUtils.createServer(config, Time.SYSTEM);
kafkaServer.startup();
kafkaServer.zkClient();
adminZkClient = new AdminZkClient(kafkaServer.zkClient());
adminZkClient = new AdminZkClient(kafkaServer.zkClient(), Option.empty());
LOG.info("Creating kafka TOPIC [{}]", TOPIC);
adminZkClient.createTopic(TOPIC, 1, 1, new Properties(), RackAwareMode.Disabled$.MODULE$);
adminZkClient.createTopic(TOPIC, 1, 1, new Properties(), RackAwareMode.Disabled$.MODULE$, false);
}

/**
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -44,6 +44,7 @@

import javax.annotation.Nullable;
import java.nio.charset.Charset;
import java.time.Duration;
import java.util.Arrays;
import java.util.Iterator;
import java.util.List;
Expand Down Expand Up @@ -304,7 +305,7 @@ private static void sendData(List<ConsumerRecord<byte[], byte[]>> recordList, @N
@After public void tearDown() {
this.kafkaRecordIterator = null;
if (this.consumer != null) {
this.consumer.close();
this.consumer.close(Duration.ZERO);
}
}

Expand Down
2 changes: 1 addition & 1 deletion pom.xml
Original file line number Diff line number Diff line change
Expand Up @@ -172,7 +172,7 @@
<junit.version>4.13.2</junit.version>
<junit.jupiter.version>5.13.3</junit.jupiter.version>
<junit.vintage.version>5.13.3</junit.vintage.version>
<kafka.version>2.5.0</kafka.version>
<kafka.version>3.9.1</kafka.version>
Copy link
Member

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

If I recall well there were issues with previous upgrade attempts. Please check the (git) history and related PRs for more information to ensure that code remains functional.

Copy link
Contributor Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

yes I am aware of that, its wip right now, I am planning to address those issues if I get a grren label in current state

<kryo.version>5.5.0</kryo.version>
<reflectasm.version>1.11.9</reflectasm.version>
<kudu.version>1.17.0</kudu.version>
Expand Down