codope commented on code in PR #11070:
URL: https://github.com/apache/hudi/pull/11070#discussion_r1594942440
##########
hudi-utilities/src/test/java/org/apache/hudi/utilities/sources/TestProtoKafkaSource.java:
##########
@@ -181,7 +218,7 @@ private static List<Sample> createSampleMessages(int count)
{
.setWrappedDouble(DoubleValue.of(RANDOM.nextDouble()))
.setWrappedFloat(FloatValue.of(RANDOM.nextFloat()))
.setWrappedBoolean(BoolValue.of(RANDOM.nextBoolean()))
-
.setWrappedBytes(BytesValue.of(ByteString.copyFrom(getUTF8Bytes(UUID.randomUUID().toString()))))
+
.setWrappedBytes(BytesValue.of(ByteString.copyFrom(UUID.randomUUID().toString().getBytes())))
Review Comment:
why removing the `getUTF8Bytes`?
##########
hudi-utilities/src/test/java/org/apache/hudi/utilities/sources/TestProtoKafkaSource.java:
##########
@@ -198,22 +235,48 @@ private static Nested generateRandomNestedMessage() {
@Override
void sendMessagesToKafka(String topic, int count, int numPartitions) {
List<Sample> messages = createSampleMessages(count);
- try (Producer<String, byte[]> producer = new
KafkaProducer<>(getProducerProperties())) {
+ try (Producer<String, byte[]> producer = new
KafkaProducer<>(getProducerProperties(false))) {
for (int i = 0; i < messages.size(); i++) {
// use consistent keys to get even spread over partitions for test
expectations
producer.send(new ProducerRecord<>(topic, Integer.toString(i %
numPartitions), messages.get(i).toByteArray()));
}
}
}
- private Properties getProducerProperties() {
+ private void sendMessagesToKafkaWithConfluentSerializer(String topic, int
numPartitions, List<Sample> messages) {
+ try (Producer<String, Message> producer = new
KafkaProducer<>(getProducerProperties(true))) {
+ for (int i = 0; i < messages.size(); i++) {
+ // use consistent keys to get even spread over partitions for test
expectations
+ producer.send(new ProducerRecord<>(topic, Integer.toString(i %
numPartitions), messages.get(i)));
+ }
+ }
+ }
+
+ private Properties getProducerProperties(boolean
useConfluentProtobufSerializer) {
Properties props = new Properties();
props.put("bootstrap.servers", testUtils.brokerAddress());
- props.put("value.serializer", ByteArraySerializer.class.getName());
- // Key serializer is required.
+ if (useConfluentProtobufSerializer) {
+ props.put("value.serializer",
+ "io.confluent.kafka.serializers.protobuf.KafkaProtobufSerializer");
+ props.put("value.deserializer",
+ "io.confluent.kafka.serializers.protobuf.KafkaProtobufDeserializer");
Review Comment:
Instead of hard-coding, how about `KafkaProtobufSerializer.class.getName()`?
Is it that we don't want to load the imports of external class?
##########
hudi-utilities/src/main/java/org/apache/hudi/utilities/deser/KafkaAvroSchemaDeserializer.java:
##########
@@ -70,13 +69,12 @@ public void configure(Map<String, ?> configs, boolean
isKey) {
*/
@Override
protected Object deserialize(
- boolean includeSchemaAndVersion,
String topic,
Boolean isKey,
byte[] payload,
Schema readerSchema)
throws SerializationException {
- return super.deserialize(includeSchemaAndVersion, topic, isKey, payload,
sourceSchema);
+ return super.deserialize(topic, isKey, payload, sourceSchema);
Review Comment:
Did we ever not include schema and version?
##########
hudi-utilities/src/main/java/org/apache/hudi/utilities/sources/helpers/ProtoConversionUtil.java:
##########
@@ -402,7 +438,21 @@ private static Object convertObject(Schema schema, Object
value) {
if (value instanceof Message) {
// check if this is a Timestamp
if
(LogicalTypes.timestampMicros().equals(schema.getLogicalType())) {
- return Timestamps.toMicros((Timestamp) value);
+ if (value instanceof Timestamp) {
+ return Timestamps.toMicros((Timestamp) value);
+ } else if (value instanceof DynamicMessage) {
+ Timestamp.Builder builder = Timestamp.newBuilder();
+ ((DynamicMessage)
value).getAllFields().forEach((fieldDescriptor, fieldValue) -> {
+ if
(fieldDescriptor.getFullName().equals("google.protobuf.Timestamp.seconds")) {
+ builder.setSeconds((Long) fieldValue);
+ } else if
(fieldDescriptor.getFullName().equals("google.protobuf.Timestamp.nanos")) {
+ builder.setNanos((Integer) fieldValue);
+ }
Review Comment:
Should it be other way around i.e. for Timestamp.seconds, cast to Integer
and Timestamp.nanos to Long?
##########
packaging/hudi-utilities-bundle/pom.xml:
##########
@@ -133,6 +133,7 @@
<include>io.confluent:common-config</include>
<include>io.confluent:common-utils</include>
<include>io.confluent:kafka-schema-registry-client</include>
+ <include>io.confluent:kafka-protobuf-serializer</include>
Review Comment:
Should we also include this in `hudi-integ-test-bundle`?
--
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.
To unsubscribe, e-mail: [email protected]
For queries about this service, please contact Infrastructure at:
[email protected]