Skip to content
Draft
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
53 changes: 42 additions & 11 deletions pom.xml
Original file line number Diff line number Diff line change
Expand Up @@ -56,7 +56,7 @@
<!-- Changing joda version changes tzdata which must match deployed JVM tzdata
Do not change this without also making sure it matches -->
<dep.joda.version>2.12.7</dep.joda.version>
<dep.tempto.version>1.53</dep.tempto.version>
<dep.tempto.version>1.54</dep.tempto.version>
<dep.testng.version>7.5</dep.testng.version>
<dep.lucene.version>8.10.0</dep.lucene.version>
<dep.assertj-core.version>3.8.0</dep.assertj-core.version>
Expand All @@ -67,7 +67,7 @@
<dep.gcs.version>1.9.17</dep.gcs.version>
<dep.alluxio.version>313</dep.alluxio.version>
<dep.slf4j.version>1.7.32</dep.slf4j.version>
<dep.kafka.version>2.3.1</dep.kafka.version>
<dep.kafka.version>3.9.0</dep.kafka.version>
<dep.pinot.version>0.11.0</dep.pinot.version>
<dep.druid.version>30.0.1</dep.druid.version>
<dep.jaxb.version>2.3.1</dep.jaxb.version>
Expand Down Expand Up @@ -207,6 +207,13 @@
<module>presto-native-sidecar-plugin</module>
</modules>

<repositories>
<repository>
<id>jitpack.io</id>
<url>https://jitpack.io</url>
</repository>
</repositories>

<dependencyManagement>
<dependencies>
<dependency>
Expand Down Expand Up @@ -1294,7 +1301,7 @@
<dependency>
<groupId>org.pcollections</groupId>
<artifactId>pcollections</artifactId>
<version>2.1.2</version>
<version>4.0.1</version>
</dependency>

<dependency>
Expand Down Expand Up @@ -1811,6 +1818,18 @@
<groupId>org.apache.zookeeper</groupId>
<artifactId>zookeeper</artifactId>
</exclusion>
<exclusion>
<groupId>commons-logging</groupId>
<artifactId>commons-logging</artifactId>
</exclusion>
<exclusion>
<groupId>com.fasterxml.jackson.dataformat</groupId>
<artifactId>jackson-dataformat-csv</artifactId>
</exclusion>
<exclusion>
<groupId>commons-io</groupId>
<artifactId>commons-io</artifactId>
</exclusion>
</exclusions>
</dependency>

Expand Down Expand Up @@ -1935,6 +1954,14 @@
<groupId>org.slf4j</groupId>
<artifactId>slf4j-log4j12</artifactId>
</exclusion>
<exclusion>
<groupId>ch.qos.logback</groupId>
<artifactId>*</artifactId>
</exclusion>
<exclusion>
<groupId>io.netty</groupId>
<artifactId>*</artifactId>
</exclusion>
</exclusions>
</dependency>

Expand Down Expand Up @@ -1963,9 +1990,9 @@
</dependency>

<dependency>
<groupId>io.prestodb.tempto</groupId>
<groupId>com.github.adkharat.tempto</groupId>
<artifactId>tempto-core</artifactId>
<version>${dep.tempto.version}</version>
<version>upgrade_kafka_client_version_cve_2022_34917-SNAPSHOT</version>
<exclusions>
<exclusion>
<groupId>com.datastax.cassandra</groupId>
Expand All @@ -1979,9 +2006,9 @@
</dependency>

<dependency>
<groupId>io.prestodb.tempto</groupId>
<groupId>com.github.adkharat.tempto</groupId>
<artifactId>tempto-ldap</artifactId>
<version>${dep.tempto.version}</version>
<version>upgrade_kafka_client_version_cve_2022_34917-SNAPSHOT</version>
<exclusions>
<exclusion>
<groupId>com.datastax.cassandra</groupId>
Expand All @@ -1991,9 +2018,9 @@
</dependency>

