You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@drill.apache.org by so...@apache.org on 2018/11/02 21:20:00 UTC

[drill] 02/04: DRILL-6810: Disable NULL_IF_NULL NullHandling for functions with ComplexWriter closes #1509

This is an automated email from the ASF dual-hosted git repository.

sorabh pushed a commit to branch master
in repository https://gitbox.apache.org/repos/asf/drill.git

commit f0beeafa9fd3d54dfcbe09b7eec5c7e15036d021
Author: Bohdan Kazydub <bo...@gmail.com>
AuthorDate: Thu Oct 18 18:15:23 2018 +0300

    DRILL-6810: Disable NULL_IF_NULL NullHandling for functions with ComplexWriter
    closes #1509
---
 .../apache/drill/exec/expr/DrillSimpleFunc.java    |   6 +-
 .../exec/expr/annotations/FunctionTemplate.java    |   9 +-
 .../drill/exec/expr/fn/DrillAggFuncHolder.java     |   6 +-
 .../exec/expr/fn/DrillComplexWriterFuncHolder.java |  10 +
 .../apache/drill/exec/expr/fn/DrillFuncHolder.java |  13 ++
 .../apache/drill/exec/expr/fn/impl/Mappify.java    |   5 +-
 .../drill/exec/expr/fn/impl/MappifyUtility.java    |  13 +-
 .../exec/expr/fn/impl/ParseQueryFunction.java      | 168 +++++++++++---
 .../drill/exec/expr/fn/impl/ParseUrlFunction.java  | 252 ++++++++++++---------
 .../drill/exec/expr/fn/impl/StringFunctions.java   |  81 +++++--
 .../drill/exec/expr/fn/impl/conv/DummyFlatten.java |   4 +-
 .../exec/expr/fn/impl/conv/JsonConvertFrom.java    |  97 +++++++-
 .../exec/expr/fn/impl/conv/JsonConvertTo.java      |   4 +-
 .../exec/expr/fn/impl/TestStringFunctions.java     |  26 +++
 .../drill/exec/fn/impl/TestParseFunctions.java     |  86 +++++++
 .../exec/physical/impl/TestConvertFunctions.java   |  30 +++
 .../complex/writer/TestComplexTypeReader.java      |  31 +++
 17 files changed, 654 insertions(+), 187 deletions(-)

diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/expr/DrillSimpleFunc.java b/exec/java-exec/src/main/java/org/apache/drill/exec/expr/DrillSimpleFunc.java
index 3600f81..72b08ba 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/expr/DrillSimpleFunc.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/expr/DrillSimpleFunc.java
@@ -17,7 +17,7 @@
  */
 package org.apache.drill.exec.expr;
 
