Skip to content

Commit a13548a

Browse files
author
rajuGT
committed
Handles google.protobuf.Value fields by delegating to a StringHandler
* There are two concerns with google.protobuf.Value type * 1. It has cyclic reference - https://github.com/protocolbuffers/protobuf/blob/main/src/google/protobuf/struct.proto#L94 * 2. oneof usage; i.e. It has no specific type information, every message can be of different type * - https://github.com/protocolbuffers/protobuf/blob/main/src/google/protobuf/struct.proto#L64 * * <p>This handler treats {@link Value} as a plain string: * - Converts between String and Value * - Ignores nested structures (STRUCT_VALUE, LIST_VALUE) * - Useful as a lightweight workaround to recursion in google.protobuf.Value
1 parent 9f841cc commit a13548a

File tree

3 files changed

+154
-1
lines changed

3 files changed

+154
-1
lines changed

dagger-common/src/main/java/com/gotocompany/dagger/common/serde/typehandler/TypeHandlerFactory.java

Lines changed: 2 additions & 0 deletions
Original file line numberDiff line numberDiff line change
@@ -6,6 +6,7 @@
66
import com.gotocompany.dagger.common.serde.typehandler.repeated.RepeatedPrimitiveHandler;
77
import com.gotocompany.dagger.common.serde.typehandler.repeated.RepeatedStructMessageHandler;
88
import com.gotocompany.dagger.common.serde.typehandler.complex.EnumHandler;
9+
import com.gotocompany.dagger.common.serde.typehandler.complex.GoogleProtobufValueHandler;
910
import com.gotocompany.dagger.common.serde.typehandler.complex.MapHandler;
1011
import com.gotocompany.dagger.common.serde.typehandler.complex.MessageHandler;
1112
import com.gotocompany.dagger.common.serde.typehandler.complex.StructMessageHandler;
@@ -69,6 +70,7 @@ private static List<TypeHandler> getSpecificHandlers(Descriptors.FieldDescriptor
6970
new RepeatedPrimitiveHandler(fieldDescriptor),
7071
new RepeatedMessageHandler(fieldDescriptor),
7172
new RepeatedEnumHandler(fieldDescriptor),
73+
new GoogleProtobufValueHandler(fieldDescriptor),
7274
new MessageHandler(fieldDescriptor)
7375
);
7476
}
Lines changed: 149 additions & 0 deletions
Original file line numberDiff line numberDiff line change
@@ -0,0 +1,149 @@
1+
package com.gotocompany.dagger.common.serde.typehandler.complex;
2+
3+
import com.gotocompany.dagger.common.core.FieldDescriptorCache;
4+
import com.gotocompany.dagger.common.serde.typehandler.TypeHandler;
5+
6+
import com.google.protobuf.Descriptors;
7+
import com.google.protobuf.DynamicMessage;
8+
import com.google.protobuf.Value;
9+
import com.google.protobuf.util.JsonFormat;
10+
11+
import com.gotocompany.dagger.common.serde.typehandler.primitive.StringHandler;
12+
import org.apache.flink.api.common.typeinfo.TypeInformation;
13+
import org.apache.flink.api.common.typeinfo.Types;
14+
import org.apache.parquet.example.data.simple.SimpleGroup;
15+
16+
import java.io.IOException;
17+
18+
/**
19+
* Handles google.protobuf.Value fields by delegating to a {@link StringHandler}.
20+
* There are two concerns with Value type
21+
* 1. It has cyclic reference - https://github.com/protocolbuffers/protobuf/blob/main/src/google/protobuf/struct.proto#L94
22+
* 2. oneof usage; i.e. It has no specific type information, every message can be of different type
23+
* - https://github.com/protocolbuffers/protobuf/blob/main/src/google/protobuf/struct.proto#L64
24+
*
25+
* <p>This handler treats {@link Value} as a plain string:
26+
* - Converts between String and Value
27+
* - Ignores nested structures (STRUCT_VALUE, LIST_VALUE)
28+
* - Useful as a lightweight workaround to recursion in google.protobuf.Value
29+
*/
30+
public class GoogleProtobufValueHandler implements TypeHandler {
31+
32+
private final Descriptors.FieldDescriptor fieldDescriptor;
33+
private final StringHandler stringHandler;
34+
35+
/**
36+
* Instantiates a new Value handler with an internal {@link StringHandler}.
37+
*
38+
* @param fieldDescriptor the protobuf field descriptor
39+
*/
40+
public GoogleProtobufValueHandler(Descriptors.FieldDescriptor fieldDescriptor) {
41+
this.fieldDescriptor = fieldDescriptor;
42+
// Initialize a basic string handler for fallback operations
43+
this.stringHandler = new StringHandler(fieldDescriptor);
44+
}
45+
46+
@Override
47+
public boolean canHandle() {
48+
return fieldDescriptor.getJavaType() == Descriptors.FieldDescriptor.JavaType.MESSAGE &&
49+
"google.protobuf.Value".equals(fieldDescriptor.getMessageType().getFullName());
50+
}
51+
52+
@Override
53+
public DynamicMessage.Builder transformToProtoBuilder(DynamicMessage.Builder builder, Object field) {
54+
if (!canHandle() || field == null) {
55+
return builder;
56+
}
57+
58+
Value valueMessage;
59+
Object parsedField = stringHandler.parseObject(field);
60+
61+
if (parsedField instanceof String) {
62+
String stringValue = (String) parsedField;
63+
try {
64+
// Attempt JSON parsing
65+
Value.Builder valBuilder = Value.newBuilder();
66+
JsonFormat.parser().merge(stringValue, valBuilder);
67+
valueMessage = valBuilder.build();
68+
} catch (IOException e) {
69+
// Not JSON — store as plain string
70+
valueMessage = Value.newBuilder().setStringValue(stringValue).build();
71+
}
72+
} else {
73+
// Fallback: stringify the object
74+
valueMessage = Value.newBuilder().setStringValue(parsedField.toString()).build();
75+
}
76+
77+
builder.setField(fieldDescriptor, valueMessage);
78+
return builder;
79+
}
80+
81+
@Override
82+
public Object transformFromPostProcessor(Object field) {
83+
return field == null ? "" : field.toString();
84+
}
85+
86+
@Override
87+
public Object transformFromProto(Object field) {
88+
if (field instanceof Value) {
89+
Value value = (Value) field;
90+
return extractString(value);
91+
} else if (field instanceof DynamicMessage) {
92+
// Convert DynamicMessage back to Value then to string
93+
try {
94+
Value.Builder builder = Value.newBuilder();
95+
JsonFormat.parser().merge(JsonFormat.printer().print((DynamicMessage) field), builder);
96+
return extractString(builder.build());
97+
} catch (Exception e) {
98+
return field.toString();
99+
}
100+
}
101+
return "";
102+
}
103+
104+
@Override
105+
public Object transformFromProtoUsingCache(Object field, FieldDescriptorCache cache) {
106+
return transformFromProto(field);
107+
}
108+
109+
@Override
110+
public Object transformFromParquet(SimpleGroup simpleGroup) {
111+
return stringHandler.parseSimpleGroup(simpleGroup);
112+
}
113+
114+
@Override
115+
public Object transformToJson(Object field) {
116+
return field == null ? "" : field.toString();
117+
}
118+
119+
@Override
120+
public TypeInformation getTypeInformation() {
121+
return Types.STRING;
122+
}
123+
124+
/**
125+
* Extracts a string representation of a {@link Value} protobuf.
126+
*/
127+
private String extractString(Value value) {
128+
switch (value.getKindCase()) {
129+
case STRING_VALUE:
130+
return value.getStringValue();
131+
case NUMBER_VALUE:
132+
return String.valueOf(value.getNumberValue());
133+
case BOOL_VALUE:
134+
return String.valueOf(value.getBoolValue());
135+
case NULL_VALUE:
136+
return "null";
137+
case STRUCT_VALUE:
138+
case LIST_VALUE:
139+
try {
140+
return JsonFormat.printer().print(value);
141+
} catch (IOException e) {
142+
return value.toString();
143+
}
144+
case KIND_NOT_SET:
145+
default:
146+
return "";
147+
}
148+
}
149+
}

dagger-common/src/main/java/com/gotocompany/dagger/common/serde/typehandler/complex/MessageHandler.java

Lines changed: 3 additions & 1 deletion
Original file line numberDiff line numberDiff line change
@@ -45,7 +45,9 @@ public MessageHandler(FieldDescriptor fieldDescriptor) {
4545

4646
@Override
4747
public boolean canHandle() {
48-
return fieldDescriptor.getJavaType() == MESSAGE && !fieldDescriptor.getMessageType().getFullName().equals("google.protobuf.Timestamp");
48+
return fieldDescriptor.getJavaType() == MESSAGE
49+
&& !fieldDescriptor.getMessageType().getFullName().equals("google.protobuf.Timestamp")
50+
&& !fieldDescriptor.getMessageType().getFullName().equals("google.protobuf.Value");
4951
}
5052

5153
@Override

0 commit comments

Comments
 (0)