<dependency>
<groupId>io.prestodb.tempto</groupId>
<groupId>com.github.adkharat.tempto</groupId>
<artifactId>tempto-kafka</artifactId>
<version>${dep.tempto.version}</version>
<version>upgrade_kafka_client_version_cve_2022_34917-SNAPSHOT</version>
<exclusions>
<exclusion>
<groupId>com.datastax.cassandra</groupId>
Expand All @@ -2003,13 +2030,17 @@
<groupId>org.slf4j</groupId>
<artifactId>slf4j-log4j12</artifactId>
</exclusion>
<exclusion>
<groupId>org.apache.zookeeper</groupId>
<artifactId>zookeeper</artifactId>
</exclusion>
</exclusions>
</dependency>

<dependency>
<groupId>io.prestodb.tempto</groupId>
<groupId>com.github.adkharat.tempto</groupId>
<artifactId>tempto-runner</artifactId>
<version>${dep.tempto.version}</version>
<version>upgrade_kafka_client_version_cve_2022_34917-SNAPSHOT</version>
<exclusions>
<exclusion>
<groupId>com.datastax.cassandra</groupId>
Expand Down
28 changes: 28 additions & 0 deletions presto-kafka/pom.xml
Original file line number Diff line number Diff line change
Expand Up @@ -18,6 +18,11 @@
</properties>

<dependencies>
<dependency>
<groupId>org.slf4j</groupId>
<artifactId>slf4j-api</artifactId>
</dependency>

<dependency>
<groupId>com.facebook.airlift</groupId>
<artifactId>bootstrap</artifactId>
Expand Down Expand Up @@ -234,6 +239,19 @@
<version>${scala.version}</version>
<scope>test</scope>
</dependency>

<dependency>
<groupId>org.apache.curator</groupId>
<artifactId>curator-test</artifactId>
<version>5.6.0</version>
<scope>test</scope>
<exclusions>
<exclusion>
<groupId>org.junit.jupiter</groupId>
<artifactId>junit-jupiter-api</artifactId>
</exclusion>
</exclusions>
</dependency>
</dependencies>

<build>
Expand All @@ -248,6 +266,16 @@
</excludes>
</configuration>
</plugin>
<plugin>
<groupId>org.basepom.maven</groupId>
<artifactId>duplicate-finder-maven-plugin</artifactId>
<configuration>
<ignoredResourcePatterns>
<ignoredResourcePattern>kafka/kafka-version.properties</ignoredResourcePattern>
<ignoredResourcePattern>about.html</ignoredResourcePattern>
</ignoredResourcePatterns>
</configuration>
</plugin>
</plugins>
</build>

Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -16,19 +16,22 @@
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.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;
Expand All @@ -45,27 +48,28 @@
public class EmbeddedKafka
implements Closeable
{
private final EmbeddedZookeeper zookeeper;
private final ZooKeeperEmbedded 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();

public static EmbeddedKafka createEmbeddedKafka()
throws IOException
throws Exception
{
return new EmbeddedKafka(new EmbeddedZookeeper(), new Properties());
return new EmbeddedKafka(new ZooKeeperEmbedded(findUnusedPort()), new Properties());
}

public static EmbeddedKafka createEmbeddedKafka(Properties overrideProperties)
throws IOException
throws Exception
{
return new EmbeddedKafka(new EmbeddedZookeeper(), overrideProperties);
return new EmbeddedKafka(new ZooKeeperEmbedded(findUnusedPort()), overrideProperties);
}

EmbeddedKafka(EmbeddedZookeeper zookeeper, Properties overrideProperties)
EmbeddedKafka(ZooKeeperEmbedded zookeeper, Properties overrideProperties)
throws IOException
{
this.zookeeper = requireNonNull(zookeeper, "zookeeper is null");
Expand All @@ -77,6 +81,7 @@ public static EmbeddedKafka createEmbeddedKafka(Properties overrideProperties)
Map<String, String> properties = ImmutableMap.<String, String>builder()
.put("broker.id", "0")
.put("host.name", "localhost")
.put("listeners", "PLAINTEXT://localhost:" + getPort())
.put("num.partitions", "2")
.put("log.flush.interval.messages", "10000")
.put("log.flush.interval.ms", "1000")
Expand All @@ -85,20 +90,24 @@ public static EmbeddedKafka createEmbeddedKafka(Properties overrideProperties)
.put("zookeeper.connection.timeout.ms", "1000000")
.put("port", Integer.toString(port))
.put("log.dirs", kafkaDataDir.getAbsolutePath())
.put("zookeeper.connect", zookeeper.getConnectString())
.put("zookeeper.connect", zookeeper.connectString())
.put("offsets.topic.replication.factor", "1")
.putAll(Maps.fromProperties(overrideProperties))
.build();

KafkaConfig config = new KafkaConfig(toProperties(properties));
this.kafka = new KafkaServerStartable(config);
Time time = Time.SYSTEM;
this.kafka = new KafkaServer(config, time, scala.Option.empty(), false);
Properties adminProps = new Properties();
adminProps.put(AdminClientConfig.BOOTSTRAP_SERVERS_CONFIG, getConnectString());
adminProps.put("log.level", "DEBUG");
this.adminClient = AdminClient.create(adminProps);
}

public void start()
throws InterruptedException, IOException
{
if (!started.getAndSet(true)) {
zookeeper.start();
kafka.startup();
}
}
Expand All @@ -110,8 +119,9 @@ public void close()
if (started.get() && !stopped.getAndSet(true)) {
kafka.shutdown();
kafka.awaitShutdown();
zookeeper.close();
zookeeper.stop();
deleteRecursively(kafkaDataDir.toPath(), ALLOW_INSECURE);
adminClient.close();
}
}

