yanghua commented on a change in pull request #1886:
URL: https://github.com/apache/hudi/pull/1886#discussion_r464145643



##########
File path: 
hudi-client/src/main/java/org/apache/hudi/callback/util/HoodieWriteCommitCallbackUtil.java
##########
@@ -0,0 +1,47 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements. See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You 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 org.apache.hudi.callback.util;
+
+import org.apache.hudi.exception.HoodieCommitCallbackException;
+
+import org.codehaus.jackson.map.ObjectMapper;
+
+import java.io.IOException;
+
+/**
+ * Util helps to prepare callback message.
+ */
+public class HoodieWriteCommitCallbackUtil {
+
+  /**
+   * Convert data to json string format.
+   *
+   * @param obj

Review comment:
       If you do not want to add the comments for `@param` and `@return`. Just 
remove them.

##########
File path: 
hudi-utilities/src/main/java/org/apache/hudi/utilities/callback/kafka/HoodieWriteCommitKafkaCallback.java
##########
@@ -0,0 +1,67 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements. See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You 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 org.apache.hudi.utilities.callback.kafka;
+
+import org.apache.hudi.callback.HoodieWriteCommitCallback;
+import org.apache.hudi.callback.common.HoodieWriteCommitCallbackMessage;
+import org.apache.hudi.callback.util.HoodieWriteCommitCallbackUtil;
+import org.apache.hudi.config.HoodieWriteConfig;
+
+import org.apache.kafka.clients.producer.Callback;
+import org.apache.kafka.clients.producer.KafkaProducer;
+import org.apache.kafka.clients.producer.ProducerRecord;
+import org.apache.kafka.clients.producer.RecordMetadata;
+import org.apache.log4j.LogManager;
+import org.apache.log4j.Logger;
+
+import java.util.Properties;
+
+/**
+ * Kafka implementation of {@link HoodieWriteCommitCallback}.
+ */
+public class HoodieWriteCommitKafkaCallback implements 
HoodieWriteCommitCallback {
+
+  private static final Logger LOG = 
LogManager.getLogger(HoodieWriteCommitKafkaCallback.class);
+
+  private KafkaProducer<String, String> producer;
+  private Properties props;
+
+  public HoodieWriteCommitKafkaCallback(HoodieWriteConfig config) {
+    this.props = config.getProps();
+    this.producer = HoodieWriteCommitKafkaCallbackUtil.createProducer(props);
+  }
+
+  @Override
+  public void call(HoodieWriteCommitCallbackMessage callbackMessage) {
+    String callbackMsg = 
HoodieWriteCommitCallbackUtil.convertToJsonString(callbackMessage);
+    ProducerRecord<String, String> record = 
HoodieWriteCommitKafkaCallbackUtil.buildProducerRecord(props, callbackMsg);
+    producer.send(record, new ProducerSendCallback());
+    producer.close();

Review comment:
       Shall we use `try-with-resource` or `try-catch` block to prevent leaking 
resources?

##########
File path: 
hudi-utilities/src/main/java/org/apache/hudi/utilities/callback/kafka/HoodieWriteCommitKafkaCallbackUtil.java
##########
@@ -0,0 +1,79 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements. See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You 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 org.apache.hudi.utilities.callback.kafka;
+
+import org.apache.hudi.callback.util.HoodieWriteCommitCallbackUtil;
+import org.apache.hudi.common.util.StringUtils;
+import org.apache.hudi.common.util.ValidationUtils;
+
+import org.apache.kafka.clients.producer.KafkaProducer;
+import org.apache.kafka.clients.producer.ProducerConfig;
+import org.apache.kafka.clients.producer.ProducerRecord;
+import org.apache.log4j.LogManager;
+import org.apache.log4j.Logger;
+
+import java.util.Properties;
+
+import static 
org.apache.hudi.config.HoodieWriteCommitCallbackConfig.CALLBACK_KAFKA_ACKS;
+import static 
org.apache.hudi.config.HoodieWriteCommitCallbackConfig.CALLBACK_KAFKA_BOOTSTRAP_SERVERS;
+import static 
org.apache.hudi.config.HoodieWriteCommitCallbackConfig.CALLBACK_KAFKA_PARTITION;
+import static 
org.apache.hudi.config.HoodieWriteCommitCallbackConfig.CALLBACK_KAFKA_RETRIES;
+import static 
org.apache.hudi.config.HoodieWriteCommitCallbackConfig.CALLBACK_KAFKA_TOPIC;
+import static org.apache.hudi.config.HoodieWriteConfig.TABLE_NAME;
+
+/**
+ * Util helps to build kafka callback.
+ */
+public class HoodieWriteCommitKafkaCallbackUtil {

Review comment:
       IMO, moving the content of this class into 
`HoodieWriteCommitKafkaCallback ` is reasonable. WDYT?

##########
File path: 
hudi-client/src/main/java/org/apache/hudi/config/HoodieWriteCommitCallbackConfig.java
##########
@@ -91,13 +103,44 @@ public Builder withCallbackHttpApiKey(String apiKey) {
       return this;
     }
 
+    public Builder withCallbackKafkaBootstrapServers(String 
kafkaBootstrapServers) {

Review comment:
       I am thinking one thing: the config options of the callback feature for 
Kafka are left in `hudi-client`, while the implementation hosts in 
`hudi-utilities`, if it's suitable.




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
[email protected]


Reply via email to