Skip to content
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

[HUDI-3287] Remove hudi-spark dependencies from hudi-kafka-connect-bundle #6079

Merged
merged 2 commits into from
Oct 28, 2022
Merged
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
Original file line number Diff line number Diff line change
Expand Up @@ -54,7 +54,7 @@ public static KeyGenerator createKeyGenerator(TypedProperties props) throws IOEx
return Objects.isNull(keyGenerator) ? createAvroKeyGeneratorByType(props) : keyGenerator;
}

private static KeyGenerator createAvroKeyGeneratorByType(TypedProperties props) throws IOException {
public static KeyGenerator createAvroKeyGeneratorByType(TypedProperties props) throws IOException {
// Use KeyGeneratorType.SIMPLE as default keyGeneratorType
String keyGeneratorType =
props.getString(HoodieWriteConfig.KEYGENERATOR_TYPE.key(), null);
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -18,8 +18,6 @@

package org.apache.hudi.connect.utils;

import com.google.protobuf.ByteString;
import org.apache.hadoop.conf.Configuration;
import org.apache.hudi.client.WriteStatus;
import org.apache.hudi.common.config.TypedProperties;
import org.apache.hudi.common.model.HoodieCommitMetadata;
Expand All @@ -36,9 +34,11 @@
import org.apache.hudi.exception.HoodieException;
import org.apache.hudi.keygen.BaseKeyGenerator;
import org.apache.hudi.keygen.CustomAvroKeyGenerator;
import org.apache.hudi.keygen.CustomKeyGenerator;
import org.apache.hudi.keygen.KeyGenerator;
import org.apache.hudi.keygen.constant.KeyGeneratorOptions;

import com.google.protobuf.ByteString;
import org.apache.hadoop.conf.Configuration;
import org.apache.kafka.clients.admin.AdminClient;
import org.apache.kafka.clients.admin.DescribeTopicsResult;
import org.apache.kafka.clients.admin.TopicDescription;
Expand Down Expand Up @@ -185,8 +185,7 @@ public static String getRecordKeyColumns(KeyGenerator keyGenerator) {
* @return partition columns Returns the partition columns separated by comma.
*/
public static String getPartitionColumns(KeyGenerator keyGenerator, TypedProperties typedProperties) {

if (keyGenerator instanceof CustomKeyGenerator || keyGenerator instanceof CustomAvroKeyGenerator) {
if (keyGenerator instanceof CustomAvroKeyGenerator) {
return ((BaseKeyGenerator) keyGenerator).getPartitionPathFields().stream().map(
pathField -> Arrays.stream(pathField.split(CustomAvroKeyGenerator.SPLIT_REGEX))
.findFirst().orElse("Illegal partition path field format: '$pathField' for ${c.getClass.getSimpleName}"))
Expand All @@ -200,7 +199,6 @@ public static String getPartitionColumns(KeyGenerator keyGenerator, TypedPropert
return typedProperties.getString(KeyGeneratorOptions.PARTITIONPATH_FIELD_NAME.key());
}


/**
* Get the Metadata from the latest commit file.
*
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -84,7 +84,7 @@ public KafkaConnectTransactionServices(KafkaConnectConfigs connectConfigs) throw
context = new HoodieJavaEngineContext(hadoopConf);

try {
KeyGenerator keyGenerator = HoodieAvroKeyGeneratorFactory.createKeyGenerator(
KeyGenerator keyGenerator = HoodieAvroKeyGeneratorFactory.createAvroKeyGeneratorByType(
new TypedProperties(connectConfigs.getProps()));
String recordKeyFields = KafkaConnectUtils.getRecordKeyColumns(keyGenerator);
String partitionColumns = KafkaConnectUtils.getPartitionColumns(keyGenerator,
Expand Down
7 changes: 0 additions & 7 deletions packaging/hudi-kafka-connect-bundle/pom.xml
Original file line number Diff line number Diff line change
Expand Up @@ -75,8 +75,6 @@
<include>org.apache.hudi:hudi-common</include>
<include>org.apache.hudi:hudi-client-common</include>
<include>org.apache.hudi:hudi-java-client</include>
<include>org.apache.hudi:hudi-spark-client</include>
<include>org.apache.hudi:hudi-spark-common_${scala.binary.version}</include>
<include>org.apache.hudi:hudi-kafka-connect</include>
<include>org.apache.hudi:hudi-utilities_${scala.binary.version}</include>
<include>org.apache.hudi:hudi-hive-sync</include>
Expand Down Expand Up @@ -322,11 +320,6 @@
</exclusion>
</exclusions>
</dependency>
<dependency>
<groupId>org.apache.hudi</groupId>
<artifactId>hudi-spark-common_${scala.binary.version}</artifactId>
<version>${project.version}</version>
</dependency>

<!-- Avro/ Parquet -->
<dependency>
Expand Down