You are viewing a plain text version of this content. The canonical link for it is here.
Posted to dev@drill.apache.org by GitBox <gi...@apache.org> on 2020/03/09 18:45:04 UTC

[GitHub] [drill] paul-rogers opened a new pull request #2016: DRILL-7631: Updates to the Json Structure Parser

paul-rogers opened a new pull request #2016: DRILL-7631: Updates to the Json Structure Parser
URL: https://github.com/apache/drill/pull/2016
 
 
   # [DRILL-7631](https://issues.apache.org/jira/browse/DRILL-7631): Updates to the Json Structure Parser
   
   ## Description
   
   The JSON structure parser converts JSON tokens from the Jackson JSON parser into a structure consisting of values, arrays and objects. The structure parser handles the JSON aspects. An associated set of listeners converts JSON values to some other format (such as to vectors via the column writers.) This PR covers only the structure parser.
   
   Work to create a new JSON record reader using this parser revealed several required revisions:
   
   * Convert the various typed listener creation calls in the listener to a generic call that takes a `ValueDef` value description.
   * Add support to replace listeners as is needed when starting with an "unknown" field, which later resolves to some type, such as `Integer`.
   * Reorganized the code to build sub-parsers based on incoming JSON structure.
   
   Note that, at present, no code uses the JSON structure parser. That code will come in a later PR.
   
   ## Documentation
   
   None
   
   ## Testing
   
   Expanded an reran the associated unit tests. Moved the unit tests so that they are in the same package as the code.
   

----------------------------------------------------------------
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.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [drill] paul-rogers commented on issue #2016: DRILL-7631: Updates to the Json Structure Parser

Posted by GitBox <gi...@apache.org>.
paul-rogers commented on issue #2016: DRILL-7631: Updates to the Json Structure Parser
URL: https://github.com/apache/drill/pull/2016#issuecomment-598033124
 
 
   @vvysotskyi, thanks much for your review! Addressed the review comments. 

----------------------------------------------------------------
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.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [drill] vvysotskyi commented on a change in pull request #2016: DRILL-7631: Updates to the Json Structure Parser

