You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@beam.apache.org by "ASF GitHub Bot (JIRA)" <ji...@apache.org> on 2018/04/24 22:55:00 UTC

[jira] [Work logged] (BEAM-4160) Convert JSON objects to Rows

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

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

                Author: ASF GitHub Bot
            Created on: 24/Apr/18 22:54
            Start Date: 24/Apr/18 22:54
    Worklog Time Spent: 10m 
      Work Description: kennknowles commented on a change in pull request #5120: [BEAM-4160] Add JsonToRow transform
URL: https://github.com/apache/beam/pull/5120#discussion_r183903319
 
 

 ##########
 File path: sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/JsonToRow.java
 ##########
 @@ -0,0 +1,122 @@
+/*
+ * 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.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;
+
+/**
+ * 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}.
+ */
+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));
 
 Review comment:
   Just curious - racy lazy initialization looks fine here, and if it double initializes you would just get GC'd. Is the library not safe? I kind of recall that it might not be, but I have forgotten the details.

----------------------------------------------------------------
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:
users@infra.apache.org


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

            Worklog Id:     (was: 94841)
            Time Spent: 10m
    Remaining Estimate: 0h

> 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: 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)