Skip to content

Commit 7b51e49

Browse files
authored
Fix PostCommit Java Nexmark Dataflow job (#33979)
* Synchronize the singleton ServiceLoader to avoid thread races
1 parent 53080f1 commit 7b51e49

File tree

1 file changed

+51
-44
lines changed

1 file changed

+51
-44
lines changed

sdks/java/core/src/main/java/org/apache/beam/sdk/schemas/utils/ConvertHelpers.java

+51-44
Original file line numberDiff line numberDiff line change
@@ -23,6 +23,7 @@
2323
import java.lang.reflect.InvocationTargetException;
2424
import java.lang.reflect.Type;
2525
import java.util.ServiceLoader;
26+
import javax.annotation.concurrent.GuardedBy;
2627
import net.bytebuddy.ByteBuddy;
2728
import net.bytebuddy.asm.AsmVisitorWrapper;
2829
import net.bytebuddy.description.type.TypeDescription;
@@ -57,12 +58,15 @@
5758
"rawtypes"
5859
})
5960
public class ConvertHelpers {
61+
6062
private static class SchemaInformationProviders {
63+
@GuardedBy("lock")
6164
private static final ServiceLoader<SchemaInformationProvider> INSTANCE =
6265
ServiceLoader.load(SchemaInformationProvider.class);
6366
}
6467

6568
private static final Logger LOG = LoggerFactory.getLogger(ConvertHelpers.class);
69+
private static final Object lock = new Object();
6670

6771
/** Return value after converting a schema. */
6872
public static class ConvertedSchemaInformation<T> implements Serializable {
@@ -85,57 +89,60 @@ public ConvertedSchemaInformation(
8589
public static <T> ConvertedSchemaInformation<T> getConvertedSchemaInformation(
8690
Schema inputSchema, TypeDescriptor<T> outputType, SchemaRegistry schemaRegistry) {
8791

88-
ConvertedSchemaInformation<T> schemaInformation = null;
8992
// Try to load schema information from loaded providers
90-
for (SchemaInformationProvider provider : SchemaInformationProviders.INSTANCE) {
91-
schemaInformation = provider.getConvertedSchemaInformation(inputSchema, outputType);
92-
if (schemaInformation != null) {
93-
return schemaInformation;
93+
try {
94+
synchronized (lock) {
95+
for (SchemaInformationProvider provider : SchemaInformationProviders.INSTANCE) {
96+
ConvertedSchemaInformation<T> schemaInformation =
97+
provider.getConvertedSchemaInformation(inputSchema, outputType);
98+
if (schemaInformation != null) {
99+
return schemaInformation;
100+
}
101+
}
94102
}
103+
} catch (Exception e) {
104+
LOG.debug("No Schema information from loaded providers found for type {}", outputType, e);
95105
}
96106

97-
if (schemaInformation == null) {
98-
// Otherwise, try to find a schema for the output type in the schema registry.
99-
Schema outputSchema = null;
100-
SchemaCoder<T> outputSchemaCoder = null;
101-
try {
102-
outputSchema = schemaRegistry.getSchema(outputType);
103-
outputSchemaCoder =
104-
SchemaCoder.of(
105-
outputSchema,
106-
outputType,
107-
schemaRegistry.getToRowFunction(outputType),
108-
schemaRegistry.getFromRowFunction(outputType));
109-
} catch (NoSuchSchemaException e) {
110-
LOG.debug("No schema found for type " + outputType, e);
111-
}
112-
FieldType unboxedType = null;
113-
// TODO: Properly handle nullable.
114-
if (outputSchema == null || !outputSchema.assignableToIgnoreNullable(inputSchema)) {
115-
// The schema is not convertible directly. Attempt to unbox it and see if the schema matches
116-
// then.
117-
Schema checkedSchema = inputSchema;
118-
if (inputSchema.getFieldCount() == 1) {
119-
unboxedType = inputSchema.getField(0).getType();
120-
if (unboxedType.getTypeName().isCompositeType()
121-
&& !outputSchema.assignableToIgnoreNullable(unboxedType.getRowSchema())) {
122-
checkedSchema = unboxedType.getRowSchema();
123-
} else {
124-
checkedSchema = null;
125-
}
126-
}
127-
if (checkedSchema != null) {
128-
throw new RuntimeException(
129-
"Cannot convert between types that don't have equivalent schemas."
130-
+ " input schema: "
131-
+ checkedSchema
132-
+ " output schema: "
133-
+ outputSchema);
107+
// Otherwise, try to find a schema for the output type in the schema registry.
108+
Schema outputSchema = null;
109+
SchemaCoder<T> outputSchemaCoder = null;
110+
try {
111+
outputSchema = schemaRegistry.getSchema(outputType);
112+
outputSchemaCoder =
113+
SchemaCoder.of(
114+
outputSchema,
115+
outputType,
116+
schemaRegistry.getToRowFunction(outputType),
117+
schemaRegistry.getFromRowFunction(outputType));
118+
} catch (NoSuchSchemaException e) {
119+
LOG.debug("No schema found for type {}", outputType, e);
120+
}
121+
FieldType unboxedType = null;
122+
// TODO: Properly handle nullable.
123+
if (outputSchema == null || !outputSchema.assignableToIgnoreNullable(inputSchema)) {
124+
// The schema is not convertible directly. Attempt to unbox it and see if the schema matches
125+
// then.
126+
Schema checkedSchema = inputSchema;
127+
if (inputSchema.getFieldCount() == 1) {
128+
unboxedType = inputSchema.getField(0).getType();
129+
if (unboxedType.getTypeName().isCompositeType()
130+
&& !outputSchema.assignableToIgnoreNullable(unboxedType.getRowSchema())) {
131+
checkedSchema = unboxedType.getRowSchema();
132+
} else {
133+
checkedSchema = null;
134134
}
135135
}
136-
schemaInformation = new ConvertedSchemaInformation<T>(outputSchemaCoder, unboxedType);
136+
if (checkedSchema != null) {
137+
throw new RuntimeException(
138+
"Cannot convert between types that don't have equivalent schemas."
139+
+ " input schema: "
140+
+ checkedSchema
141+
+ " output schema: "
142+
+ outputSchema);
143+
}
137144
}
138-
return schemaInformation;
145+
return new ConvertedSchemaInformation<>(outputSchemaCoder, unboxedType);
139146
}
140147

141148
/**

0 commit comments

Comments
 (0)