-public interface DrillSimpleFunc extends DrillFunc{
-  public void setup();
-  public void eval();
+public interface DrillSimpleFunc extends DrillFunc {
+  void setup();
+  void eval();
 }
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/expr/annotations/FunctionTemplate.java b/exec/java-exec/src/main/java/org/apache/drill/exec/expr/annotations/FunctionTemplate.java
index 3aa7365..5bfb238 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/expr/annotations/FunctionTemplate.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/expr/annotations/FunctionTemplate.java
@@ -103,13 +103,13 @@ public @interface FunctionTemplate {
     DEFAULT(OutputWidthCalculators.DefaultOutputWidthCalculator.INSTANCE),
     CLONE(OutputWidthCalculators.CloneOutputWidthCalculator.INSTANCE),
     CONCAT(OutputWidthCalculators.ConcatOutputWidthCalculator.INSTANCE),
-    // Custom calculator are required for functions that don't fall in to any pre-defined
+    // Custom calculator is required for functions that don't fall into any predefined
     // calculator categories - like replace and lpad
     // place holder markers on functions until support
     // for CUSTOM calculators is implemented
-    // CUSTOM_FIXED_WIDTH_DEFUALT will default to a fixed size - for functions like
+    // CUSTOM_FIXED_WIDTH_DEFAULT will default to a fixed size - for functions like
     // lpad() where the ouput size does not easily map to the input size
-    CUSTOM_FIXED_WIDTH_DEFUALT(OutputWidthCalculators.DefaultOutputWidthCalculator.INSTANCE),
+    CUSTOM_FIXED_WIDTH_DEFAULT(OutputWidthCalculators.DefaultOutputWidthCalculator.INSTANCE),
     // CUSTOM CLONE will default to CLONE - for functions like replace() where the output
     // size  does not easily map to the input size but is likely to be at most the size of the input.
     CUSTOM_CLONE_DEFAULT(OutputWidthCalculators.CloneOutputWidthCalculator.INSTANCE);
@@ -138,6 +138,9 @@ public @interface FunctionTemplate {
      * Indicates that a method's associated logical operation returns NULL if
      * either input is NULL, and therefore that the method must not be called
      * with null inputs.  (The calling framework must handle NULLs.)
+     *
+     * <p>Not Supported for aggregate functions and for functions with {@link Output} of type
+     * {@link org.apache.drill.exec.vector.complex.writer.BaseWriter.ComplexWriter}.</p>
      */
     NULL_IF_NULL
   }
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/expr/fn/DrillAggFuncHolder.java b/exec/java-exec/src/main/java/org/apache/drill/exec/expr/fn/DrillAggFuncHolder.java
index e6930f3..76d416e 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/expr/fn/DrillAggFuncHolder.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/expr/fn/DrillAggFuncHolder.java
@@ -65,7 +65,6 @@ class DrillAggFuncHolder extends DrillFuncHolder {
       FunctionAttributes attributes,
       FunctionInitializer initializer) {
     super(attributes, initializer);
-    checkArgument(attributes.getNullHandling() == NullHandling.INTERNAL, "An aggregation function is required to do its own null handling.");
   }
 
   @Override
@@ -263,4 +262,9 @@ class DrillAggFuncHolder extends DrillFuncHolder {
 
   }
 
+  @Override
+  protected void checkNullHandling(NullHandling nullHandling) {
+    checkArgument(nullHandling == NullHandling.INTERNAL,
+        "An aggregate function is required to handle null input(s) on its own.");
+  }
 }
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/expr/fn/DrillComplexWriterFuncHolder.java b/exec/java-exec/src/main/java/org/apache/drill/exec/expr/fn/DrillComplexWriterFuncHolder.java
index 9be1b3a..cb78bec 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/expr/fn/DrillComplexWriterFuncHolder.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/expr/fn/DrillComplexWriterFuncHolder.java
@@ -20,6 +20,7 @@ package org.apache.drill.exec.expr.fn;
 import org.apache.drill.common.expression.FieldReference;
 import org.apache.drill.exec.expr.ClassGenerator;
 import org.apache.drill.exec.expr.ClassGenerator.HoldingContainer;
+import org.apache.drill.exec.expr.annotations.FunctionTemplate.NullHandling;
 import org.apache.drill.exec.physical.impl.project.ProjectRecordBatch;
 import org.apache.drill.exec.record.VectorAccessibleComplexWriter;
 import org.apache.drill.exec.vector.complex.writer.BaseWriter.ComplexWriter;
@@ -31,6 +32,8 @@ import com.sun.codemodel.JExpression;
 import com.sun.codemodel.JInvocation;
 import com.sun.codemodel.JVar;
 
+import static org.apache.drill.shaded.guava.com.google.common.base.Preconditions.checkArgument;
+
 public class DrillComplexWriterFuncHolder extends DrillSimpleFuncHolder {
 
   public DrillComplexWriterFuncHolder(FunctionAttributes functionAttributes, FunctionInitializer initializer) {
@@ -90,4 +93,11 @@ public class DrillComplexWriterFuncHolder extends DrillSimpleFuncHolder {
 
     return null;
   }
+
+  @Override
+  protected void checkNullHandling(NullHandling nullHandling) {
+    checkArgument(nullHandling == NullHandling.INTERNAL,
+        "Function with @Output of type 'org.apache.drill.exec.vector.complex.writer.BaseWriter.ComplexWriter'" +
+            " is required to handle null input(s) on its own.");
+  }
 }
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/expr/fn/DrillFuncHolder.java b/exec/java-exec/src/main/java/org/apache/drill/exec/expr/fn/DrillFuncHolder.java
index fc021c9..ac6738d 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/expr/fn/DrillFuncHolder.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/expr/fn/DrillFuncHolder.java
@@ -63,6 +63,19 @@ public abstract class DrillFuncHolder extends AbstractFuncHolder {
       FunctionInitializer initializer) {
     this.attributes = attributes;
     this.initializer = initializer;
+
+    checkNullHandling(attributes.getNullHandling());
+  }
+
+  /**
+   * Check if function type supports provided null handling strategy.
+   * <p>Keep in mind that this method is invoked in {@link #DrillFuncHolder(FunctionAttributes, FunctionInitializer)}
+   * constructor so make sure not to use any state fields when overriding the method to avoid uninitialized state.</p>
+   *
+   * @param nullHandling null handling strategy defined for a function
+   * @throws IllegalArgumentException if provided {@code nullHandling} is not supported
+   */
+  protected void checkNullHandling(NullHandling nullHandling) {
   }
 
   protected String meth(String methodName) {
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/expr/fn/impl/Mappify.java b/exec/java-exec/src/main/java/org/apache/drill/exec/expr/fn/impl/Mappify.java
index 3db9f5a..ac4c088 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/expr/fn/impl/Mappify.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/expr/fn/impl/Mappify.java
@@ -50,15 +50,18 @@ public class Mappify {
    * scalar value fields
    * value fields need to be of the same data type
    */
-  @FunctionTemplate(names = {"mappify", "kvgen"}, scope = FunctionTemplate.FunctionScope.SIMPLE, nulls = FunctionTemplate.NullHandling.NULL_IF_NULL, isRandom = true)
+  @FunctionTemplate(names = {"mappify", "kvgen"}, scope = FunctionTemplate.FunctionScope.SIMPLE, isRandom = true)
   public static class ConvertMapToKeyValuePairs implements DrillSimpleFunc {
+
     @Param  FieldReader reader;
     @Inject DrillBuf buffer;
     @Output ComplexWriter writer;
 
+    @Override
     public void setup() {
     }
 
+    @Override
     public void eval() {
       buffer = org.apache.drill.exec.expr.fn.impl.MappifyUtility.mappify(reader, writer, buffer, "Mappify/kvgen");
     }
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/expr/fn/impl/MappifyUtility.java b/exec/java-exec/src/main/java/org/apache/drill/exec/expr/fn/impl/MappifyUtility.java
index 0fad0db..25c3ba4 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/expr/fn/impl/MappifyUtility.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/expr/fn/impl/MappifyUtility.java
@@ -38,14 +38,21 @@ public class MappifyUtility {
   public static final String fieldValue = "value";
 
   public static DrillBuf mappify(FieldReader reader, BaseWriter.ComplexWriter writer, DrillBuf buffer, String caller) {
-    // Currently we expect single map as input
-    if (DataMode.REPEATED == reader.getType().getMode() || !(reader.getType().getMinorType() == TypeProtos.MinorType.MAP)) {
+    // Currently we expect single map or null as input
+    if (reader.getType().getMode() == DataMode.REPEATED
+        || (reader.isSet() && reader.getType().getMinorType() != TypeProtos.MinorType.MAP)) {
       throw new DrillRuntimeException("kvgen function only supports Simple maps as input");
     }
     BaseWriter.ListWriter listWriter = writer.rootAsList();
     listWriter.startList();
     BaseWriter.MapWriter mapWriter = listWriter.map();
 
+    if (!reader.isSet()) {
+      // Return empty list
+      listWriter.endList();
+      return buffer;
+    }
+
     // Iterate over the fields in the map
     Iterator<String> fieldIterator = reader.iterator();
     while (fieldIterator.hasNext()) {
@@ -53,7 +60,7 @@ public class MappifyUtility {
       FieldReader fieldReader = reader.reader(str);
 
       // Skip the field if its null
-      if (fieldReader.isSet() == false) {
+      if (!fieldReader.isSet()) {
         mapWriter.end();
         continue;
       }
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/expr/fn/impl/ParseQueryFunction.java b/exec/java-exec/src/main/java/org/apache/drill/exec/expr/fn/impl/ParseQueryFunction.java
index 39e62bb..5fd589d 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/expr/fn/impl/ParseQueryFunction.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/expr/fn/impl/ParseQueryFunction.java
@@ -23,65 +23,165 @@ import org.apache.drill.exec.expr.annotations.FunctionTemplate;
 import org.apache.drill.exec.expr.annotations.Output;
 import org.apache.drill.exec.expr.annotations.Param;
 import org.apache.drill.exec.expr.holders.NullableVarCharHolder;
+import org.apache.drill.exec.expr.holders.VarCharHolder;
 import org.apache.drill.exec.vector.complex.writer.BaseWriter;
 
 import javax.inject.Inject;
 
-@FunctionTemplate(
-        name="parse_query",
-        scope= FunctionTemplate.FunctionScope.SIMPLE,
-        nulls = FunctionTemplate.NullHandling.NULL_IF_NULL
-)
+/**
+ * The {@code parse_query} function splits up a query string and returns a map of the key-value pairs.
+ * If input string contains one or more {@code '?'} characters the string will be
+ * split by the first occurrence of the character and key-value mapping will be performed for
+ * the second part of split string (the part starting after {@code '?'} character) only.
+ *
+ * <p>For example, {@code parse_query('url?arg1=x&arg2=y')} will return:
+ * <pre>
+ * {
+ *   "arg1": "x",
+ *   "arg2": "y"
+ * }
+ * </pre>
+ */
+public class ParseQueryFunction {
 
-public class ParseQueryFunction implements DrillSimpleFunc {
+  @FunctionTemplate(name = "parse_query", scope = FunctionTemplate.FunctionScope.SIMPLE)
+  public static class ParseQuery implements DrillSimpleFunc {
 
     @Param
-    NullableVarCharHolder input;
-
+    VarCharHolder in;
     @Output
     BaseWriter.ComplexWriter outWriter;
-
     @Inject
     DrillBuf outBuffer;
 
+    @Override
     public void setup() {
     }
 
+    @Override
     public void eval() {
-
-        org.apache.drill.exec.vector.complex.writer.BaseWriter.MapWriter queryMapWriter = outWriter.rootAsMap();
-
-        String queryString = org.apache.drill.exec.expr.fn.impl.StringFunctionHelpers.toStringFromUTF8(input.start, input.end, input.buffer);
-
-        if( queryString.isEmpty() || queryString.equals("null")){
-            queryString = "";
+      org.apache.drill.exec.vector.complex.writer.BaseWriter.MapWriter mapWriter = outWriter.rootAsMap();
+
+      String queryString =
+          org.apache.drill.exec.expr.fn.impl.StringFunctionHelpers.toStringFromUTF8(in.start, in.end, in.buffer);
+      // Check if input string contains '?' character. If it does - split the string by the first occurrence
+      // of the character and preserve the part starting from the '?' exclusively so that only part with query
+      // parameters is left.
+      int questionMarkIndex = queryString.indexOf("?");
+      if (questionMarkIndex > -1) {
+        queryString = queryString.substring(questionMarkIndex + 1);
+      }
+
+      if (queryString.trim().isEmpty() || queryString.equalsIgnoreCase("null")) {
+        queryString = "";
+      }
+
+      if (!queryString.isEmpty()) {
+        char firstLetter = queryString.charAt(0);
+
+        // If the first character is a &, it doesn't split properly.
+        // This checks if the first character is an & and if so, removes it.
+        if (firstLetter == '&') {
+          queryString = queryString.substring(1);
+        }
+      }
+
+      String[] queryParameters = queryString.split("&");
+      mapWriter.start();
+      for (String parameter : queryParameters) {
+        String[] keyValue = parameter.split("=", 2);
+        if (keyValue.length != 2) {
+          // Ignore malformed key-value pair
+          continue;
         }
 
-        String firstLetter = queryString.substring(0, 1);
+        byte[] valueBytes = keyValue[1].getBytes();
+        outBuffer.reallocIfNeeded(valueBytes.length);
+        outBuffer.setBytes(0, valueBytes);
 
-        //If the first character is a &, it doesn't split properly.  This checks to see if the first character is an & and if so, removes it.
-        if(firstLetter.equals("&")){
-            queryString = queryString.substring(1);
-        }
+        org.apache.drill.exec.expr.holders.VarCharHolder valueHolder =
+            new org.apache.drill.exec.expr.holders.VarCharHolder();
+        valueHolder.start = 0;
+        valueHolder.end = valueBytes.length;
+        valueHolder.buffer = outBuffer;
 
-        String[] arguments = queryString.split("&");
+        mapWriter.varChar(keyValue[0]).write(valueHolder);
+      }
+      mapWriter.end();
+    }
+  }
 
-        for (int i = 0; i < arguments.length; i++) {
-            String[] queryParts = arguments[i].split("=");
+  @FunctionTemplate(name = "parse_query", scope = FunctionTemplate.FunctionScope.SIMPLE)
+  public static class ParseQueryNullableInput implements DrillSimpleFunc {
 
-            org.apache.drill.exec.expr.holders.VarCharHolder rowHolder = new org.apache.drill.exec.expr.holders.VarCharHolder();
+    @Param
+    NullableVarCharHolder in;
+    @Output
+    BaseWriter.ComplexWriter outWriter;
+    @Inject
+    DrillBuf outBuffer;
 
-            byte[] rowStringBytes = queryParts[1].getBytes();
+    @Override
+    public void setup() {
+    }
 
-            outBuffer.reallocIfNeeded(rowStringBytes.length);
-            outBuffer.setBytes(0, rowStringBytes);
+    @Override
+    public void eval() {
+      org.apache.drill.exec.vector.complex.writer.BaseWriter.MapWriter mapWriter = outWriter.rootAsMap();
+
+      if (in.isSet == 0) {
+        // Return empty map
+        mapWriter.start();
+        mapWriter.end();
+        return;
+      }
+
+      String queryString =
+            org.apache.drill.exec.expr.fn.impl.StringFunctionHelpers.toStringFromUTF8(in.start, in.end, in.buffer);
+      // Check if input string contains '?' character. If it does - split the string by the first occurrence
+      // of the character and preserve the part starting from the '?' exclusively so that only part with query
+      // parameters is left.
+      int questionMarkIndex = queryString.indexOf("?");
+      if (questionMarkIndex > -1) {
+        queryString = queryString.substring(questionMarkIndex + 1);
+      }
+
+      if (queryString.trim().isEmpty() || queryString.equalsIgnoreCase("null")) {
+        queryString = "";
+      }
+
+      if (!queryString.isEmpty()) {
+        char firstLetter = queryString.charAt(0);
+
+        // If the first character is a &, it doesn't split properly.
+        // This checks if the first character is an & and if so, removes it.
+        if (firstLetter == '&') {
+          queryString = queryString.substring(1);
+        }
+      }
+
+      String[] queryParameters = queryString.split("&");
+      mapWriter.start();
+      for (String parameter : queryParameters) {
+        String[] keyValue = parameter.split("=", 2);
+        if (keyValue.length != 2) {
+          // Ignore malformed key-value pair
+          continue;
+        }
 
-            rowHolder.start = 0;
-            rowHolder.end = rowStringBytes.length;
-            rowHolder.buffer = outBuffer;
+        byte[] valueBytes = keyValue[1].getBytes();
+        outBuffer.reallocIfNeeded(valueBytes.length);
+        outBuffer.setBytes(0, valueBytes);
 
-            queryMapWriter.varChar(queryParts[0]).write(rowHolder);
+        org.apache.drill.exec.expr.holders.VarCharHolder valueHolder =
+            new org.apache.drill.exec.expr.holders.VarCharHolder();
+        valueHolder.start = 0;
+        valueHolder.end = valueBytes.length;
+        valueHolder.buffer = outBuffer;
 
-        }
+        mapWriter.varChar(keyValue[0]).write(valueHolder);
+      }
+      mapWriter.end();
     }
-}
\ No newline at end of file
+  }
+}
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/expr/fn/impl/ParseUrlFunction.java b/exec/java-exec/src/main/java/org/apache/drill/exec/expr/fn/impl/ParseUrlFunction.java
index abde5da..52a545a 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/expr/fn/impl/ParseUrlFunction.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/expr/fn/impl/ParseUrlFunction.java
@@ -22,134 +22,172 @@ import org.apache.drill.exec.expr.DrillSimpleFunc;
 import org.apache.drill.exec.expr.annotations.FunctionTemplate;
 import org.apache.drill.exec.expr.annotations.Output;
 import org.apache.drill.exec.expr.annotations.Param;
+import org.apache.drill.exec.expr.holders.NullableVarCharHolder;
 import org.apache.drill.exec.expr.holders.VarCharHolder;
 import org.apache.drill.exec.vector.complex.writer.BaseWriter;
 
 import javax.inject.Inject;
 
-@FunctionTemplate(
-        name="parse_url",
-        scope= FunctionTemplate.FunctionScope.SIMPLE,
-        nulls = FunctionTemplate.NullHandling.NULL_IF_NULL
-)
-
-public class ParseUrlFunction implements DrillSimpleFunc {
-
-    @Param VarCharHolder input;
+/**
+ * The {@code parse_url} function takes an URL and returns a map of components of the URL.
+ * It acts as a wrapper for {@link java.net.URL}. If an optional URL component is absent,
+ * e.g. there is no anchor (reference) element, the {@code "ref"} entry will not be added to resulting map.
+ *
+ * <p>For example, {@code parse_url('http://example.com/some/path?key=value#ref')} will return:
+ * <pre>
+ * {
+ *   "protocol":"http",
+ *   "authority":"example.com",
+ *   "host":"example.com",
+ *   "path":"/some/path",
+ *   "query":"key=value",
+ *   "filename":"/some/path?key=value",
+ *   "ref":"ref"
+ * }
+ * </pre>
+ */
+public class ParseUrlFunction {
 
-    @Output BaseWriter.ComplexWriter outWriter;
+  @FunctionTemplate(name = "parse_url", scope = FunctionTemplate.FunctionScope.SIMPLE)
+  public static class ParseUrl implements DrillSimpleFunc {
 
-    @Inject DrillBuf outBuffer;
+    @Param
+    VarCharHolder in;
+    @Output
+    BaseWriter.ComplexWriter outWriter;
+    @Inject
+    DrillBuf outBuffer;
 
-    public void setup() {}
+    @Override
+    public void setup() {
+    }
 
+    @Override
     public void eval() {
-
-        org.apache.drill.exec.vector.complex.writer.BaseWriter.MapWriter urlMapWriter = outWriter.rootAsMap();
-
-        String urlString = org.apache.drill.exec.expr.fn.impl.StringFunctionHelpers.toStringFromUTF8(input.start, input.end, input.buffer);
-
-        try {
-            java.net.URL aURL = new java.net.URL(urlString);
-
-            String protocol = aURL.getProtocol();
-            String authority = aURL.getAuthority();
-            String host = aURL.getHost();
-            java.lang.Integer port = aURL.getPort();
-            String path = aURL.getPath();
-            String query = aURL.getQuery();
-            String filename = aURL.getFile();
-            String ref = aURL.getRef();
-
-            org.apache.drill.exec.expr.holders.VarCharHolder rowHolder = new org.apache.drill.exec.expr.holders.VarCharHolder();
-
-            byte[] rowStringBytes = protocol.getBytes();
-
-            outBuffer.reallocIfNeeded(rowStringBytes.length);
-            outBuffer.setBytes(0, rowStringBytes);
-
-            rowHolder.start = 0;
-            rowHolder.end = rowStringBytes.length;
-            rowHolder.buffer = outBuffer;
-
-            urlMapWriter.varChar("protocol").write(rowHolder);
-
-
-            byte[] authRowStringBytes = authority.getBytes();
-
-            outBuffer.reallocIfNeeded(authRowStringBytes.length);
-            outBuffer.setBytes(0, authRowStringBytes);
-
-            rowHolder.start = 0;
-            rowHolder.end = authRowStringBytes.length;
-            rowHolder.buffer = outBuffer;
-
-            urlMapWriter.varChar("authority").write(rowHolder);
-
-
-            byte[] hostRowStringBytes = host.getBytes();
-
-            outBuffer.reallocIfNeeded(hostRowStringBytes.length);
-            outBuffer.setBytes(0, hostRowStringBytes);
-
+      org.apache.drill.exec.vector.complex.writer.BaseWriter.MapWriter mapWriter = outWriter.rootAsMap();
+
+      String urlString =
+          org.apache.drill.exec.expr.fn.impl.StringFunctionHelpers.toStringFromUTF8(in.start, in.end, in.buffer);
+      try {
+        mapWriter.start();
+
+        java.net.URL aURL = new java.net.URL(urlString);
+        // Diamond operator can not be used here because Janino does not support
+        // type inference for generic instance creation.
+        // LinkedHashMap is used to preserve insertion-order.
+        java.util.Map<String, String> urlComponents = new java.util.LinkedHashMap<String, String>();
+        urlComponents.put("protocol", aURL.getProtocol());
+        urlComponents.put("authority", aURL.getAuthority());
+        urlComponents.put("host", aURL.getHost());
+        urlComponents.put("path", aURL.getPath());
+        urlComponents.put("query", aURL.getQuery());
+        urlComponents.put("filename", aURL.getFile());
+        urlComponents.put("ref", aURL.getRef());
+
+        org.apache.drill.exec.expr.holders.VarCharHolder rowHolder =
+            new org.apache.drill.exec.expr.holders.VarCharHolder();
+        for (java.util.Map.Entry<String, String> entry : urlComponents.entrySet()) {
+          if (entry.getValue() != null) {
+            // Explicit casting to String is required because of Janino's limitations regarding generics.
+            byte[] protocolBytes = ((String) entry.getValue()).getBytes();
+            outBuffer.reallocIfNeeded(protocolBytes.length);
+            outBuffer.setBytes(0, protocolBytes);
             rowHolder.start = 0;
-            rowHolder.end = hostRowStringBytes.length;
+            rowHolder.end = protocolBytes.length;
             rowHolder.buffer = outBuffer;
+            mapWriter.varChar((String) entry.getKey()).write(rowHolder);
+          }
+        }
 
-            urlMapWriter.varChar("host").write(rowHolder);
-
-
-            byte[] pathRowStringBytes = path.getBytes();
-
-            outBuffer.reallocIfNeeded(pathRowStringBytes.length);
-            outBuffer.setBytes(0, pathRowStringBytes);
-
-            rowHolder.start = 0;
-            rowHolder.end = pathRowStringBytes.length;
-            rowHolder.buffer = outBuffer;
-
-            urlMapWriter.varChar("path").write(rowHolder);
-
-
-            byte[] queryRowStringBytes = query.getBytes();
-
-            outBuffer.reallocIfNeeded(queryRowStringBytes.length);
-            outBuffer.setBytes(0, queryRowStringBytes);
-
-            rowHolder.start = 0;
-            rowHolder.end = queryRowStringBytes.length;
-            rowHolder.buffer = outBuffer;
-
-            urlMapWriter.varChar("query").write(rowHolder);
-
-
-            byte[] filenameRowStringBytes = filename.getBytes();
-
-            outBuffer.reallocIfNeeded(filenameRowStringBytes.length);
-            outBuffer.setBytes(0, filenameRowStringBytes);
-
-            rowHolder.start = 0;
-            rowHolder.end = filenameRowStringBytes.length;
-            rowHolder.buffer = outBuffer;
+        java.lang.Integer port = aURL.getPort();
+        // If port number is not specified in URL string, it is assigned a default value of -1.
+        // Include port number into resulting map only if it was specified.
+        if (port != -1) {
+          org.apache.drill.exec.expr.holders.IntHolder intHolder = new org.apache.drill.exec.expr.holders.IntHolder();
+          intHolder.value = port;
+          mapWriter.integer("port").write(intHolder);
+        }
 
-            urlMapWriter.varChar("filename").write(rowHolder);
+        mapWriter.end();
+      } catch (Exception e) {
+        // Enclose map
+        mapWriter.end();
+      }
+    }
+  }
 
+  @FunctionTemplate(name = "parse_url", scope = FunctionTemplate.FunctionScope.SIMPLE)
+  public static class ParseUrlNullableInput implements DrillSimpleFunc {
 
-            byte[] refRowStringBytes = ref.getBytes();
+    @Param
+    NullableVarCharHolder in;
+    @Output
+    BaseWriter.ComplexWriter outWriter;
+    @Inject
+    DrillBuf outBuffer;
 
-            outBuffer.reallocIfNeeded(refRowStringBytes.length);
-            outBuffer.setBytes(0, refRowStringBytes);
+    @Override
+    public void setup() {
+    }
 
+    @Override
+    public void eval() {
+      org.apache.drill.exec.vector.complex.writer.BaseWriter.MapWriter mapWriter = outWriter.rootAsMap();
+
+      if (in.isSet == 0) {
+        // Return empty map
+        mapWriter.start();
+        mapWriter.end();
+        return;
+      }
+
+      String urlString =
+          org.apache.drill.exec.expr.fn.impl.StringFunctionHelpers.toStringFromUTF8(in.start, in.end, in.buffer);
+      try {
+        mapWriter.start();
+
+        java.net.URL aURL = new java.net.URL(urlString);
+        // Diamond operator can not be used here because Janino does not support
+        // type inference for generic instance creation.
+        // LinkedHashMap is used to preserve insertion-order.
+        java.util.Map<String, String> urlComponents = new java.util.LinkedHashMap<String, String>();
+        urlComponents.put("protocol", aURL.getProtocol());
+        urlComponents.put("authority", aURL.getAuthority());
+        urlComponents.put("host", aURL.getHost());
+        urlComponents.put("path", aURL.getPath());
+        urlComponents.put("query", aURL.getQuery());
+        urlComponents.put("filename", aURL.getFile());
+        urlComponents.put("ref", aURL.getRef());
+
+        org.apache.drill.exec.expr.holders.VarCharHolder rowHolder =
+            new org.apache.drill.exec.expr.holders.VarCharHolder();
+        for (java.util.Map.Entry<String, String> entry : urlComponents.entrySet()) {
+          if (entry.getValue() != null) {
+            // Explicit casting to String is required because of Janino's limitations regarding generics.
+            byte[] protocolBytes = ((String) entry.getValue()).getBytes();
+            outBuffer.reallocIfNeeded(protocolBytes.length);
+            outBuffer.setBytes(0, protocolBytes);
             rowHolder.start = 0;
-            rowHolder.end = refRowStringBytes.length;
+            rowHolder.end = protocolBytes.length;
             rowHolder.buffer = outBuffer;
+            mapWriter.varChar((String) entry.getKey()).write(rowHolder);
+          }
+        }
 
-            urlMapWriter.varChar("ref").write(rowHolder);
-
-            org.apache.drill.exec.expr.holders.IntHolder intHolder = new org.apache.drill.exec.expr.holders.IntHolder();
-            intHolder.value = port;
-            urlMapWriter.integer("port").write(intHolder);
+        java.lang.Integer port = aURL.getPort();
+        // If port number is not specified in URL string, it is assigned a default value of -1.
+        // Include port number into resulting map only if it was specified.
+        if (port != -1) {
+          org.apache.drill.exec.expr.holders.IntHolder intHolder = new org.apache.drill.exec.expr.holders.IntHolder();
+          intHolder.value = port;
+          mapWriter.integer("port").write(intHolder);
         }
-        catch (Exception e ) {}
+
+        mapWriter.end();
+      } catch (Exception e) {
+        // Enclose map
+        mapWriter.end();
+      }
     }
+  }
 }
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/expr/fn/impl/StringFunctions.java b/exec/java-exec/src/main/java/org/apache/drill/exec/expr/fn/impl/StringFunctions.java
index c23a1a2..a77faee 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/expr/fn/impl/StringFunctions.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/expr/fn/impl/StringFunctions.java
@@ -17,7 +17,6 @@
  */
 package org.apache.drill.exec.expr.fn.impl;
 
-
 import io.netty.buffer.ByteBuf;
 import io.netty.buffer.DrillBuf;
 import org.apache.drill.exec.expr.DrillSimpleFunc;
@@ -385,7 +384,7 @@ public class StringFunctions{
 
 
   @FunctionTemplate(name = "split_part", scope = FunctionScope.SIMPLE, nulls = NullHandling.NULL_IF_NULL,
-                    outputWidthCalculatorType = OutputWidthCalculatorType.CUSTOM_FIXED_WIDTH_DEFUALT)
+                    outputWidthCalculatorType = OutputWidthCalculatorType.CUSTOM_FIXED_WIDTH_DEFAULT)
   public static class SplitPart implements DrillSimpleFunc {
     @Param  VarCharHolder str;
     @Param  VarCharHolder splitter;
@@ -645,7 +644,7 @@ public class StringFunctions{
   }
 
   @FunctionTemplate(names = {"substring", "substr" }, scope = FunctionScope.SIMPLE, nulls = NullHandling.INTERNAL,
-                    outputWidthCalculatorType = OutputWidthCalculatorType.CUSTOM_FIXED_WIDTH_DEFUALT)
+                    outputWidthCalculatorType = OutputWidthCalculatorType.CUSTOM_FIXED_WIDTH_DEFAULT)
   public static class SubstringRegexNullable implements DrillSimpleFunc {
     @Param NullableVarCharHolder input;
     @Param(constant=true) VarCharHolder pattern;
@@ -688,7 +687,7 @@ public class StringFunctions{
   // If length = 0, return empty
   // If length < 0, and |length| > total charcounts, return empty.
   @FunctionTemplate(name = "left", scope = FunctionScope.SIMPLE, nulls = NullHandling.NULL_IF_NULL,
-                    outputWidthCalculatorType = OutputWidthCalculatorType.CUSTOM_FIXED_WIDTH_DEFUALT)
+                    outputWidthCalculatorType = OutputWidthCalculatorType.CUSTOM_FIXED_WIDTH_DEFAULT)
   public static class Left implements DrillSimpleFunc {
     @Param VarCharHolder string;
     @Param BigIntHolder length;
@@ -726,7 +725,7 @@ public class StringFunctions{
 
   //Return last 'length' characters in the string. When 'length' is negative, return all but first |length| characters.
   @FunctionTemplate(name = "right", scope = FunctionScope.SIMPLE, nulls = NullHandling.NULL_IF_NULL,
-                    outputWidthCalculatorType = OutputWidthCalculatorType.CUSTOM_FIXED_WIDTH_DEFUALT)
+                    outputWidthCalculatorType = OutputWidthCalculatorType.CUSTOM_FIXED_WIDTH_DEFAULT)
   public static class Right implements DrillSimpleFunc {
     @Param VarCharHolder string;
     @Param BigIntHolder length;
@@ -864,7 +863,7 @@ public class StringFunctions{
       scope = FunctionScope.SIMPLE,
       returnType = ReturnType.PAD,
       nulls = NullHandling.NULL_IF_NULL,
-      outputWidthCalculatorType = OutputWidthCalculatorType.CUSTOM_FIXED_WIDTH_DEFUALT)
+      outputWidthCalculatorType = OutputWidthCalculatorType.CUSTOM_FIXED_WIDTH_DEFAULT)
   public static class Lpad implements DrillSimpleFunc {
     @Param  VarCharHolder text;
     @Param  BigIntHolder length;
@@ -943,7 +942,7 @@ public class StringFunctions{
       scope = FunctionScope.SIMPLE,
       returnType = ReturnType.PAD,
       nulls = NullHandling.NULL_IF_NULL,
-      outputWidthCalculatorType = OutputWidthCalculatorType.CUSTOM_FIXED_WIDTH_DEFUALT)
+      outputWidthCalculatorType = OutputWidthCalculatorType.CUSTOM_FIXED_WIDTH_DEFAULT)
   public static class LpadTwoArg implements DrillSimpleFunc {
     @Param  VarCharHolder text;
     @Param  BigIntHolder length;
@@ -1006,7 +1005,7 @@ public class StringFunctions{
       scope = FunctionScope.SIMPLE,
       returnType = ReturnType.PAD,
       nulls = NullHandling.NULL_IF_NULL,
-      outputWidthCalculatorType = OutputWidthCalculatorType.CUSTOM_FIXED_WIDTH_DEFUALT)
+      outputWidthCalculatorType = OutputWidthCalculatorType.CUSTOM_FIXED_WIDTH_DEFAULT)
   public static class Rpad implements DrillSimpleFunc {
     @Param  VarCharHolder text;
     @Param  BigIntHolder length;
@@ -1088,7 +1087,7 @@ public class StringFunctions{
       scope = FunctionScope.SIMPLE,
       returnType = ReturnType.PAD,
       nulls = NullHandling.NULL_IF_NULL,
-      outputWidthCalculatorType = OutputWidthCalculatorType.CUSTOM_FIXED_WIDTH_DEFUALT)
+      outputWidthCalculatorType = OutputWidthCalculatorType.CUSTOM_FIXED_WIDTH_DEFAULT)
   public static class RpadTwoArg implements DrillSimpleFunc {
     @Param  VarCharHolder text;
     @Param  BigIntHolder length;
@@ -1372,11 +1371,11 @@ public class StringFunctions{
     } // end of eval
   }
 
-  @FunctionTemplate(name = "split", scope = FunctionScope.SIMPLE, nulls = NullHandling.NULL_IF_NULL,
-                    outputWidthCalculatorType = OutputWidthCalculatorType.CUSTOM_FIXED_WIDTH_DEFUALT)
+  @FunctionTemplate(name = "split", scope = FunctionScope.SIMPLE,
+      outputWidthCalculatorType = OutputWidthCalculatorType.CUSTOM_FIXED_WIDTH_DEFAULT)
   public static class Split implements DrillSimpleFunc {
-    @Param  VarCharHolder input;
-    @Param  VarCharHolder delimiter;
+    @Param VarCharHolder in;
+    @Param VarCharHolder delimiter;
 
     @Workspace com.google.common.base.Splitter splitter;
     @Inject DrillBuf buffer;
@@ -1396,20 +1395,66 @@ public class StringFunctions{
 
     @Override
     public void eval() {
+      String inputString =
+          org.apache.drill.exec.expr.fn.impl.StringFunctionHelpers.toStringFromUTF8(in.start, in.end, in.buffer);
       // Convert the iterable to an array as Janino will not handle generics.
-      Object[] tokens = com.google.common.collect.Iterables.toArray(splitter.split(
-          org.apache.drill.exec.expr.fn.impl.StringFunctionHelpers.toStringFromUTF8(input.start, input.end, input.buffer)), String.class);
+      Object[] tokens = com.google.common.collect.Iterables.toArray(splitter.split(inputString), String.class);
       org.apache.drill.exec.vector.complex.writer.BaseWriter.ListWriter list = writer.rootAsList();
       list.startList();
-      for(int i = 0; i < tokens.length; i++ ) {
-        final byte[] strBytes = ((String)tokens[i]).getBytes(com.google.common.base.Charsets.UTF_8);
+      for (Object token : tokens) {
+        final byte[] strBytes = ((String) token).getBytes(com.google.common.base.Charsets.UTF_8);
         buffer = buffer.reallocIfNeeded(strBytes.length);
         buffer.setBytes(0, strBytes);
         list.varChar().writeVarChar(0, strBytes.length, buffer);
       }
       list.endList();
     }
+  }
+
+  @FunctionTemplate(name = "split", scope = FunctionScope.SIMPLE,
+      outputWidthCalculatorType = OutputWidthCalculatorType.CUSTOM_FIXED_WIDTH_DEFAULT)
+  public static class SplitNullableInput implements DrillSimpleFunc {
+    @Param NullableVarCharHolder in;
+    @Param VarCharHolder delimiter;
+
+    @Workspace com.google.common.base.Splitter splitter;
+    @Inject DrillBuf buffer;
+
+    @Output org.apache.drill.exec.vector.complex.writer.BaseWriter.ComplexWriter writer;
 
+    @Override
+    public void setup() {
+      int len = delimiter.end - delimiter.start;
+      if (len != 1) {
+        throw new IllegalArgumentException("Only single character delimiters are supported for split()");
+      }
+      char splitChar = org.apache.drill.exec.expr.fn.impl.StringFunctionHelpers.
+          toStringFromUTF8(delimiter.start, delimiter.end, delimiter.buffer).charAt(0);
+      splitter = com.google.common.base.Splitter.on(splitChar);
+    }
+
+    @Override
+    public void eval() {
+      Object[] tokens;
+      if (in.isSet == 1) {
+        String inputString =
+            org.apache.drill.exec.expr.fn.impl.StringFunctionHelpers.toStringFromUTF8(in.start, in.end, in.buffer);
+        // Convert the iterable to an array as Janino will not handle generics.
+        tokens = com.google.common.collect.Iterables.toArray(splitter.split(inputString), String.class);
+      } else {
+        tokens = new Object[0];
+      }
+      org.apache.drill.exec.vector.complex.writer.BaseWriter.ListWriter list = writer.rootAsList();
+      list.startList();
+      org.apache.drill.exec.vector.complex.writer.VarCharWriter varCharWriter = list.varChar();
+      for (Object token : tokens) {
+        final byte[] strBytes = ((String) token).getBytes(com.google.common.base.Charsets.UTF_8);
+        buffer = buffer.reallocIfNeeded(strBytes.length);
+        buffer.setBytes(0, strBytes);
+        varCharWriter.writeVarChar(0, strBytes.length, buffer);
+      }
+      list.endList();
+    }
   }
 
   @FunctionTemplate(name = "concatOperator",
@@ -1670,7 +1715,7 @@ public class StringFunctions{
   * Returns the input char sequences repeated nTimes.
   */
   @FunctionTemplate(names = {"repeat", "repeatstr"}, scope = FunctionScope.SIMPLE, nulls = NullHandling.NULL_IF_NULL,
-                    outputWidthCalculatorType = OutputWidthCalculatorType.CUSTOM_FIXED_WIDTH_DEFUALT)
+                    outputWidthCalculatorType = OutputWidthCalculatorType.CUSTOM_FIXED_WIDTH_DEFAULT)
   public static class RepeatString implements DrillSimpleFunc {
 
     @Param  VarCharHolder in;
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/expr/fn/impl/conv/DummyFlatten.java b/exec/java-exec/src/main/java/org/apache/drill/exec/expr/fn/impl/conv/DummyFlatten.java
index 943a353..6ac7d78 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/expr/fn/impl/conv/DummyFlatten.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/expr/fn/impl/conv/DummyFlatten.java
@@ -17,11 +17,9 @@
  */
 package org.apache.drill.exec.expr.fn.impl.conv;
 
-
 import org.apache.drill.exec.expr.DrillSimpleFunc;
 import org.apache.drill.exec.expr.annotations.FunctionTemplate;
 import org.apache.drill.exec.expr.annotations.FunctionTemplate.FunctionScope;
-import org.apache.drill.exec.expr.annotations.FunctionTemplate.NullHandling;
 import org.apache.drill.exec.expr.annotations.Output;
 import org.apache.drill.exec.vector.complex.writer.BaseWriter;
 
@@ -29,7 +27,7 @@ import org.apache.drill.exec.vector.complex.writer.BaseWriter;
  * This and {@link DummyConvertTo} class merely act as a placeholder so that Optiq
  * allows the 'flatten()' function in SQL.
  */
-@FunctionTemplate(name = "flatten", scope = FunctionScope.SIMPLE, nulls = NullHandling.NULL_IF_NULL)
+@FunctionTemplate(name = "flatten", scope = FunctionScope.SIMPLE)
 public class DummyFlatten implements DrillSimpleFunc {
 
   @Output BaseWriter.ComplexWriter out;
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/expr/fn/impl/conv/JsonConvertFrom.java b/exec/java-exec/src/main/java/org/apache/drill/exec/expr/fn/impl/conv/JsonConvertFrom.java
index db1dda5..bbfc282 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/expr/fn/impl/conv/JsonConvertFrom.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/expr/fn/impl/conv/JsonConvertFrom.java
@@ -25,10 +25,11 @@ import javax.inject.Inject;
 import org.apache.drill.exec.expr.DrillSimpleFunc;
 import org.apache.drill.exec.expr.annotations.FunctionTemplate;
 import org.apache.drill.exec.expr.annotations.FunctionTemplate.FunctionScope;
-import org.apache.drill.exec.expr.annotations.FunctionTemplate.NullHandling;
 import org.apache.drill.exec.expr.annotations.Output;
 import org.apache.drill.exec.expr.annotations.Param;
 import org.apache.drill.exec.expr.annotations.Workspace;
+import org.apache.drill.exec.expr.holders.NullableVarBinaryHolder;
+import org.apache.drill.exec.expr.holders.NullableVarCharHolder;
 import org.apache.drill.exec.expr.holders.VarBinaryHolder;
 import org.apache.drill.exec.expr.holders.VarCharHolder;
 import org.apache.drill.exec.vector.complex.writer.BaseWriter.ComplexWriter;
@@ -37,10 +38,11 @@ public class JsonConvertFrom {
 
  static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(JsonConvertFrom.class);
 
-  private JsonConvertFrom(){}
+  private JsonConvertFrom() {
+  }
 
-  @FunctionTemplate(name = "convert_fromJSON", scope = FunctionScope.SIMPLE, nulls = NullHandling.NULL_IF_NULL, isRandom = true)
-  public static class ConvertFromJson implements DrillSimpleFunc{
+  @FunctionTemplate(name = "convert_fromJSON", scope = FunctionScope.SIMPLE, isRandom = true)
+  public static class ConvertFromJson implements DrillSimpleFunc {
 
     @Param VarBinaryHolder in;
     @Inject DrillBuf buffer;
@@ -48,27 +50,27 @@ public class JsonConvertFrom {
 
     @Output ComplexWriter writer;
 
-    public void setup(){
+    @Override
+    public void setup() {
       jsonReader = new org.apache.drill.exec.vector.complex.fn.JsonReader.Builder(buffer)
           .defaultSchemaPathColumns()
           .build();
     }
 
-    public void eval(){
-
+    @Override
+    public void eval() {
       try {
         jsonReader.setSource(in.start, in.end, in.buffer);
         jsonReader.write(writer);
         buffer = jsonReader.getWorkBuf();
-
       } catch (Exception e) {
         throw new org.apache.drill.common.exceptions.DrillRuntimeException("Error while converting from JSON. ", e);
       }
     }
   }
 
-  @FunctionTemplate(name = "convert_fromJSON", scope = FunctionScope.SIMPLE, nulls = NullHandling.NULL_IF_NULL, isRandom = true)
-  public static class ConvertFromJsonVarchar implements DrillSimpleFunc{
+  @FunctionTemplate(name = "convert_fromJSON", scope = FunctionScope.SIMPLE, isRandom = true)
+  public static class ConvertFromJsonVarchar implements DrillSimpleFunc {
 
     @Param VarCharHolder in;
     @Inject DrillBuf buffer;
@@ -76,23 +78,94 @@ public class JsonConvertFrom {
 
     @Output ComplexWriter writer;
 
-    public void setup(){
+    @Override
+    public void setup() {
       jsonReader = new org.apache.drill.exec.vector.complex.fn.JsonReader.Builder(buffer)
           .defaultSchemaPathColumns()
           .build();
     }
 
-    public void eval(){
+    @Override
+    public void eval() {
       try {
         jsonReader.setSource(in.start, in.end, in.buffer);
         jsonReader.write(writer);
         buffer = jsonReader.getWorkBuf();
+      } catch (Exception e) {
+        throw new org.apache.drill.common.exceptions.DrillRuntimeException("Error while converting from JSON. ", e);
+      }
+    }
+  }
+
+  @FunctionTemplate(name = "convert_fromJSON", scope = FunctionScope.SIMPLE, isRandom = true)
+  public static class ConvertFromJsonNullableInput implements DrillSimpleFunc {
+
+    @Param NullableVarBinaryHolder in;
+    @Inject DrillBuf buffer;
+    @Workspace org.apache.drill.exec.vector.complex.fn.JsonReader jsonReader;
 
+    @Output ComplexWriter writer;
+
+    @Override
+    public void setup() {
+      jsonReader = new org.apache.drill.exec.vector.complex.fn.JsonReader.Builder(buffer)
+          .defaultSchemaPathColumns()
+          .build();
+    }
+
+    @Override
+    public void eval() {
+      if (in.isSet == 0) {
+        // Return empty map
+        org.apache.drill.exec.vector.complex.writer.BaseWriter.MapWriter mapWriter = writer.rootAsMap();
+        mapWriter.start();
+        mapWriter.end();
+        return;
+      }
+
+      try {
+        jsonReader.setSource(in.start, in.end, in.buffer);
+        jsonReader.write(writer);
+        buffer = jsonReader.getWorkBuf();
       } catch (Exception e) {
         throw new org.apache.drill.common.exceptions.DrillRuntimeException("Error while converting from JSON. ", e);
       }
     }
   }
 
+  @FunctionTemplate(name = "convert_fromJSON", scope = FunctionScope.SIMPLE, isRandom = true)
+  public static class ConvertFromJsonVarcharNullableInput implements DrillSimpleFunc {
+
+    @Param NullableVarCharHolder in;
+    @Inject DrillBuf buffer;
+    @Workspace org.apache.drill.exec.vector.complex.fn.JsonReader jsonReader;
+
+    @Output ComplexWriter writer;
+
+    @Override
+    public void setup() {
+      jsonReader = new org.apache.drill.exec.vector.complex.fn.JsonReader.Builder(buffer)
+          .defaultSchemaPathColumns()
+          .build();
+    }
 
+    @Override
+    public void eval() {
+      if (in.isSet == 0) {
+        // Return empty map
+        org.apache.drill.exec.vector.complex.writer.BaseWriter.MapWriter mapWriter = writer.rootAsMap();
+        mapWriter.start();
+        mapWriter.end();
+        return;
+      }
+
+      try {
+        jsonReader.setSource(in.start, in.end, in.buffer);
+        jsonReader.write(writer);
+        buffer = jsonReader.getWorkBuf();
+      } catch (Exception e) {
+        throw new org.apache.drill.common.exceptions.DrillRuntimeException("Error while converting from JSON. ", e);
+      }
+    }
+  }
 }
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/expr/fn/impl/conv/JsonConvertTo.java b/exec/java-exec/src/main/java/org/apache/drill/exec/expr/fn/impl/conv/JsonConvertTo.java
index 1bb2512..4e69d37 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/expr/fn/impl/conv/JsonConvertTo.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/expr/fn/impl/conv/JsonConvertTo.java
@@ -46,7 +46,7 @@ public class JsonConvertTo {
 
   @FunctionTemplate(names = { "convert_toJSON", "convert_toSIMPLEJSON" },
                     scope = FunctionScope.SIMPLE, nulls = NullHandling.NULL_IF_NULL,
-                    outputWidthCalculatorType = FunctionTemplate.OutputWidthCalculatorType.CUSTOM_FIXED_WIDTH_DEFUALT)
+                    outputWidthCalculatorType = FunctionTemplate.OutputWidthCalculatorType.CUSTOM_FIXED_WIDTH_DEFAULT)
   public static class ConvertToJson implements DrillSimpleFunc{
 
     @Param FieldReader input;
@@ -77,7 +77,7 @@ public class JsonConvertTo {
   }
 
   @FunctionTemplate(name = "convert_toEXTENDEDJSON", scope = FunctionScope.SIMPLE, nulls = NullHandling.NULL_IF_NULL,
-                    outputWidthCalculatorType = FunctionTemplate.OutputWidthCalculatorType.CUSTOM_FIXED_WIDTH_DEFUALT)
+                    outputWidthCalculatorType = FunctionTemplate.OutputWidthCalculatorType.CUSTOM_FIXED_WIDTH_DEFAULT)
   public static class ConvertToExtendedJson implements DrillSimpleFunc{
 
     @Param FieldReader input;
diff --git a/exec/java-exec/src/test/java/org/apache/drill/exec/expr/fn/impl/TestStringFunctions.java b/exec/java-exec/src/test/java/org/apache/drill/exec/expr/fn/impl/TestStringFunctions.java
index 36e7ead..8158c39 100644
--- a/exec/java-exec/src/test/java/org/apache/drill/exec/expr/fn/impl/TestStringFunctions.java
+++ b/exec/java-exec/src/test/java/org/apache/drill/exec/expr/fn/impl/TestStringFunctions.java
@@ -1393,6 +1393,32 @@ public class TestStringFunctions extends BaseTestQuery {
   }
 
   @Test
+  public void testSplitWithNullInput() throws Exception {
+    // Contents of the generated file:
+    /*
+      {"a": "aaaaaa.bbb.cc.ddddd"}
+      {"a": null}
+      {"a": "aa"}
+     */
+    try (BufferedWriter writer = new BufferedWriter(new FileWriter(new File(dirTestWatcher.getRootDir(), "nullable_strings.json")))) {
+      String[] fieldValue = {"\"aaaaaa.bbb.cc.ddddd\"", null, "\"aa\""};
+      for (String value : fieldValue) {
+        String entry = String.format("{ \"a\": %s}\n", value);
+        writer.write(entry);
+      }
+    }
+
+    testBuilder()
+        .sqlQuery("select split(a, '.') wordsCount from dfs.`nullable_strings.json` t")
+        .unOrdered()
+        .baselineColumns("wordsCount")
+        .baselineValues(ImmutableList.of(new Text("aaaaaa"), new Text("bbb"), new Text("cc"), new Text("ddddd")))
+        .baselineValues(ImmutableList.of())
+        .baselineValues(ImmutableList.of(new Text("aa")))
+        .go();
+  }
+
+  @Test
   public void testReverse() throws Exception {
     testBuilder()
       .sqlQuery("select reverse('qwerty') words from (values(1))")
diff --git a/exec/java-exec/src/test/java/org/apache/drill/exec/fn/impl/TestParseFunctions.java b/exec/java-exec/src/test/java/org/apache/drill/exec/fn/impl/TestParseFunctions.java
new file mode 100644
index 0000000..2e4c4b7
--- /dev/null
+++ b/exec/java-exec/src/test/java/org/apache/drill/exec/fn/impl/TestParseFunctions.java
@@ -0,0 +1,86 @@
+/*
+ * 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.fn.impl;
+
+import org.apache.drill.categories.SqlFunctionTest;
+import org.apache.drill.test.ClusterFixture;
+import org.apache.drill.test.ClusterFixtureBuilder;
+import org.apache.drill.test.ClusterTest;
+import org.junit.BeforeClass;
+import org.junit.Test;
+import org.junit.experimental.categories.Category;
+
+import java.io.BufferedWriter;
+import java.io.File;
+import java.io.FileWriter;
+
+import static org.apache.drill.test.TestBuilder.mapOf;
+
+@Category(SqlFunctionTest.class)
+public class TestParseFunctions extends ClusterTest {
+
+  @BeforeClass
+  public static void setup() throws Exception {
+    generateDataSource();
+    ClusterFixtureBuilder builder = ClusterFixture.builder(dirTestWatcher);
+    startCluster(builder);
+  }
+
+  private static void generateDataSource() throws Exception {
+    // Contents of the generated file:
+    /*
+      {"url": "ftp://somewhere.com:3190/someFile?a=12&b=someValue"}
+      {"url": null}
+      {"url": "http://someUrl?p1=v1&p2=v=2&"}
+     */
+    try (BufferedWriter writer = new BufferedWriter(new FileWriter(
+        new File(dirTestWatcher.getRootDir(), "nullable_urls.json")))) {
+      String[] urls = {"\"ftp://somewhere.com:3190/someFile?a=12&b=someValue\"", null, "\"http://someUrl?p1=v1&p2=v=2&\""};
+      for (String url : urls) {
+        String entry = String.format("{ \"url\": %s}\n", url);
+        writer.write(entry);
+      }
+    }
+  }
+
+  @Test
+  public void testParseQueryFunction() throws Exception {
+    testBuilder()
+        .sqlQuery("select parse_query(url) parameters from dfs.`nullable_urls.json`")
+        .unOrdered()
+        .baselineColumns("parameters")
+        .baselineValues(mapOf("a", "12", "b", "someValue"))
+        .baselineValues(mapOf())
+        .baselineValues(mapOf("p1", "v1", "p2", "v=2"))
+        .go();
+  }
+
+  @Test
+  public void testParseUrlFunction() throws Exception {
+    testBuilder()
+        .sqlQuery("select parse_url(url) data from dfs.`nullable_urls.json`")
+        .unOrdered()
+        .baselineColumns("data")
+        .baselineValues(mapOf("protocol", "ftp", "authority", "somewhere.com:3190", "host", "somewhere.com",
+            "path", "/someFile", "query", "a=12&b=someValue", "filename", "/someFile?a=12&b=someValue", "port", 3190))
+        .baselineValues(mapOf())
+        .baselineValues(mapOf("protocol", "http", "authority", "someUrl", "host", "someUrl", "path", "",
+            "query", "p1=v1&p2=v=2&", "filename", "?p1=v1&p2=v=2&"))
+        .go();
+  }
+}
diff --git a/exec/java-exec/src/test/java/org/apache/drill/exec/physical/impl/TestConvertFunctions.java b/exec/java-exec/src/test/java/org/apache/drill/exec/physical/impl/TestConvertFunctions.java
index 9ab09db..d2e0020 100644
--- a/exec/java-exec/src/test/java/org/apache/drill/exec/physical/impl/TestConvertFunctions.java
+++ b/exec/java-exec/src/test/java/org/apache/drill/exec/physical/impl/TestConvertFunctions.java
@@ -25,6 +25,9 @@ import static org.junit.Assert.assertNotNull;
 import static org.junit.Assert.assertTrue;
 import static org.junit.Assert.fail;
 
+import java.io.BufferedWriter;
+import java.io.File;
+import java.io.FileWriter;
 import java.time.LocalDate;
 import java.time.LocalTime;
 import java.util.ArrayList;
@@ -242,6 +245,33 @@ public class TestConvertFunctions extends BaseTestQuery {
   }
 
   @Test
+  public void testConvertFromJsonNullableInput() throws Exception {
+    // Contents of the generated file:
+    /*
+      {"k": "{a: 1, b: 2}"}
+      {"k": null}
+      {"k": "{c: 3}"}
+     */
+    try (BufferedWriter writer = new BufferedWriter(new FileWriter(
+        new File(dirTestWatcher.getRootDir(), "nullable_json_strings.json")))) {
+      String[] fieldValue = {"\"{a: 1, b: 2}\"", null, "\"{c: 3}\""};
+      for (String value : fieldValue) {
+        String entry = String.format("{\"k\": %s}\n", value);
+        writer.write(entry);
+      }
+    }
+
+    testBuilder()
+        .sqlQuery("select convert_from(k, 'json') as col from dfs.`nullable_json_strings.json`")
+        .unOrdered()
+        .baselineColumns("col")
+        .baselineValues(mapOf("a", 1L, "b", 2L))
+        .baselineValues(mapOf())
+        .baselineValues(mapOf("c", 3L))
+        .go();
+  }
+
+  @Test
   public void testConvertToComplexJSON() throws Exception {
 
     String result1 =
diff --git a/exec/java-exec/src/test/java/org/apache/drill/exec/vector/complex/writer/TestComplexTypeReader.java b/exec/java-exec/src/test/java/org/apache/drill/exec/vector/complex/writer/TestComplexTypeReader.java
index 3bc35bf..07f2146 100644
--- a/exec/java-exec/src/test/java/org/apache/drill/exec/vector/complex/writer/TestComplexTypeReader.java
+++ b/exec/java-exec/src/test/java/org/apache/drill/exec/vector/complex/writer/TestComplexTypeReader.java
@@ -24,6 +24,7 @@ import org.junit.Ignore;
 import org.junit.Test;
 
 import java.io.BufferedWriter;
+import java.io.File;
 import java.io.FileWriter;
 import java.io.IOException;
 import java.io.PrintWriter;
@@ -31,6 +32,7 @@ import java.nio.file.Files;
 import java.nio.file.Path;
 
 import static org.apache.drill.test.TestBuilder.listOf;
+import static org.apache.drill.test.TestBuilder.mapOf;
 
 public class TestComplexTypeReader extends BaseTestQuery {
   @BeforeClass
@@ -232,6 +234,35 @@ public class TestComplexTypeReader extends BaseTestQuery {
   }
 
   @Test
+  public void testKVGenWithNullableInput() throws Exception {
+    // Contents of the generated file:
+    /*
+      {"foo": {"obj":1, "bar":10}}
+      {"foo": {"obj":2, "bar":20}}
+      {"foo": null}
+      {"foo": {"obj": null, "bar": 30}}
+     */
+    try (BufferedWriter writer = new BufferedWriter(new FileWriter(
+        new File(dirTestWatcher.getRootDir(), "input_nested.json")))) {
+      String[] fieldValue = {"{\"obj\":1, \"bar\":10}", "{\"obj\":2, \"bar\":20}", null, "{\"obj\": null, \"bar\": 30}"};
+      for (String value : fieldValue) {
+        String entry = String.format("{\"foo\": %s}\n", value);
+        writer.write(entry);
+      }
+    }
+
+    testBuilder()
+        .sqlQuery("select kvgen(foo) kv from dfs.`input_nested.json`")
+        .unOrdered()
+        .baselineColumns("kv")
+        .baselineValues(listOf(mapOf("key", "obj", "value", 1L), mapOf("key", "bar", "value", 10L)))
+        .baselineValues(listOf(mapOf("key", "obj", "value", 2L), mapOf("key", "bar", "value", 20L)))
+        .baselineValues(listOf())
+        .baselineValues(listOf(mapOf("key", "bar", "value", 30L)))
+        .go();
+  }
+
+  @Test
   @Ignore( "until flattening code creates correct ListVector (DRILL-4045)" )
   public void testNestedFlatten() throws Exception {
     test("select flatten(rl) from cp.`jsoninput/input2.json`");