Skip to content
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

Adding Beam Schemas capability to parse json-schemas. This is the de-… #24271

Merged
merged 14 commits into from
Dec 2, 2022

Conversation

pabloem
Copy link
Member

@pabloem pabloem commented Nov 18, 2022

…facto standard way to define JSON schemas

r: @damondouglas

fyi: @reuvenlax @lukecwik - I am adding a new dependency to Java Core SDK. It is not exposed in the API. It is used to parse JSON schemas.


Thank you for your contribution! Follow this checklist to help us incorporate your contribution quickly and easily:

  • Choose reviewer(s) and mention them in a comment (R: @username).
  • Mention the appropriate issue in your description (for example: addresses #123), if applicable. This will automatically add a link to the pull request in the issue. If you would like the issue to automatically close on merging the pull request, comment fixes #<ISSUE NUMBER> instead.
  • Update CHANGES.md with noteworthy changes.
  • If this contribution is large, please file an Apache Individual Contributor License Agreement.

See the Contributor Guide for more tips on how to make review process smoother.

To check the build health, please visit https://github.com/apache/beam/blob/master/.test-infra/BUILD_STATUS.md

GitHub Actions Tests Status (on master branch)

Build python source distribution and wheels
Python tests
Java tests
Go tests

See CI.md for more information about GitHub Actions CI.

@github-actions
Copy link
Contributor

Assigning reviewers. If you would like to opt out of this review, comment assign to next reviewer:

R: @apilloud for label java.
R: @damccorm for label build.

Available commands:

  • stop reviewer notifications - opt out of the automated review tooling
  • remind me after tests pass - tag the comment author after tests pass
  • waiting on author - shift the attention set back to the author (any comment or push by the author will return the attention set to the reviewers)

The PR bot will only process comments in the main thread (not review comments).

@codecov
Copy link

codecov bot commented Nov 18, 2022

Codecov Report

Merging #24271 (aff5ea4) into master (67e2008) will decrease coverage by 0.06%.
The diff coverage is n/a.

@@            Coverage Diff             @@
##           master   #24271      +/-   ##
==========================================
- Coverage   73.43%   73.37%   -0.07%     
==========================================
  Files         716      717       +1     
  Lines       96571    96939     +368     
==========================================
+ Hits        70920    71128     +208     
- Misses      24328    24488     +160     
  Partials     1323     1323              
Flag Coverage Δ
python 82.98% <ø> (-0.15%) ⬇️

Flags with carried forward coverage won't be shown. Click here to find out more.

Impacted Files Coverage Δ
...l/job_management/v1/beam_expansion_api_pb2_grpc.py 51.85% <0.00%> (-6.05%) ⬇️
.../python/apache_beam/testing/test_stream_service.py 88.09% <0.00%> (-4.77%) ⬇️
sdks/python/apache_beam/utils/interactive_utils.py 95.12% <0.00%> (-2.44%) ⬇️
...che_beam/runners/interactive/interactive_runner.py 90.50% <0.00%> (-1.27%) ⬇️
sdks/python/apache_beam/transforms/external.py 78.88% <0.00%> (-0.86%) ⬇️
sdks/python/apache_beam/pipeline.py 92.12% <0.00%> (-0.15%) ⬇️
sdks/python/apache_beam/dataframe/frames.py 95.33% <0.00%> (-0.13%) ⬇️
sdks/python/apache_beam/io/filebasedsource.py 98.85% <0.00%> (-0.02%) ⬇️
sdks/python/apache_beam/portability/common_urns.py 100.00% <0.00%> (ø)
...examples/inference/pytorch_image_classification.py 0.00% <0.00%> (ø)
... and 12 more

📣 We’re building smart automated test selection to slash your CI/CD build times. Learn more

Copy link
Member

@lukecwik lukecwik left a comment

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

What do we do with properties that don't translate to beam schemas like array uniqueness of items?

We should describe to users what we reject and be explicit about what we support and what we ignore for each JSON type from https://json-schema.org/understanding-json-schema/reference/index.html

We'll also want to be careful when supporting nulls and test for the null case explicitly.

@@ -91,6 +91,8 @@ dependencies {
shadow library.java.avro
shadow library.java.snappy_java
shadow library.java.joda_time
shadow library.java.json_org
shadow library.java.everit_json_schema
Copy link
Member

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Should we make this feature require the user provide the library?

Need to analyze how large the dependency tree is and how stable it is.

Copy link
Member Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

I've marked is as a provided dependency.

Before moving forward - another possibility is to create a new extension module to hold these dependencies and this functionality. The module would be used by some of our schema transform implementations (kafka and pubsub), so it would still come back as a depenency for other Beam modules, but not for core - wdyt?

@reuvenlax
Copy link
Contributor

Should we add a SchemaProvider as well?

if (propertySchema == null) {
throw new IllegalArgumentException("Unable to parse schema " + jsonSchema.toString());
}
if (propertySchema instanceof org.everit.json.schema.ObjectSchema) {
Copy link
Contributor

@damondouglas damondouglas Nov 21, 2022

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Is it possible to add the following support methods? I advocate that removing the embedded checks in this method allows these to be independently tested. First, the following shows how beamSchemaFromJsonSchema might look with the proposed overloaded support methods. Then, some of the support methods follow. To avoid the instanceof checks, I tried to find in the libraries Javadoc, whether the base Schema had a getType() getter but it seems not. Perhaps we could have a Map<Class<T extends org.everit.json.schema.Schema>, Function<T extends org.everit.json.schema.Schema, Schema.Field>> that maps a particular json Schema class to the Java function that converts to the Schema.Field.

Schema.Builder beamSchemaBuilder = Schema.builder()
for (String propertyName : jsonSchema.getPropertySchemas().keySet()) {
    org.everit.json.schema.Schema propertySchema = jsonSchema.getPropertySchemas().get(propertyName);
    if (propertySchema == null) {
        throw new IllegalArgumentException("Unable to parse schema " + jsonSchema.toString());
    }
    
    Schema.Field field = beamFieldFromJsonField(propertySchema);
    builder = builder.addField(field);
}

return builder.build();
static Schema.Field beamFieldFromJsonField(org.everit.json.schema.Schema propertySchema) {
    if (propertySchema instanceof org.everit.json.schema.ObjectSchema) {
        return beamFieldFromJsonField((org.everit.json.schema.ObjectSchema) propertySchema);
    }
    if (propertySchema instanceof org.everit.json.schema.ArraySchema) {
        return beamFieldFromJsonField((org.everit.json.schema.ObjectSchema) propertySchema);
    }
    // etc etc to all the various types.
}

static Schema.Field beamFieldFromJsonField(org.everit.json.schema.ObjectSchema objectSchema) {
   // do some magic
}

static Schema.Field beamFieldFromJsonField(org.everit.json.schema.ArraySchema arraySchema) {
   // do some more magic
}

static Schema.Field beamFieldFromJsonField(org.everit.json.schema.BooleanField booleanSchema) {
  // do even some more magical magic
}

Copy link
Member

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

The conversions for type to schema are short I think it would be simpler to not add this indirection

@pabloem
Copy link
Member Author

pabloem commented Nov 23, 2022

Should we add a SchemaProvider as well?

Can you elaborate @reuvenlax ? Any pointers? : ) Happy to implement whatever...

@pabloem
Copy link
Member Author

pabloem commented Nov 23, 2022

Run Java PreCommit

3 similar comments
@pabloem
Copy link
Member Author

pabloem commented Nov 23, 2022

Run Java PreCommit

@pabloem
Copy link
Member Author

pabloem commented Nov 23, 2022

Run Java PreCommit

@pabloem
Copy link
Member Author

pabloem commented Nov 23, 2022

Run Java PreCommit

@pabloem
Copy link
Member Author

pabloem commented Nov 24, 2022

What do we do with properties that don't translate to beam schemas like array uniqueness of items?

We should describe to users what we reject and be explicit about what we support and what we ignore for each JSON type from https://json-schema.org/understanding-json-schema/reference/index.html

We'll also want to be careful when supporting nulls and test for the null case explicitly.

I've tried to document in some detail what we do/don't support, as well as the required provided dependency. LMK what you think @lukecwik

I've also added support for nullable/non-nullable fields (based on the requiredfields tag)

@pabloem
Copy link
Member Author

pabloem commented Nov 24, 2022

Run Spotless PreCommit

@pabloem
Copy link
Member Author

pabloem commented Nov 24, 2022

Run Java_GCP_IO_Direct PreCommit

1 similar comment
@pabloem
Copy link
Member Author

pabloem commented Nov 24, 2022

Run Java_GCP_IO_Direct PreCommit

@pabloem
Copy link
Member Author

pabloem commented Nov 28, 2022

Run Java_Debezium_IO_Direct PreCommit

@pabloem
Copy link
Member Author

pabloem commented Nov 28, 2022

Run Java_PVR_Flink_Docker PreCommit

@pabloem
Copy link
Member Author

pabloem commented Nov 28, 2022

@lukecwik please let me know what you think of the javadoc for it, and other changes.

Comment on lines 48 to 51
* <dependency>
* <groupId>com.github.erosb < /groupId>
* <artifactId>everit-json-schema < /artifactId>
* <version>1.14.1 < /version>
Copy link
Member

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

We should point the person to use the same version that Beam was tested with and not hard-code the version in the documentation.

Copy link
Member Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Done. Thanks.

Copy link
Member

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

annoying but is fine

@lukecwik
Copy link
Member

lukecwik commented Dec 1, 2022

Should we add a SchemaProvider as well?

Can you elaborate @reuvenlax ? Any pointers? : ) Happy to implement whatever...

This only applies if you have a type that you want to convert to a schema and doesn't apply to schemas stored in an arbitrary location (e.g. string, file, URL, ...)

@lukecwik
Copy link
Member

lukecwik commented Dec 1, 2022

@lukecwik please let me know what you think of the javadoc for it, and other changes.

Nice improvement over the original and good tests.

pabloem and others added 7 commits December 2, 2022 13:46
…/JsonUtils.java

Co-authored-by: Lukasz Cwik <lcwik@google.com>
…/JsonUtils.java

Co-authored-by: Lukasz Cwik <lcwik@google.com>
…/JsonUtils.java

Co-authored-by: Lukasz Cwik <lcwik@google.com>
…/JsonUtils.java

Co-authored-by: Lukasz Cwik <lcwik@google.com>
…/JsonUtils.java

Co-authored-by: Lukasz Cwik <lcwik@google.com>
@pabloem
Copy link
Member Author

pabloem commented Dec 2, 2022

Run Java_Examples_Dataflow PreCommit

3 similar comments
@pabloem
Copy link
Member Author

pabloem commented Dec 2, 2022

Run Java_Examples_Dataflow PreCommit

@pabloem
Copy link
Member Author

pabloem commented Dec 2, 2022

Run Java_Examples_Dataflow PreCommit

@pabloem
Copy link
Member Author

pabloem commented Dec 2, 2022

Run Java_Examples_Dataflow PreCommit

@pabloem
Copy link
Member Author

pabloem commented Dec 2, 2022

Run Java_GCP_IO_Direct PreCommit

@pabloem pabloem merged commit 2dbf94f into apache:master Dec 2, 2022
@pabloem pabloem deleted the jsonschema-parse branch December 2, 2022 21:46
@pabloem
Copy link
Member Author

pabloem commented Dec 2, 2022

thanks all!

prodriguezdefino pushed a commit to prodriguezdefino/beam-pabs that referenced this pull request Dec 6, 2022
apache#24271)

* Adding Beam Schemas capability to parse json-schemas. This is the de-facto standard way to define JSON schemas

* json sample schema files for tests

* addressing comments

* fixup

* fixup

* documenting and fixing nullable cases

* fixup

* Update sdks/java/core/src/main/java/org/apache/beam/sdk/schemas/utils/JsonUtils.java

Co-authored-by: Lukasz Cwik <lcwik@google.com>

* Update sdks/java/core/src/main/java/org/apache/beam/sdk/schemas/utils/JsonUtils.java

Co-authored-by: Lukasz Cwik <lcwik@google.com>

* Update sdks/java/core/src/main/java/org/apache/beam/sdk/schemas/utils/JsonUtils.java

Co-authored-by: Lukasz Cwik <lcwik@google.com>

* Update sdks/java/core/src/main/java/org/apache/beam/sdk/schemas/utils/JsonUtils.java

Co-authored-by: Lukasz Cwik <lcwik@google.com>

* Update sdks/java/core/src/main/java/org/apache/beam/sdk/schemas/utils/JsonUtils.java

Co-authored-by: Lukasz Cwik <lcwik@google.com>

* improve docs

* fixup

Co-authored-by: Lukasz Cwik <lcwik@google.com>
lostluck pushed a commit to lostluck/beam that referenced this pull request Dec 22, 2022
apache#24271)

* Adding Beam Schemas capability to parse json-schemas. This is the de-facto standard way to define JSON schemas

* json sample schema files for tests

* addressing comments

* fixup

* fixup

* documenting and fixing nullable cases

* fixup

* Update sdks/java/core/src/main/java/org/apache/beam/sdk/schemas/utils/JsonUtils.java

Co-authored-by: Lukasz Cwik <lcwik@google.com>

* Update sdks/java/core/src/main/java/org/apache/beam/sdk/schemas/utils/JsonUtils.java

Co-authored-by: Lukasz Cwik <lcwik@google.com>

* Update sdks/java/core/src/main/java/org/apache/beam/sdk/schemas/utils/JsonUtils.java

Co-authored-by: Lukasz Cwik <lcwik@google.com>

* Update sdks/java/core/src/main/java/org/apache/beam/sdk/schemas/utils/JsonUtils.java

Co-authored-by: Lukasz Cwik <lcwik@google.com>

* Update sdks/java/core/src/main/java/org/apache/beam/sdk/schemas/utils/JsonUtils.java

Co-authored-by: Lukasz Cwik <lcwik@google.com>

* improve docs

* fixup

Co-authored-by: Lukasz Cwik <lcwik@google.com>
@mosche mosche mentioned this pull request Dec 22, 2022
3 tasks
Sign up for free to join this conversation on GitHub. Already have an account? Sign in to comment
Projects
None yet
Development

Successfully merging this pull request may close these issues.

4 participants