Posted by GitBox <gi...@apache.org>.
vvysotskyi commented on a change in pull request #2016: DRILL-7631: Updates to the Json Structure Parser
URL: https://github.com/apache/drill/pull/2016#discussion_r390423322
 
 

 ##########
 File path: exec/java-exec/src/main/java/org/apache/drill/exec/store/easy/json/parser/ObjectListener.java
 ##########
 @@ -52,80 +52,64 @@
  */
 public interface ObjectListener {
 
+  public enum FieldType {
+
+    /**
+     * The field is unprojected, ignore its content. No value listener
+     * is created.
+     */
+    IGNORE,
+
+    /**
+     * Parse the JSON object according to its type.
+     */
+    TYPED,
+
+    /**
+     * The field is to be treated as "all-text". Used when the parser-level
+     * setting for {@code allTextMode} is {@code false}; allows per-field
+     * overrides to, perhaps, ride over inconsistent scalar types for a
+     * single field. The listener will receive only strings.
+     */
+    TEXT,
+
+    /**
+     * Parse the value, and all its children, as JSON.
+     * That is, converts the parsed JSON back into a
+     * JSON string. The listener will receive only strings.
+     */
+    JSON
+  }
+
   /**
    * Called at the start of a set of values for an object. That is, called
    * when the structure parser accepts the <code>{</code> token.
    */
   void onStart();
 
-  /**
-   * Called at the end of a set of values for an object. That is, called
-   * when the structure parser accepts the <code>}</code> token.
-   */
-  void onEnd();
-
   /**
    * Called by the structure parser when it first sees a new field for
-   * and object to determine if that field is to be projected (is needed
-   * by the listener.) If not projected, the structure parser will not
+   * and object to determine how to parse the field.
+   * If not projected, the structure parser will not
    * ask for a value listener and will insert a "dummy" parser that will
    * free-wheel over any value of that field. As a result, unprojected
    * fields can not cause type errors: they are invisible as long as
    * they are syntactically valid.
+   * <p>
+   * The {@link FieldType#JSON} type says to parse the entire field, and
+   * its children, as a JSON string. The parser will ask for a value
+   * listener to accept the JSON text.
    *
    * @param key the object field name
-   * @return {@code true} if this listener wants to provide a listener
-   * for the field, {@code false} if the field should be ignored
-   */
-  boolean isProjected(String key);
-
-  /**
-   * A new field has appeared with a scalar (or {@code null}) value.
-   * That is: {@code key: <scalar>}.
-   *
-   * @param key the field name
-   * @param type the type as given by the JSON token for the value
-   * @return a value listener for the scalar value
+   * @return how the field should be parsed
    */
-  ValueListener addScalar(String key, JsonType type);
+  FieldType fieldType(String key);
 
-  /**
-   * A new field has appeared with a scalar, {@code null} or empty array
-   * value. That is, one of:
-   * <ul>
-   * <li><code>key: [+ &lt;scalar></code></li>
-   * <li><code>key: [+ null</code></li>
-   * <li><code>key: [+ ]</code></li>
-   * </ul>
-   * Where "[+" means one or more opening array elements.
-   *
-   * @param key the field name
-   * @param arrayDims number of dimensions observed in the first appearance
-   * of the array (more may appear later)
-   * @param type the observed type of the first element of the array, or
-   * {@link JsonType.NULL} if {@code null} was see, or
-   * {@link JsonType.EMPTY} if an empty array was seen
-   * @return a listener for the field itself which is prepared to
-   * return an array listener
-   */
-  ValueListener addArray(String key, int arrayDims, JsonType type);
+  ValueListener addField(String key, ValueDef valueDef);
 
 Review comment:
   Please add JavaDoc.

----------------------------------------------------------------
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.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [drill] vvysotskyi commented on a change in pull request #2016: DRILL-7631: Updates to the Json Structure Parser

Posted by GitBox <gi...@apache.org>.
vvysotskyi commented on a change in pull request #2016: DRILL-7631: Updates to the Json Structure Parser
URL: https://github.com/apache/drill/pull/2016#discussion_r390433673
 
 

 ##########
 File path: exec/java-exec/src/main/java/org/apache/drill/exec/store/easy/json/parser/ObjectListener.java
 ##########
 @@ -52,80 +52,64 @@
  */
 public interface ObjectListener {
 
+  public enum FieldType {
 
 Review comment:
   No need to declare it explicitly as public. Everything declared in the interface is public by default.

----------------------------------------------------------------
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.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [drill] vvysotskyi commented on a change in pull request #2016: DRILL-7631: Updates to the Json Structure Parser

Posted by GitBox <gi...@apache.org>.
vvysotskyi commented on a change in pull request #2016: DRILL-7631: Updates to the Json Structure Parser
URL: https://github.com/apache/drill/pull/2016#discussion_r390435908
 
 

 ##########
 File path: exec/java-exec/src/main/java/org/apache/drill/exec/store/easy/json/parser/ValueDef.java
 ##########
 @@ -0,0 +1,84 @@
+/*
+ * 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.drill.exec.store.easy.json.parser;
+
+/**
+ * Description of a JSON value as inferred from looking ahead in
+ * the JSON stream. Includs a type (which can be empty for an empty
+ * array, or null), and an array size (which is 0 for simple values.)
+ * <p>
+ * To be clear, this is the JSON parser's best guess at a field type
+ * from the input token stream. This is <i>not</i> a description of the
+ * desired data type as JSON can only react to what it sees on input.
+ */
+public class ValueDef {
+
+  /**
+   * Description of JSON types as derived from JSON tokens.
+   */
+  public enum JsonType {
+    OBJECT, NULL, BOOLEAN,
+    INTEGER, FLOAT, STRING, EMBEDDED_OBJECT,
+
+    /**
+     * Indicates an empty array.
+     */
+    EMPTY,
+
+    /**
+     * Indicates an unknown array, appears when replacing the
+     * value listener for an array.
+     */
+    UNKNOWN;
+
+    public boolean isObject() { return this == OBJECT; }
+
+    public boolean isUnknown() {
+      return this == NULL || this == EMPTY ||
+             this == UNKNOWN;
+    }
+
+    public boolean isScalar() {
+      return !isObject() && !isUnknown();
+    }
+  }
+
+  public static final ValueDef UNKNOWN_ARRAY = new ValueDef(JsonType.UNKNOWN, 1);
+  public static final ValueDef UNKNOWN = new ValueDef(JsonType.UNKNOWN, 0);
+
+  private final int arrayDims;
+  private final JsonType type;
+
+  public ValueDef(JsonType type, int dims) {
+    this.type = type;
+    this.arrayDims = dims;
+  }
+
+  public JsonType type() { return type; }
+  public int dimensions() { return arrayDims; }
+  public boolean isArray() { return arrayDims > 0; }
+
+  @Override
+  public String toString() {
+    String result = type.name();
+    for (int i = 0; i < arrayDims; i++) {
+      result += "[]";
 
 Review comment:
   Could you please replace strings concatenation with `StringBuilder` usage.

----------------------------------------------------------------
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.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [drill] asfgit closed pull request #2016: DRILL-7631: Updates to the Json Structure Parser

Posted by GitBox <gi...@apache.org>.
asfgit closed pull request #2016: DRILL-7631: Updates to the Json Structure Parser
URL: https://github.com/apache/drill/pull/2016
 
 
   

----------------------------------------------------------------
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.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [drill] vvysotskyi commented on a change in pull request #2016: DRILL-7631: Updates to the Json Structure Parser

Posted by GitBox <gi...@apache.org>.
vvysotskyi commented on a change in pull request #2016: DRILL-7631: Updates to the Json Structure Parser
URL: https://github.com/apache/drill/pull/2016#discussion_r390434300
 
 

 ##########
 File path: exec/java-exec/src/main/java/org/apache/drill/exec/store/easy/json/parser/ValueDef.java
 ##########
 @@ -0,0 +1,84 @@
+/*
+ * 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.drill.exec.store.easy.json.parser;
+
+/**
+ * Description of a JSON value as inferred from looking ahead in
+ * the JSON stream. Includs a type (which can be empty for an empty
 
 Review comment:
   ```suggestion
    * the JSON stream. Includes a type (which can be empty for an empty
   ```

----------------------------------------------------------------
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.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [drill] vvysotskyi commented on a change in pull request #2016: DRILL-7631: Updates to the Json Structure Parser

Posted by GitBox <gi...@apache.org>.
vvysotskyi commented on a change in pull request #2016: DRILL-7631: Updates to the Json Structure Parser
URL: https://github.com/apache/drill/pull/2016#discussion_r390425236
 
 

 ##########
 File path: exec/java-exec/src/main/java/org/apache/drill/exec/store/easy/json/parser/ValueDefFactory.java
 ##########
 @@ -0,0 +1,99 @@
+/*
+ * 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.drill.exec.store.easy.json.parser;
+
+import org.apache.drill.exec.store.easy.json.parser.ValueDef.JsonType;
+
+import com.fasterxml.jackson.core.JsonToken;
+
+/**
+ * Constructs a {@link ValueDef} by looking ahead on the input stream.
+ * Looking ahead is safe because this class only looks at syntactic
+ * tokens such as <code>{</code>, {@code [} or the first value token.
+ * The underlying JSON parser is left with the first value token
+ * as its current token. Pushes other tokens back on the token stack
+ * so they can be re-consumed by the actual parser.
+ */
+public class ValueDefFactory {
+
+  private int arrayDims;
+  private JsonType jsonType = JsonType.EMPTY;
+
+  public ValueDefFactory(TokenIterator tokenizer) {
+    inferValueType(tokenizer);
+  }
+
+  public static ValueDef lookAhead(TokenIterator tokenizer) {
+    ValueDefFactory factory= new ValueDefFactory(tokenizer);
 
 Review comment:
   ```suggestion
       ValueDefFactory factory = new ValueDefFactory(tokenizer);
   ```

----------------------------------------------------------------
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.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services