[ 
https://issues.apache.org/jira/browse/BEAM-4160?focusedWorklogId=94919&page=com.atlassian.jira.plugin.system.issuetabpanels:worklog-tabpanel#worklog-94919
 ]

ASF GitHub Bot logged work on BEAM-4160:
----------------------------------------

                Author: ASF GitHub Bot
            Created on: 25/Apr/18 02:10
            Start Date: 25/Apr/18 02:10
    Worklog Time Spent: 10m 
      Work Description: kennknowles closed pull request #5120: [BEAM-4160] Add 
JsonToRow transform
URL: https://github.com/apache/beam/pull/5120
 
 
   

This is a PR merged from a forked repository.
As GitHub hides the original diff on merge, it is displayed below for
the sake of provenance:

As this is a foreign pull request (from a fork), the diff is supplied
below (as it won't show otherwise due to GitHub magic):

diff --git 
a/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/JsonToRow.java 
b/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/JsonToRow.java
new file mode 100644
index 00000000000..eb3011df4d4
--- /dev/null
+++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/JsonToRow.java
@@ -0,0 +1,126 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.beam.sdk.transforms;
+
+import com.fasterxml.jackson.databind.ObjectMapper;
+import com.fasterxml.jackson.databind.module.SimpleModule;
+import java.io.IOException;
+import javax.annotation.Nullable;
+import org.apache.beam.sdk.annotations.Experimental;
+import org.apache.beam.sdk.schemas.Schema;
+import org.apache.beam.sdk.schemas.Schema.TypeName;
+import org.apache.beam.sdk.util.RowJsonDeserializer;
+import org.apache.beam.sdk.values.PCollection;
+import org.apache.beam.sdk.values.Row;
+
+/**
+ * <i>Experimental</i>
+ *
+ * <p>Creates a {@link PTransform} to convert input JSON objects to {@link Row 
Rows}
+ * with given {@link Schema}.
+ *
+ * <p>Currently supported {@link Schema} field types are: <ul> <li>{@link 
Schema.TypeName#BYTE}</li>
+ * <li>{@link Schema.TypeName#INT16}</li> <li>{@link 
Schema.TypeName#INT32}</li> <li>{@link
+ * Schema.TypeName#INT64}</li> <li>{@link Schema.TypeName#FLOAT}</li> 
<li>{@link
+ * Schema.TypeName#DOUBLE}</li> <li>{@link Schema.TypeName#BOOLEAN}</li> 
<li>{@link
+ * Schema.TypeName#STRING}</li> </ul>
+ *
+ * <p>For specifics of JSON deserialization see {@link RowJsonDeserializer}.
+ *
+ * <p>Conversion is strict, with minimal type coercion:
+ *
+ * <p>Booleans are only parsed from {@code true} or {@code false} literals, 
not from {@code "true"}
+ * or {@code "false"} strings or any other values (exception is thrown in 
these cases).
+ *
+ * <p>If a JSON number doesn't fit into the corresponding schema field type, 
an exception is be
+ * thrown. Strings are not auto-converted to numbers. Floating point numbers 
are not auto-converted
+ * to integral numbers. Precision loss also causes exceptions.
+ *
+ * <p>Only JSON string values can be parsed into {@link TypeName#STRING}. 
Numbers, booleans are not
+ * automatically converted, exceptions are thrown in these cases.
+ *
+ * <p>If a schema field is missing from the JSON value, an exception will be 
thrown.
+ *
+ * <p>Explicit {@code null} literals are allowed in JSON objects. No other 
values are parsed into
+ * {@code null}.
+ */
+@Experimental
+public class JsonToRow {
+
+  public static PTransform<PCollection<? extends String>, PCollection<Row>> 
withSchema(
+      Schema rowSchema) {
+    return JsonToRowFn.forSchema(rowSchema);
+  }
+
+  static class JsonToRowFn extends PTransform<PCollection<? extends String>, 
PCollection<Row>> {
+    private transient volatile @Nullable ObjectMapper objectMapper;
+    private Schema schema;
+
+    static JsonToRowFn forSchema(Schema rowSchema) {
+      return new JsonToRowFn(rowSchema);
+    }
+
+    private JsonToRowFn(Schema schema) {
+      this.schema = schema;
+    }
+
+    @Override
+    public PCollection<Row> expand(PCollection<? extends String> jsonStrings) {
+      return jsonStrings
+          .apply(
+              ParDo.of(
+                  new DoFn<String, Row>() {
+                    @ProcessElement
+                    public void processElement(ProcessContext context) {
+                      context.output(jsonToRow(context.element()));
+                    }
+                  }))
+          .setCoder(schema.getRowCoder());
+    }
+
+    private Row jsonToRow(String jsonString) {
+      try {
+        return objectMapper().readValue(jsonString, Row.class);
+      } catch (IOException e) {
+        throw new IllegalArgumentException("Unable to parse json object: " + 
jsonString, e);
+      }
+    }
+
+    private ObjectMapper objectMapper() {
+      if (this.objectMapper == null) {
+        synchronized (this) {
+          if (this.objectMapper == null) {
+            this.objectMapper = 
newObjectMapperWith(RowJsonDeserializer.forSchema(this.schema));
+          }
+        }
+      }
+
+      return this.objectMapper;
+    }
+
+    private static ObjectMapper newObjectMapperWith(RowJsonDeserializer 
deserializer) {
+      SimpleModule module = new SimpleModule("rowDeserializationModule");
+      module.addDeserializer(Row.class, deserializer);
+
+      ObjectMapper objectMapper = new ObjectMapper();
+      objectMapper.registerModule(module);
+
+      return objectMapper;
+    }
+  }
+}
diff --git 
a/sdks/java/core/src/test/java/org/apache/beam/sdk/transforms/JsonToRowTest.java
 
b/sdks/java/core/src/test/java/org/apache/beam/sdk/transforms/JsonToRowTest.java
new file mode 100644
index 00000000000..021bf287426
--- /dev/null
+++ 
b/sdks/java/core/src/test/java/org/apache/beam/sdk/transforms/JsonToRowTest.java
@@ -0,0 +1,94 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.beam.sdk.transforms;
+
+import java.io.Serializable;
+import org.apache.beam.sdk.schemas.Schema;
+import org.apache.beam.sdk.testing.NeedsRunner;
+import org.apache.beam.sdk.testing.PAssert;
+import org.apache.beam.sdk.testing.TestPipeline;
+import org.apache.beam.sdk.values.PCollection;
+import org.apache.beam.sdk.values.Row;
+import org.junit.Rule;
+import org.junit.Test;
+import org.junit.experimental.categories.Category;
+import org.junit.runner.RunWith;
+import org.junit.runners.JUnit4;
+
+/**
+ * Unit tests for {@link JsonToRow}.
+ */
+@RunWith(JUnit4.class)
+public class JsonToRowTest implements Serializable {
+
+  private static final boolean NOT_NULLABLE = false;
+
+  @Rule
+  public transient TestPipeline pipeline = TestPipeline.create();
+
+  @Test
+  @Category(NeedsRunner.class)
+  public void testParsesRows() throws Exception {
+    Schema personSchema =
+        Schema
+            .builder()
+            .addStringField("name", NOT_NULLABLE)
+            .addInt32Field("height", NOT_NULLABLE)
+            .addBooleanField("knowsJavascript", NOT_NULLABLE)
+            .build();
+
+    PCollection<String> jsonPersons = pipeline
+        .apply(
+            "jsonPersons",
+            Create.of(
+                jsonPerson("person1", "80", "true"),
+                jsonPerson("person2", "70", "false"),
+                jsonPerson("person3", "60", "true"),
+                jsonPerson("person4", "50", "false"),
+                jsonPerson("person5", "40", "true")));
+
+    PCollection<Row> personRows =
+        jsonPersons
+            .apply(JsonToRow.withSchema(personSchema))
+            .setCoder(personSchema.getRowCoder());
+
+    PAssert
+        .that(personRows)
+        .containsInAnyOrder(
+            row(personSchema, "person1", 80, true),
+            row(personSchema, "person2", 70, false),
+            row(personSchema, "person3", 60, true),
+            row(personSchema, "person4", 50, false),
+            row(personSchema, "person5", 40, true));
+
+    pipeline.run();
+  }
+
+  private String jsonPerson(String name, String height, String knowsJs) {
+    return
+        "{\n"
+        + "  \"name\": \"" + name + "\",\n"
+        + "  \"height\": " + height + ",\n"
+        + "  \"knowsJavascript\": " + knowsJs + "\n"
+        + "}";
+  }
+
+  private Row row(Schema schema, Object... values) {
+    return Row.withSchema(schema).addValues(values).build();
+  }
+}
diff --git 
a/sdks/java/extensions/sql/src/test/java/org/apache/beam/sdk/extensions/sql/JsonToRowSqlTest.java
 
b/sdks/java/extensions/sql/src/test/java/org/apache/beam/sdk/extensions/sql/JsonToRowSqlTest.java
new file mode 100644
index 00000000000..eb7b721b5c1
--- /dev/null
+++ 
b/sdks/java/extensions/sql/src/test/java/org/apache/beam/sdk/extensions/sql/JsonToRowSqlTest.java
@@ -0,0 +1,90 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.beam.sdk.extensions.sql;
+
+import org.apache.beam.sdk.schemas.Schema;
+import org.apache.beam.sdk.testing.PAssert;
+import org.apache.beam.sdk.testing.TestPipeline;
+import org.apache.beam.sdk.transforms.Create;
+import org.apache.beam.sdk.transforms.JsonToRow;
+import org.apache.beam.sdk.values.PCollection;
+import org.apache.beam.sdk.values.Row;
+import org.junit.Rule;
+import org.junit.Test;
+
+/**
+ * JsonToRowSqlTest.
+ */
+public class JsonToRowSqlTest {
+  private static final boolean NOT_NULLABLE = false;
+
+  @Rule
+  public transient TestPipeline pipeline = TestPipeline.create();
+
+  @Test
+  public void testParsesRows() throws Exception {
+    Schema personSchema =
+        Schema
+            .builder()
+            .addStringField("name", NOT_NULLABLE)
+            .addInt32Field("height", NOT_NULLABLE)
+            .addBooleanField("knowsJavascript", NOT_NULLABLE)
+            .build();
+
+    PCollection<String> jsonPersons =
+        pipeline.apply(
+            Create
+                .of(
+                    jsonPerson("person1", "80", "true"),
+                    jsonPerson("person2", "70", "false"),
+                    jsonPerson("person3", "60", "true"),
+                    jsonPerson("person4", "50", "false"),
+                    jsonPerson("person5", "40", "true")));
+
+    Schema resultSchema =
+        Schema
+            .builder()
+            .addInt32Field("avg_height", NOT_NULLABLE)
+            .build();
+
+    PCollection<Row> sqlResult =
+        jsonPersons
+            .apply(JsonToRow.withSchema(personSchema))
+            .apply(BeamSql.query("SELECT AVG(height) as avg_height FROM 
PCOLLECTION"));
+
+    PAssert
+        .that(sqlResult)
+        .containsInAnyOrder(
+            row(resultSchema, 60));
+
+    pipeline.run();
+  }
+
+  private String jsonPerson(String name, String height, String knowsJs) {
+    return
+        "{\n"
+        + "  \"name\": \"" + name + "\",\n"
+        + "  \"height\": " + height + ",\n"
+        + "  \"knowsJavascript\": " + knowsJs + "\n"
+        + "}";
+  }
+
+  private Row row(Schema schema, Object... values) {
+    return Row.withSchema(schema).addValues(values).build();
+  }
+}


 

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
[email protected]


Issue Time Tracking
-------------------

    Worklog Id:     (was: 94919)
    Time Spent: 1h 10m  (was: 1h)

> Convert JSON objects to Rows
> ----------------------------
>
>                 Key: BEAM-4160
>                 URL: https://issues.apache.org/jira/browse/BEAM-4160
>             Project: Beam
>          Issue Type: New Feature
>          Components: dsl-sql, sdk-java-core
>            Reporter: Anton Kedin
>            Assignee: Anton Kedin
>            Priority: Major
>          Time Spent: 1h 10m
>  Remaining Estimate: 0h
>
> Automate conversion of JSON objects to Rows to reduce overhead for querying 
> JSON-based sources



--
This message was sent by Atlassian JIRA
(v7.6.3#76005)

Reply via email to