-
Notifications
You must be signed in to change notification settings - Fork 28
New issue
Have a question about this project? Sign up for a free GitHub account to open an issue and contact its maintainers and the community.
By clicking “Sign up for GitHub”, you agree to our terms of service and privacy statement. We’ll occasionally send you account related emails.
Already on GitHub? Sign in to your account
upgraded_kafka_client_version_to_3.9.0 #279
base: master
Are you sure you want to change the base?
Changes from all commits
File filter
Filter by extension
Conversations
Jump to
Diff view
Diff view
There are no files selected for viewing
Original file line number | Diff line number | Diff line change |
---|---|---|
|
@@ -41,6 +41,7 @@ jar { | |
|
||
shadowJar { | ||
version = '' | ||
zip64 = true | ||
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. why is this required? There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. @ZacBlanco on gradle build, I am getting error
Here, ShadowJar plugin is creating a JAR file that exceeds the limit of 65,535 entries. This might due to the ZIP format limitation that ShadowJar uses by default. Link There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. Got it, thanks for the explanation. This is fine then I guess. We were probably close to the limit and the newer version pushed us over? It would be good to know the number of entries before/after this change, but I don't think it's worth trying to fix |
||
} | ||
|
||
build.dependsOn.add([shadowJar]) | ||
|
Original file line number | Diff line number | Diff line change |
---|---|---|
|
@@ -24,56 +24,54 @@ | |
import io.prestodb.tempto.internal.fulfillment.table.TableName; | ||
import io.prestodb.tempto.query.QueryExecutor; | ||
import io.prestodb.tempto.query.QueryResult; | ||
import kafka.admin.AdminUtils; | ||
import kafka.admin.RackAwareMode; | ||
import kafka.utils.ZKStringSerializer$; | ||
import kafka.utils.ZkUtils; | ||
import org.I0Itec.zkclient.ZkClient; | ||
import org.I0Itec.zkclient.ZkConnection; | ||
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.admin.DeleteTopicsResult; | ||
import org.apache.kafka.clients.producer.KafkaProducer; | ||
import org.apache.kafka.clients.producer.Producer; | ||
import org.apache.kafka.clients.producer.ProducerRecord; | ||
import org.slf4j.Logger; | ||
|
||
import javax.inject.Inject; | ||
import javax.inject.Named; | ||
import javax.inject.Singleton; | ||
|
||
import java.util.Collections; | ||
import java.util.Iterator; | ||
import java.util.Properties; | ||
import java.util.concurrent.ExecutionException; | ||
import java.util.function.Consumer; | ||
|
||
import static java.lang.String.format; | ||
import static java.util.Objects.requireNonNull; | ||
import static org.slf4j.LoggerFactory.getLogger; | ||
|
||
|
||
@TableManager.Descriptor(tableDefinitionClass = KafkaTableDefinition.class, type = "KAFKA") | ||
@Singleton | ||
public class KafkaTableManager | ||
implements TableManager<KafkaTableDefinition> | ||
{ | ||
private static final Logger LOGGER = getLogger(KafkaTableManager.class); | ||
private final String databaseName; | ||
private final QueryExecutor prestoQueryExecutor; | ||
private final String brokerHost; | ||
private final Integer brokerPort; | ||
private final String prestoKafkaCatalog; | ||
private final String zookeeperHost; | ||
private final Integer zookeeperPort; | ||
|
||
@Inject | ||
public KafkaTableManager( | ||
@Named("databaseName") String databaseName, | ||
@Named("broker.host") String brokerHost, | ||
@Named("broker.port") int brokerPort, | ||
@Named("zookeeper.host") String zookeeperHost, | ||
@Named("zookeeper.port") int zookeeperPort, | ||
@Named("presto_database_name") String prestoDatabaseName, | ||
@Named("presto_kafka_catalog") String prestoKafkaCatalog, | ||
Injector injector) | ||
{ | ||
this.databaseName = requireNonNull(databaseName, "databaseName is null"); | ||
this.brokerHost = requireNonNull(brokerHost, "brokerHost is null"); | ||
this.brokerPort = brokerPort; | ||
this.zookeeperHost = requireNonNull(zookeeperHost, "zookeeperHost is null"); | ||
this.zookeeperPort = zookeeperPort; | ||
requireNonNull(injector, "injector is null"); | ||
requireNonNull(prestoDatabaseName, "prestoDatabaseName is null"); | ||
this.prestoQueryExecutor = injector.getInstance(Key.get(QueryExecutor.class, Names.named(prestoDatabaseName))); | ||
|
@@ -95,6 +93,18 @@ public TableInstance<KafkaTableDefinition> createImmutable(KafkaTableDefinition | |
return new KafkaTableInstance(createdTableName, tableDefinition); | ||
} | ||
|
||
private void withKafkaAdminClient(Consumer<AdminClient> routine) { | ||
Properties properties = new Properties(); | ||
properties.put(AdminClientConfig.BOOTSTRAP_SERVERS_CONFIG, brokerHost + ":" + brokerPort); | ||
try (AdminClient adminClient = AdminClient.create(properties)) { | ||
routine.accept(adminClient); | ||
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. What would happen if there is an issue/exception while doing above? Should we add a catch and logger for it? There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. @agrawalreetika added catch to handle exception and logged error. |
||
} catch (Exception e) { | ||
LOGGER.error("An error occurred while performing an operation with Kafka AdminClient on broker " | ||
+ brokerHost + ":" + brokerPort); | ||
throw new RuntimeException("An error occurred while performing an operation with Kafka AdminClient", e); | ||
} | ||
} | ||
|
||
private void verifyTableExistsInPresto(String schema, String name) | ||
{ | ||
String sql = format("select count(1) from %s.information_schema.tables where table_schema='%s' and table_name='%s'", prestoKafkaCatalog, schema, name); | ||
|
@@ -104,34 +114,36 @@ private void verifyTableExistsInPresto(String schema, String name) | |
} | ||
} | ||
|
||
private void deleteTopic(String topic) | ||
{ | ||
withZookeeper(zkUtils -> { | ||
if (AdminUtils.topicExists(zkUtils, topic)) { | ||
AdminUtils.deleteTopic(zkUtils, topic); | ||
|
||
for (int checkTry = 0; checkTry < 5; ++checkTry) { | ||
if (!AdminUtils.topicExists(zkUtils, topic)) { | ||
return; | ||
} | ||
try { | ||
Thread.sleep(1_000); | ||
} | ||
catch (InterruptedException e) { | ||
Thread.currentThread().interrupt(); | ||
throw new RuntimeException("could not delete topic " + topic); | ||
} | ||
private void deleteTopic(String topic) { | ||
withKafkaAdminClient(adminClient -> { | ||
try { | ||
// Check if topic exists | ||
if (!adminClient.listTopics().names().get().contains(topic)) { | ||
LOGGER.warn("Topic {} does not exist. Skipping deletion.", topic); | ||
return; | ||
} | ||
throw new RuntimeException("could not delete topic " + topic); | ||
DeleteTopicsResult deleteTopicsResult = adminClient.deleteTopics(Collections.singletonList(topic)); | ||
deleteTopicsResult.all().get(); | ||
try { | ||
Thread.sleep(1_000); // Wait for metadata propagation | ||
} catch (InterruptedException e) { | ||
Thread.currentThread().interrupt(); | ||
throw new RuntimeException("Interrupted while waiting for topic deletion: " + topic, e); | ||
} | ||
} catch (ExecutionException | InterruptedException e) { | ||
throw new RuntimeException("Failed to delete topic " + topic, e); | ||
} | ||
}); | ||
} | ||
|
||
private void createTopic(String topic, int partitionsCount, int replicationLevel) | ||
{ | ||
withZookeeper(zkUtils -> { | ||
Properties topicConfiguration = new Properties(); | ||
AdminUtils.createTopic(zkUtils, topic, partitionsCount, replicationLevel, topicConfiguration, RackAwareMode.Disabled$.MODULE$); | ||
private void createTopic(String topic, int partitionsCount, int replicationLevel) { | ||
withKafkaAdminClient(adminClient -> { | ||
NewTopic newTopic = new NewTopic(topic, partitionsCount, (short) replicationLevel); | ||
try { | ||
adminClient.createTopics(Collections.singletonList(newTopic)).all().get(); | ||
} catch (ExecutionException | InterruptedException e) { | ||
throw new RuntimeException("Failed to create topic " + topic, e); | ||
} | ||
}); | ||
} | ||
|
||
|
@@ -164,25 +176,6 @@ private void insertDataIntoTopic(String topic, KafkaDataSource dataSource) | |
} | ||
} | ||
|
||
private void withZookeeper(Consumer<ZkUtils> routine) | ||
{ | ||
int sessionTimeOutInMs = 15_000; | ||
int connectionTimeOutInMs = 10_000; | ||
String zookeeperHosts = zookeeperHost + ":" + zookeeperPort; | ||
|
||
ZkClient zkClient = new ZkClient(zookeeperHosts, | ||
sessionTimeOutInMs, | ||
connectionTimeOutInMs, | ||
ZKStringSerializer$.MODULE$); | ||
try { | ||
ZkUtils zkUtils = new ZkUtils(zkClient, new ZkConnection(zookeeperHosts), false); | ||
routine.accept(zkUtils); | ||
} | ||
finally { | ||
zkClient.close(); | ||
} | ||
} | ||
|
||
@Override | ||
public TableInstance<KafkaTableDefinition> createMutable(KafkaTableDefinition tableDefinition, MutableTableRequirement.State state, TableHandle tableHandle) | ||
{ | ||
|
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
This should only be upgraded and set to a non-SNAPSHOT version during the release process. It looks like after the last release it wasn't properly updated. Since we already release 1.53, this should be
1.54-SNAPSHOT
. During release we will update to 1.54, and then move to 1.55-SNAPSHOT afterwards.So could you set this to 1.54-SNAPSHOT for now?
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
@ZacBlanco Done. Updated to 1.54-SNAPSHOT