private ObjectMapper objectMapper() { if (this.objectMapper == null) { synchronized (this) { if (this.objectMapper == null) { this.objectMapper = newObjectMapperWith(RowJsonDeserializer.forSchema(this.schema)); } } } return this.objectMapper; } }
/** Creates a deserializer for a {@link Row} {@link Schema}. */ public static RowJsonDeserializer forSchema(Schema schema) { schema.getFields().forEach(RowJsonValidation::verifyFieldTypeSupported); return new RowJsonDeserializer(schema); }
@Override public Row deserialize(JsonParser jsonParser, DeserializationContext deserializationContext) throws IOException { // Parse and convert the root object to Row as if it's a nested field with name 'root' return (Row) extractJsonNodeValue( FieldValue.of("root", FieldType.row(schema), jsonParser.readValueAsTree())); }
private static Object extractJsonNodeValue(FieldValue fieldValue) { if (!fieldValue.isJsonValuePresent()) { throw new UnsupportedRowJsonException( "Field '" + fieldValue.name() + "' is not present in the JSON object"); } if (fieldValue.isJsonNull()) { return null; } if (fieldValue.isRowType()) { return jsonObjectToRow(fieldValue); } if (fieldValue.isArrayType()) { return jsonArrayToList(fieldValue); } return extractJsonPrimitiveValue(fieldValue); }
private Row parsePayloadJsonRow(PubsubMessage pubsubMessage) { String payloadJson = new String(pubsubMessage.getPayload(), StandardCharsets.UTF_8); if (objectMapper == null) { objectMapper = newObjectMapperWith(RowJsonDeserializer.forSchema(payloadSchema())); } return JsonToRowUtils.jsonToRow(objectMapper, payloadJson); }
private static Object jsonArrayToList(FieldValue arrayFieldValue) { if (!arrayFieldValue.isJsonArray()) { throw new UnsupportedRowJsonException( "Expected JSON array for field '" + arrayFieldValue.name() + "'. " + "Instead got " + arrayFieldValue.jsonNodeType().name()); } return arrayFieldValue .jsonArrayElements() .map( jsonArrayElement -> extractJsonNodeValue( FieldValue.of( arrayFieldValue.name() + "[]", arrayFieldValue.arrayElementType(), jsonArrayElement))) .collect(toList()); }
@Test public void testParsesArrayOfArrays() throws Exception { Schema schema = Schema.builder() .addArrayField("f_arrayOfIntArrays", FieldType.array(FieldType.INT32)) .build(); String rowString = "{\n" + "\"f_arrayOfIntArrays\" : [ [1, 2], [3, 4], [5]]\n" + "}"; RowJsonDeserializer deserializer = RowJsonDeserializer.forSchema(schema); Row parsedRow = newObjectMapperWith(deserializer).readValue(rowString, Row.class); Row expectedRow = Row.withSchema(schema) .addArray(Arrays.asList(1, 2), Arrays.asList(3, 4), Arrays.asList(5)) .build(); assertEquals(expectedRow, parsedRow); }
private static Row jsonObjectToRow(FieldValue rowFieldValue) { if (!rowFieldValue.isJsonObject()) { throw new UnsupportedRowJsonException( "Expected JSON object for field '" + rowFieldValue.name() + "'. " + "Unable to convert '" + rowFieldValue.jsonValue().asText() + "'" + " to Beam Row, it is not a JSON object. Currently only JSON objects " + "can be parsed to Beam Rows"); } return rowFieldValue .rowSchema() .getFields() .stream() .map( schemaField -> extractJsonNodeValue( FieldValue.of( schemaField.getName(), schemaField.getType(), rowFieldValue.jsonFieldValue(schemaField.getName())))) .collect(toRow(rowFieldValue.rowSchema())); }
private void testUnsupportedConversion(FieldType fieldType, String jsonFieldValue) throws Exception { String fieldName = "f_" + fieldType.getTypeName().name().toLowerCase(); ObjectMapper jsonParser = newObjectMapperWith(RowJsonDeserializer.forSchema(schemaWithField(fieldName, fieldType))); thrown.expectMessage(fieldName); thrown.expectCause(unsupportedWithMessage(jsonFieldValue, "out of range")); jsonParser.readValue(jsonObjectWith(fieldName, jsonFieldValue), Row.class); }
private void testSupportedConversion( FieldType fieldType, String jsonFieldValue, Object expectedRowFieldValue) throws Exception { String fieldName = "f_" + fieldType.getTypeName().name().toLowerCase(); Schema schema = schemaWithField(fieldName, fieldType); Row expectedRow = Row.withSchema(schema).addValues(expectedRowFieldValue).build(); ObjectMapper jsonParser = newObjectMapperWith(RowJsonDeserializer.forSchema(schema)); Row parsedRow = jsonParser.readValue(jsonObjectWith(fieldName, jsonFieldValue), Row.class); assertEquals(expectedRow, parsedRow); }
@Test public void testThrowsForMissingNotNullableField() throws Exception { Schema schema = Schema.builder().addByteField("f_byte").addStringField("f_string").build(); String rowString = "{\n" + "\"f_byte\" : 12\n" + "}"; RowJsonDeserializer deserializer = RowJsonDeserializer.forSchema(schema); thrown.expect(UnsupportedRowJsonException.class); thrown.expectMessage("'f_string' is not present"); newObjectMapperWith(deserializer).readValue(rowString, Row.class); }
@Test public void testThrowsForUnsupportedType() throws Exception { Schema schema = Schema.builder().addDateTimeField("f_dateTime").build(); thrown.expect(UnsupportedRowJsonException.class); thrown.expectMessage("DATETIME is not supported"); RowJsonDeserializer.forSchema(schema); }
@Test public void testParsesArrayField() throws Exception { Schema schema = Schema.builder() .addInt32Field("f_int32") .addArrayField("f_intArray", FieldType.INT32) .build(); String rowString = "{\n" + "\"f_int32\" : 32,\n" + "\"f_intArray\" : [ 1, 2, 3, 4, 5]\n" + "}"; RowJsonDeserializer deserializer = RowJsonDeserializer.forSchema(schema); Row parsedRow = newObjectMapperWith(deserializer).readValue(rowString, Row.class); Row expectedRow = Row.withSchema(schema).addValues(32, Arrays.asList(1, 2, 3, 4, 5)).build(); assertEquals(expectedRow, parsedRow); }
@Test public void testThrowsForUnsupportedArrayElementType() throws Exception { Schema schema = Schema.builder().addArrayField("f_dateTimeArray", FieldType.DATETIME).build(); thrown.expect(UnsupportedRowJsonException.class); thrown.expectMessage("DATETIME is not supported"); RowJsonDeserializer.forSchema(schema); }
@Test public void testThrowsForMismatchedArrayField() throws Exception { Schema schema = Schema.builder() .addArrayField("f_arrayOfIntArrays", FieldType.array(FieldType.INT32)) .build(); String rowString = "{\n" + "\"f_arrayOfIntArrays\" : { }\n" // expect array, get object + "}"; RowJsonDeserializer deserializer = RowJsonDeserializer.forSchema(schema); thrown.expect(UnsupportedRowJsonException.class); thrown.expectMessage("Expected JSON array"); newObjectMapperWith(deserializer).readValue(rowString, Row.class); }
@Test public void testParsesNulls() throws Exception { Schema schema = Schema.builder() .addByteField("f_byte") .addNullableField("f_string", FieldType.STRING) .build(); String rowString = "{\n" + "\"f_byte\" : 12,\n" + "\"f_string\" : null\n" + "}"; RowJsonDeserializer deserializer = RowJsonDeserializer.forSchema(schema); Row parsedRow = newObjectMapperWith(deserializer).readValue(rowString, Row.class); Row expectedRow = Row.withSchema(schema).addValues((byte) 12, null).build(); assertEquals(expectedRow, parsedRow); }
@Test public void testThrowsForUnsupportedNestedFieldType() throws Exception { Schema nestedSchema = Schema.builder().addArrayField("f_dateTimeArray", FieldType.DATETIME).build(); Schema schema = Schema.builder().addRowField("f_nestedRow", nestedSchema).build(); thrown.expect(UnsupportedRowJsonException.class); thrown.expectMessage("DATETIME is not supported"); RowJsonDeserializer.forSchema(schema); }
@Test public void testThrowsForMismatchedRowField() throws Exception { Schema nestedRowSchema = Schema.builder().addInt32Field("f_nestedInt32").addStringField("f_nestedString").build(); Schema schema = Schema.builder().addInt32Field("f_int32").addRowField("f_row", nestedRowSchema).build(); String rowString = "{\n" + "\"f_int32\" : 32,\n" + "\"f_row\" : []\n" // expect object, get array + "}"; RowJsonDeserializer deserializer = RowJsonDeserializer.forSchema(schema); thrown.expect(UnsupportedRowJsonException.class); thrown.expectMessage("Expected JSON object"); newObjectMapperWith(deserializer).readValue(rowString, Row.class); }
@Test public void testParsesRowField() throws Exception { Schema nestedRowSchema = Schema.builder().addInt32Field("f_nestedInt32").addStringField("f_nestedString").build(); Schema schema = Schema.builder().addInt32Field("f_int32").addRowField("f_row", nestedRowSchema).build(); String rowString = "{\n" + "\"f_int32\" : 32,\n" + "\"f_row\" : {\n" + " \"f_nestedInt32\" : 54,\n" + " \"f_nestedString\" : \"foo\"\n" + " }\n" + "}"; RowJsonDeserializer deserializer = RowJsonDeserializer.forSchema(schema); Row parsedRow = newObjectMapperWith(deserializer).readValue(rowString, Row.class); Row expectedRow = Row.withSchema(schema) .addValues(32, Row.withSchema(nestedRowSchema).addValues(54, "foo").build()) .build(); assertEquals(expectedRow, parsedRow); }
+ "}"; RowJsonDeserializer deserializer = RowJsonDeserializer.forSchema(schema);