Expand All @@ -123,15 +133,15 @@ public void createTopics(String... topics)
public void createTopics(int partitions, int replication, Properties topicProperties, String... topics)
{
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);
}
}

Expand All @@ -146,11 +156,6 @@ public KafkaProducer<Long, Object> createProducer()
return new KafkaProducer<>(properties);
}

public int getZookeeperPort()
{
return zookeeper.getPort();
}

public int getPort()
{
return port;
Expand All @@ -163,6 +168,6 @@ public String getConnectString()

public String getZookeeperConnectString()
{
return zookeeper.getConnectString();
return zookeeper.connectString();
}
}
Original file line number Diff line number Diff line change
@@ -0,0 +1,66 @@
/*
* Licensed under the Apache License, Version 2.0 (the "License");
* you may not use this file except in compliance with the License.
* You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS,
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
* See the License for the specific language governing permissions and
* limitations under the License.
*/
package com.facebook.presto.kafka.util;

import org.apache.curator.test.TestingServer;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;

import java.io.IOException;

public class ZooKeeperEmbedded
{
private static final Logger log = LoggerFactory.getLogger(ZooKeeperEmbedded.class);

private final TestingServer server;

/**
* Starts a ZooKeeper instance that listens at the defined port.
*
* @param port The port (aka `clientPort`) to listen to. Default: 2181.
* @throws Exception
*/
public ZooKeeperEmbedded(int port) throws Exception
{
log.debug("Starting embedded ZooKeeper server on port {} ...", port);
this.server = new TestingServer(port);
}

public void stop() throws IOException
{
log.debug("Shutting down embedded ZooKeeper server at {} ...", server.getConnectString());
server.close();
log.debug("Shutdown of embedded ZooKeeper server at {} completed", server.getConnectString());
}

/**
* The ZooKeeper connection string aka `zookeeper.connect` in `hostnameOrIp:port` format.
* Example: `127.0.0.1:2181`.
*
* You can use this to e.g. tell Kafka brokers how to connect to this instance.
*/
public String connectString()
{
return server.getConnectString();
}

/**
* The hostname of the ZooKeeper instance. Example: `127.0.0.1`
*/
public String hostname()
{
// "server:1:2:3" -> "server:1:2"
return connectString().substring(0, connectString().lastIndexOf(':'));
}
}
Loading
Loading