diff --git a/pom.xml b/pom.xml index 3985475d962a6..d2bbd9b058dea 100644 --- a/pom.xml +++ b/pom.xml @@ -65,7 +65,7 @@ 1.9.17 313 1.7.32 - 2.3.1 + 3.7.1 0.11.0 0.19.0 2.3.1 @@ -201,6 +201,13 @@ presto-openapi + + + jitpack.io + https://jitpack.io + + + @@ -1232,7 +1239,7 @@ org.pcollections pcollections - 2.1.2 + 4.0.1 @@ -1829,6 +1836,10 @@ org.apache.zookeeper zookeeper + + commons-logging + commons-logging + @@ -2009,9 +2020,9 @@ - io.prestodb.tempto + com.github.adkharat.tempto tempto-kafka - ${dep.tempto.version} + upgrade_kafka_client_version_cve_2022_34917-SNAPSHOT com.datastax.cassandra diff --git a/presto-kafka/pom.xml b/presto-kafka/pom.xml index 6dc3580b8c067..49693eabd4de6 100644 --- a/presto-kafka/pom.xml +++ b/presto-kafka/pom.xml @@ -248,6 +248,16 @@ + + org.basepom.maven + duplicate-finder-maven-plugin + + + kafka/kafka-version.properties + about.html + + + diff --git a/presto-kafka/src/test/java/com/facebook/presto/kafka/util/EmbeddedKafka.java b/presto-kafka/src/test/java/com/facebook/presto/kafka/util/EmbeddedKafka.java index 8b0adfe9662f4..2cc9f328cbdc8 100644 --- a/presto-kafka/src/test/java/com/facebook/presto/kafka/util/EmbeddedKafka.java +++ b/presto-kafka/src/test/java/com/facebook/presto/kafka/util/EmbeddedKafka.java @@ -16,19 +16,23 @@ import com.google.common.collect.ImmutableMap; import com.google.common.collect.Maps; import com.google.common.io.Files; -import kafka.admin.AdminUtils; -import kafka.admin.RackAwareMode; import kafka.server.KafkaConfig; -import kafka.server.KafkaServerStartable; -import kafka.utils.ZkUtils; +import kafka.server.KafkaServer; +import org.apache.kafka.clients.admin.AdminClient; +import org.apache.kafka.clients.admin.AdminClientConfig; +import org.apache.kafka.clients.admin.NewTopic; import org.apache.kafka.clients.producer.KafkaProducer; import org.apache.kafka.common.serialization.LongSerializer; +import org.apache.kafka.common.utils.SystemTime; +import org.apache.kafka.common.utils.Time; import java.io.Closeable; import java.io.File; import java.io.IOException; +import java.util.Collections; import java.util.Map; import java.util.Properties; +import java.util.concurrent.ExecutionException; import java.util.concurrent.atomic.AtomicBoolean; import static com.facebook.presto.kafka.util.TestUtils.findUnusedPort; @@ -48,7 +52,8 @@ public class EmbeddedKafka private final EmbeddedZookeeper zookeeper; private final int port; private final File kafkaDataDir; - private final KafkaServerStartable kafka; + private final KafkaServer kafka; + private final AdminClient adminClient; private final AtomicBoolean started = new AtomicBoolean(); private final AtomicBoolean stopped = new AtomicBoolean(); @@ -91,7 +96,11 @@ public static EmbeddedKafka createEmbeddedKafka(Properties overrideProperties) .build(); KafkaConfig config = new KafkaConfig(toProperties(properties)); - this.kafka = new KafkaServerStartable(config); + Time time = new SystemTime(); + this.kafka = new KafkaServer(config, time, scala.Option.empty(), false); + Properties adminProps = new Properties(); + adminProps.put(AdminClientConfig.BOOTSTRAP_SERVERS_CONFIG, getConnectString()); + this.adminClient = AdminClient.create(adminProps); } public void start() @@ -112,6 +121,7 @@ public void close() kafka.awaitShutdown(); zookeeper.close(); deleteRecursively(kafkaDataDir.toPath(), ALLOW_INSECURE); + adminClient.close(); } } @@ -124,14 +134,15 @@ public void createTopics(int partitions, int replication, Properties topicProper { checkState(started.get() && !stopped.get(), "not started!"); - ZkUtils zkUtils = ZkUtils.apply(getZookeeperConnectString(), 30_000, 30_000, false); try { for (String topic : topics) { - AdminUtils.createTopic(zkUtils, topic, partitions, replication, topicProperties, RackAwareMode.Disabled$.MODULE$); + NewTopic newTopic = new NewTopic(topic, partitions, (short) replication); + newTopic.configs(Maps.fromProperties(topicProperties)); + adminClient.createTopics(Collections.singleton(newTopic)).all().get(); } } - finally { - zkUtils.close(); + catch (InterruptedException | ExecutionException e) { + throw new RuntimeException("Failed to create topics", e); } } diff --git a/presto-product-tests/pom.xml b/presto-product-tests/pom.xml index 0c97a9b0063f5..7aaa5e444a7c2 100644 --- a/presto-product-tests/pom.xml +++ b/presto-product-tests/pom.xml @@ -18,6 +18,13 @@ 2.12.2 + + + jitpack.io + https://jitpack.io + + + org.apache.avro @@ -53,8 +60,14 @@ tempto-ldap - io.prestodb.tempto + com.github.adkharat.tempto tempto-kafka + + + commons-cli + commons-cli + + io.prestodb.tempto @@ -141,6 +154,10 @@ io.netty * + + io.dropwizard.metrics + metrics-core +