lukecwik commented on code in PR #24271:
URL: https://github.com/apache/beam/pull/24271#discussion_r1037602011
##########
sdks/java/core/src/main/java/org/apache/beam/sdk/schemas/utils/JsonUtils.java:
##########
@@ -26,8 +26,58 @@
import org.apache.beam.sdk.util.RowJson;
import org.apache.beam.sdk.util.RowJsonUtils;
import org.apache.beam.sdk.values.Row;
+import org.everit.json.schema.ArraySchema;
+import org.everit.json.schema.NumberSchema;
+import org.everit.json.schema.ObjectSchema;
+import org.everit.json.schema.ReferenceSchema;
+import org.json.JSONObject;
-/** Utils to convert JSON records to Beam {@link Row}. */
+/**
+ * Utils to convert JSON records to Beam {@link Row}.
+ *
+ * <h2>JSON-Schema (https://json-schema.org) support</h2>
+ *
+ * <p>This class provides utility methods to parse, validate and translate
between <b>JSON
+ * Schema</b>-formatted schemas and Beam Schemas. The support is based on the
<code>
+ * everit-json-schema</code> package, which is <b>not provided by default</b>.
+ *
+ * <p>Therefore, functionality in {@link JsonUtils::beamSchemaFromJsonSchema}
requires that you
+ * include {@code everit-json-schema} in your project like so:
+ *
+ * <pre>{@code
+ * <dependency>
+ * <groupId>com.github.erosb < /groupId>
+ * <artifactId>everit-json-schema < /artifactId>
+ * <version>1.14.1 < /version>
Review Comment:
```suggestion
* <dependency>
* <groupId>com.github.erosb</groupId>
* <artifactId>everit-json-schema</artifactId>
* <version>1.14.1</version>
```
##########
sdks/java/core/src/main/java/org/apache/beam/sdk/schemas/utils/JsonUtils.java:
##########
@@ -26,8 +26,58 @@
import org.apache.beam.sdk.util.RowJson;
import org.apache.beam.sdk.util.RowJsonUtils;
import org.apache.beam.sdk.values.Row;
+import org.everit.json.schema.ArraySchema;
+import org.everit.json.schema.NumberSchema;
+import org.everit.json.schema.ObjectSchema;
+import org.everit.json.schema.ReferenceSchema;
+import org.json.JSONObject;
-/** Utils to convert JSON records to Beam {@link Row}. */
+/**
+ * Utils to convert JSON records to Beam {@link Row}.
+ *
+ * <h2>JSON-Schema (https://json-schema.org) support</h2>
+ *
+ * <p>This class provides utility methods to parse, validate and translate
between <b>JSON
+ * Schema</b>-formatted schemas and Beam Schemas. The support is based on the
<code>
+ * everit-json-schema</code> package, which is <b>not provided by default</b>.
+ *
+ * <p>Therefore, functionality in {@link JsonUtils::beamSchemaFromJsonSchema}
requires that you
+ * include {@code everit-json-schema} in your project like so:
+ *
+ * <pre>{@code
+ * <dependency>
+ * <groupId>com.github.erosb < /groupId>
+ * <artifactId>everit-json-schema < /artifactId>
+ * <version>1.14.1 < /version>
+ * < /dependency>
+ * }</pre>
+ *
+ * <h3>JSON-Schema supported features</h3>
+ *
+ * <p>The current Beam implementation does not support all possible features
of JSON-schema. The
+ * current implementation supports:
+ *
+ * <p>
+ *
+ * <ul>
+ * <li>String, boolean and numeric values (integer and floating-point).
+ * <li>Arrays, nested arrays and arbitratily nested object types
Review Comment:
```suggestion
* <li>Arrays with any <b>one</b> of the supported types as elements. This
includes nested arrays.
* <li>Objects with any of the supported types as fields. This includes
nested objects.
```
##########
sdks/java/core/src/main/java/org/apache/beam/sdk/schemas/utils/JsonUtils.java:
##########
@@ -26,8 +26,58 @@
import org.apache.beam.sdk.util.RowJson;
import org.apache.beam.sdk.util.RowJsonUtils;
import org.apache.beam.sdk.values.Row;
+import org.everit.json.schema.ArraySchema;
+import org.everit.json.schema.NumberSchema;
+import org.everit.json.schema.ObjectSchema;
+import org.everit.json.schema.ReferenceSchema;
+import org.json.JSONObject;
-/** Utils to convert JSON records to Beam {@link Row}. */
+/**
+ * Utils to convert JSON records to Beam {@link Row}.
+ *
+ * <h2>JSON-Schema (https://json-schema.org) support</h2>
+ *
+ * <p>This class provides utility methods to parse, validate and translate
between <b>JSON
+ * Schema</b>-formatted schemas and Beam Schemas. The support is based on the
<code>
+ * everit-json-schema</code> package, which is <b>not provided by default</b>.
+ *
+ * <p>Therefore, functionality in {@link JsonUtils::beamSchemaFromJsonSchema}
requires that you
+ * include {@code everit-json-schema} in your project like so:
+ *
+ * <pre>{@code
+ * <dependency>
+ * <groupId>com.github.erosb < /groupId>
+ * <artifactId>everit-json-schema < /artifactId>
+ * <version>1.14.1 < /version>
Review Comment:
We should point the person to use the same version that Beam was tested with
and not hard-code the version in the documentation.
##########
sdks/java/core/src/main/java/org/apache/beam/sdk/schemas/utils/JsonUtils.java:
##########
@@ -73,6 +123,93 @@ public String apply(Row input) {
};
}
+ public static Schema beamSchemaFromJsonSchema(String jsonSchemaStr) {
+ org.everit.json.schema.ObjectSchema jsonSchema =
jsonSchemaFromString(jsonSchemaStr);
+ return beamSchemaFromJsonSchema(jsonSchema);
+ }
+
+ private static Schema
beamSchemaFromJsonSchema(org.everit.json.schema.ObjectSchema jsonSchema) {
+ 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);
+ }
+ java.util.function.BiFunction<String, Schema.FieldType, Schema.Field>
fieldConstructor =
+ jsonSchema.getRequiredProperties().contains(propertyName)
+ ? Schema.Field::of
+ : Schema.Field::nullable;
+ if (propertySchema instanceof org.everit.json.schema.ArraySchema) {
+ if (((ArraySchema) propertySchema).getAllItemSchema() == null) {
+ throw new IllegalArgumentException(
+ "Array schema is not properly formatted or unsupported ("
+ + propertyName
+ + "). Note that JSON-schema's tuple-like arrays are not
supported by Beam.");
+ }
+ beamSchemaBuilder =
+ beamSchemaBuilder.addField(
+ fieldConstructor.apply(
+ propertyName,
+ Schema.FieldType.array(
+ beamTypeFromJsonSchemaType(
+ ((ArraySchema)
propertySchema).getAllItemSchema()))));
+ } else {
+ try {
+ beamSchemaBuilder =
+ beamSchemaBuilder.addField(
+ fieldConstructor.apply(propertyName,
beamTypeFromJsonSchemaType(propertySchema)));
+ } catch (IllegalArgumentException e) {
+ throw new IllegalArgumentException(
+ "Unsupported field type " + propertySchema.getClass() + " in
field " + propertyName,
+ e);
+ }
+ }
+ }
+ return beamSchemaBuilder.build();
+ }
+
+ private static Schema.FieldType beamTypeFromJsonSchemaType(
+ org.everit.json.schema.Schema propertySchema) {
+ if (propertySchema instanceof org.everit.json.schema.ObjectSchema) {
+ return Schema.FieldType.row(beamSchemaFromJsonSchema((ObjectSchema)
propertySchema));
+ } else if (propertySchema instanceof org.everit.json.schema.BooleanSchema)
{
+ return Schema.FieldType.BOOLEAN;
+ } else if (propertySchema instanceof org.everit.json.schema.NumberSchema) {
+ return ((NumberSchema) propertySchema).requiresInteger()
+ ? Schema.FieldType.INT64
+ : Schema.FieldType.DOUBLE;
+ }
+ if (propertySchema instanceof org.everit.json.schema.StringSchema) {
+ return Schema.FieldType.STRING;
+ } else if (propertySchema instanceof
org.everit.json.schema.ReferenceSchema) {
+ org.everit.json.schema.Schema sch = ((ReferenceSchema)
propertySchema).getReferredSchema();
+ return beamTypeFromJsonSchemaType(sch);
+ } else if (propertySchema instanceof org.everit.json.schema.ArraySchema) {
+ if (((ArraySchema) propertySchema).getAllItemSchema() == null) {
+ throw new IllegalArgumentException(
+ "Array schema is not properly formatted or unsupported ("
+ + propertySchema
+ + "). Note that JSON-schema's tuple-like arrays are not
supported by Beam.");
Review Comment:
```suggestion
"Tuple-like arrays are unsupported. Expected a single item
type.");
```
##########
sdks/java/core/src/main/java/org/apache/beam/sdk/schemas/utils/JsonUtils.java:
##########
@@ -73,6 +123,93 @@ public String apply(Row input) {
};
}
+ public static Schema beamSchemaFromJsonSchema(String jsonSchemaStr) {
+ org.everit.json.schema.ObjectSchema jsonSchema =
jsonSchemaFromString(jsonSchemaStr);
+ return beamSchemaFromJsonSchema(jsonSchema);
+ }
+
+ private static Schema
beamSchemaFromJsonSchema(org.everit.json.schema.ObjectSchema jsonSchema) {
+ 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);
+ }
+ java.util.function.BiFunction<String, Schema.FieldType, Schema.Field>
fieldConstructor =
+ jsonSchema.getRequiredProperties().contains(propertyName)
+ ? Schema.Field::of
+ : Schema.Field::nullable;
+ if (propertySchema instanceof org.everit.json.schema.ArraySchema) {
+ if (((ArraySchema) propertySchema).getAllItemSchema() == null) {
+ throw new IllegalArgumentException(
+ "Array schema is not properly formatted or unsupported ("
+ + propertyName
+ + "). Note that JSON-schema's tuple-like arrays are not
supported by Beam.");
+ }
+ beamSchemaBuilder =
+ beamSchemaBuilder.addField(
+ fieldConstructor.apply(
+ propertyName,
+ Schema.FieldType.array(
+ beamTypeFromJsonSchemaType(
+ ((ArraySchema)
propertySchema).getAllItemSchema()))));
+ } else {
+ try {
+ beamSchemaBuilder =
+ beamSchemaBuilder.addField(
+ fieldConstructor.apply(propertyName,
beamTypeFromJsonSchemaType(propertySchema)));
+ } catch (IllegalArgumentException e) {
+ throw new IllegalArgumentException(
+ "Unsupported field type " + propertySchema.getClass() + " in
field " + propertyName,
+ e);
+ }
+ }
+ }
+ return beamSchemaBuilder.build();
+ }
+
+ private static Schema.FieldType beamTypeFromJsonSchemaType(
+ org.everit.json.schema.Schema propertySchema) {
+ if (propertySchema instanceof org.everit.json.schema.ObjectSchema) {
+ return Schema.FieldType.row(beamSchemaFromJsonSchema((ObjectSchema)
propertySchema));
+ } else if (propertySchema instanceof org.everit.json.schema.BooleanSchema)
{
+ return Schema.FieldType.BOOLEAN;
+ } else if (propertySchema instanceof org.everit.json.schema.NumberSchema) {
+ return ((NumberSchema) propertySchema).requiresInteger()
+ ? Schema.FieldType.INT64
+ : Schema.FieldType.DOUBLE;
+ }
+ if (propertySchema instanceof org.everit.json.schema.StringSchema) {
Review Comment:
```suggestion
} else if (propertySchema instanceof
org.everit.json.schema.StringSchema) {
```
##########
sdks/java/core/src/main/java/org/apache/beam/sdk/schemas/utils/JsonUtils.java:
##########
@@ -73,6 +123,93 @@ public String apply(Row input) {
};
}
+ public static Schema beamSchemaFromJsonSchema(String jsonSchemaStr) {
+ org.everit.json.schema.ObjectSchema jsonSchema =
jsonSchemaFromString(jsonSchemaStr);
+ return beamSchemaFromJsonSchema(jsonSchema);
+ }
+
+ private static Schema
beamSchemaFromJsonSchema(org.everit.json.schema.ObjectSchema jsonSchema) {
+ Schema.Builder beamSchemaBuilder = Schema.builder();
+ for (String propertyName : jsonSchema.getPropertySchemas().keySet()) {
+ org.everit.json.schema.Schema propertySchema =
+ jsonSchema.getPropertySchemas().get(propertyName);
Review Comment:
```suggestion
for (Map.Entry<String, org.everit.json.schema.Schema> entry :
jsonSchema.getPropertySchemas().entries()) {
String propertyName = entry.getKey();
org.everit.json.schema.Schema propertySchema = entry.getValue();
```
##########
sdks/java/core/src/main/java/org/apache/beam/sdk/schemas/utils/JsonUtils.java:
##########
@@ -26,8 +26,58 @@
import org.apache.beam.sdk.util.RowJson;
import org.apache.beam.sdk.util.RowJsonUtils;
import org.apache.beam.sdk.values.Row;
+import org.everit.json.schema.ArraySchema;
+import org.everit.json.schema.NumberSchema;
+import org.everit.json.schema.ObjectSchema;
+import org.everit.json.schema.ReferenceSchema;
+import org.json.JSONObject;
-/** Utils to convert JSON records to Beam {@link Row}. */
+/**
+ * Utils to convert JSON records to Beam {@link Row}.
+ *
+ * <h2>JSON-Schema (https://json-schema.org) support</h2>
+ *
+ * <p>This class provides utility methods to parse, validate and translate
between <b>JSON
+ * Schema</b>-formatted schemas and Beam Schemas. The support is based on the
<code>
+ * everit-json-schema</code> package, which is <b>not provided by default</b>.
+ *
+ * <p>Therefore, functionality in {@link JsonUtils::beamSchemaFromJsonSchema}
requires that you
+ * include {@code everit-json-schema} in your project like so:
+ *
+ * <pre>{@code
+ * <dependency>
+ * <groupId>com.github.erosb < /groupId>
+ * <artifactId>everit-json-schema < /artifactId>
+ * <version>1.14.1 < /version>
+ * < /dependency>
+ * }</pre>
+ *
+ * <h3>JSON-Schema supported features</h3>
+ *
+ * <p>The current Beam implementation does not support all possible features
of JSON-schema. The
+ * current implementation supports:
+ *
+ * <p>
+ *
+ * <ul>
+ * <li>String, boolean and numeric values (integer and floating-point).
+ * <li>Arrays, nested arrays and arbitratily nested object types
+ * <li>Fields marked as required are non-null. Other fields are nullable.
+ * </ul>
+ *
+ * <p><b>The following JSON-schema features are not supported:</b>
+ *
+ * <p>
+ *
+ * <ul>
+ * <li>Tuple-like arrays (or arrays with multiple item types).
+ * <li>Validation of row regular expressions, enum values, etc.
Review Comment:
```suggestion
* <li>Validation of row regular expressions, enum values, array lengths,
etc.
```
##########
sdks/java/core/src/main/java/org/apache/beam/sdk/schemas/utils/JsonUtils.java:
##########
@@ -73,6 +123,93 @@ public String apply(Row input) {
};
}
+ public static Schema beamSchemaFromJsonSchema(String jsonSchemaStr) {
+ org.everit.json.schema.ObjectSchema jsonSchema =
jsonSchemaFromString(jsonSchemaStr);
+ return beamSchemaFromJsonSchema(jsonSchema);
+ }
+
+ private static Schema
beamSchemaFromJsonSchema(org.everit.json.schema.ObjectSchema jsonSchema) {
+ 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);
+ }
+ java.util.function.BiFunction<String, Schema.FieldType, Schema.Field>
fieldConstructor =
+ jsonSchema.getRequiredProperties().contains(propertyName)
+ ? Schema.Field::of
+ : Schema.Field::nullable;
+ if (propertySchema instanceof org.everit.json.schema.ArraySchema) {
+ if (((ArraySchema) propertySchema).getAllItemSchema() == null) {
+ throw new IllegalArgumentException(
+ "Array schema is not properly formatted or unsupported ("
+ + propertyName
+ + "). Note that JSON-schema's tuple-like arrays are not
supported by Beam.");
Review Comment:
```suggestion
throw new IllegalArgumentException(
"Tuple-like arrays are unsupported. Expected a single item
type for field "
+ propertyName);
```
--
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]