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
[BEAM-9384] Add SchemaRegistry.getSchemaCoder to get SchemaCoders for registered types #10974
Changes from all commits
File filter
Filter by extension
Conversations
Jump to
Diff view
Diff view
There are no files selected for viewing
Original file line number | Diff line number | Diff line change |
---|---|---|
|
@@ -226,27 +226,16 @@ public <T> void registerJavaBean(TypeDescriptor<T> typeDescriptor) { | |
} | ||
|
||
/** | ||
* Get a schema for a given {@link Class} type. If no schema exists, throws {@link | ||
* Retrieve a {@link Schema} for a given {@link Class} type. If no schema exists, throws {@link | ||
* NoSuchSchemaException}. | ||
*/ | ||
public <T> Schema getSchema(Class<T> clazz) throws NoSuchSchemaException { | ||
return getSchema(TypeDescriptor.of(clazz)); | ||
} | ||
|
||
private <ReturnT> ReturnT getProviderResult(Function<SchemaProvider, ReturnT> f) | ||
throws NoSuchSchemaException { | ||
for (SchemaProvider provider : providers) { | ||
ReturnT result = f.apply(provider); | ||
if (result != null) { | ||
return result; | ||
} | ||
} | ||
throw new NoSuchSchemaException(); | ||
} | ||
|
||
/** | ||
* Retrieve a schema for a given {@link TypeDescriptor} type. If no schema exists, throws {@link | ||
* NoSuchSchemaException}. | ||
* Retrieve a {@link Schema} for a given {@link TypeDescriptor} type. If no schema exists, throws | ||
* {@link NoSuchSchemaException}. | ||
*/ | ||
public <T> Schema getSchema(TypeDescriptor<T> typeDescriptor) throws NoSuchSchemaException { | ||
SchemaEntry entry = entries.get(typeDescriptor); | ||
|
@@ -256,13 +245,17 @@ public <T> Schema getSchema(TypeDescriptor<T> typeDescriptor) throws NoSuchSchem | |
return getProviderResult((SchemaProvider p) -> p.schemaFor(typeDescriptor)); | ||
} | ||
|
||
/** Rerieve the function that converts an object of the specified type to a {@link Row} object. */ | ||
/** | ||
* Retrieve the function that converts an object of the specified type to a {@link Row} object. | ||
*/ | ||
public <T> SerializableFunction<T, Row> getToRowFunction(Class<T> clazz) | ||
throws NoSuchSchemaException { | ||
return getToRowFunction(TypeDescriptor.of(clazz)); | ||
} | ||
|
||
/** Rerieve the function that converts an object of the specified type to a {@link Row} object. */ | ||
/** | ||
* Retrieve the function that converts an object of the specified type to a {@link Row} object. | ||
*/ | ||
public <T> SerializableFunction<T, Row> getToRowFunction(TypeDescriptor<T> typeDescriptor) | ||
throws NoSuchSchemaException { | ||
SchemaEntry entry = entries.get(typeDescriptor); | ||
|
@@ -288,6 +281,38 @@ public <T> SerializableFunction<Row, T> getFromRowFunction(TypeDescriptor<T> typ | |
return getProviderResult((SchemaProvider p) -> p.fromRowFunction(typeDescriptor)); | ||
} | ||
|
||
/** | ||
* Retrieve a {@link SchemaCoder} for a given {@link Class} type. If no schema exists, throws | ||
* {@link * NoSuchSchemaException}. | ||
*/ | ||
public <T> SchemaCoder<T> getSchemaCoder(Class<T> clazz) throws NoSuchSchemaException { | ||
return getSchemaCoder(TypeDescriptor.of(clazz)); | ||
} | ||
|
||
/** | ||
* Retrieve a {@link SchemaCoder} for a given {@link TypeDescriptor} type. If no schema exists, | ||
* throws {@link * NoSuchSchemaException}. | ||
*/ | ||
public <T> SchemaCoder<T> getSchemaCoder(TypeDescriptor<T> typeDescriptor) | ||
throws NoSuchSchemaException { | ||
return SchemaCoder.of( | ||
getSchema(typeDescriptor), | ||
typeDescriptor, | ||
getToRowFunction(typeDescriptor), | ||
getFromRowFunction(typeDescriptor)); | ||
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. This looks fine, but can you explain the use case? The goal was for users to never have to deal with SchemaCoder (the fact that schemas are implemented via a special coder should be a Beam implementation details), but I'd understand if we have cases where the coder is still needed. There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. See #10978 for a concrete case. PTransform authors may benefit of being able to infer a There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. I agree with you in the fact that There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. It's useful when integrating schema code with code that does not yet understand schemas. In the KafkaIO example I think that the ideal solution would be to allow a Schema on KafkaRecord (this probably requires us to add Java generic type awareness to schema inference though), in which case they keyCoder and valueCoder isn't needed. I agree that allowing easy inference of SchemaCoder allows for lower-effort integration of schemas in code like this, though hopefully this is just a short-term solution. There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. Interesting. I have not thought about making KafkaRecord 'schema' like good point. There are some consequences on that that are still not clear to me (like how will we deal with the runtime resolution part of Schemas for KV that we do now with the Confluent Schema Registry support. I am going to give it a try and ping you once I have something in the other PR #10978. Let's continue that discussion there. |
||
} | ||
|
||
private <ReturnT> ReturnT getProviderResult(Function<SchemaProvider, ReturnT> f) | ||
throws NoSuchSchemaException { | ||
for (SchemaProvider provider : providers) { | ||
ReturnT result = f.apply(provider); | ||
if (result != null) { | ||
return result; | ||
} | ||
} | ||
throw new NoSuchSchemaException(); | ||
} | ||
|
||
static { | ||
// find all statically-registered SchemaProviders. | ||
List<SchemaProvider> providersToRegister = Lists.newArrayList(); | ||
|
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
This change is required because we should not be exposing Guava (even our own vendored version) in the public API and it was surfaced by a
GcpCoreApiSurfaceTest
break