You are viewing a plain text version of this content. The canonical link for it is here.
Posted to dev@drill.apache.org by vladimirtkach <gi...@git.apache.org> on 2017/11/07 13:57:17 UTC

[GitHub] drill pull request #1026: DRILL-5919: Add session option to allow json reade...

GitHub user vladimirtkach opened a pull request:

    https://github.com/apache/drill/pull/1026

    DRILL-5919: Add session option to allow json reader/writer to work with NaN,INF

     Added two session options `store.json.reader.non_numeric_numbers` and `store.json.reader.non_numeric_numbers` that allow to read/write `NaN` and `Infinity` as numbers

You can merge this pull request into a Git repository by running:

    $ git pull https://github.com/vladimirtkach/drill DRILL-5919

Alternatively you can review and apply these changes as the patch at:

    https://github.com/apache/drill/pull/1026.patch

To close this pull request, make a commit to your master/trunk branch
with (at least) the following in the commit message:

    This closes #1026
    
----
commit 0e972bac9d472f6681e6f16d232f61e6d0bfcb44
Author: Volodymyr Tkach <vo...@gmail.com>
Date:   2017-11-03T16:13:29Z

    DRILL-5919: Add session option to allow json reader/writer to work with NaN,INF

----


---

[GitHub] drill pull request #1026: DRILL-5919: Add non-numeric support for JSON proce...

Posted by arina-ielchiieva <gi...@git.apache.org>.
Github user arina-ielchiieva commented on a diff in the pull request:

    https://github.com/apache/drill/pull/1026#discussion_r149903182
  
    --- Diff: exec/java-exec/src/test/java/org/apache/drill/exec/vector/complex/writer/TestJsonNonNumerics.java ---
    @@ -0,0 +1,167 @@
    +/*
    +* 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.vector.complex.writer;
    +
    +import com.google.common.collect.ImmutableMap;
    +import org.apache.commons.io.FileUtils;
    +import org.apache.drill.BaseTestQuery;
    +import org.apache.drill.common.exceptions.UserRemoteException;
    +import org.apache.drill.common.expression.SchemaPath;
    +import org.apache.drill.exec.record.RecordBatchLoader;
    +import org.apache.drill.exec.record.VectorWrapper;
    +import org.apache.drill.exec.rpc.user.QueryDataBatch;
    +import org.apache.drill.exec.vector.VarCharVector;
    +import org.junit.Test;
    +
    +import java.io.File;
    +import java.util.List;
    +
    +import static org.hamcrest.CoreMatchers.containsString;
    +import static org.junit.Assert.*;
    +
    +public class TestJsonNonNumerics extends BaseTestQuery {
    +
    +  @Test
    +  public void testNonNumericSelect() throws Exception {
    +    File file = new File(getTempDir(""), "nan_test.json");
    +    String json = "{\"nan\":NaN, \"inf\":Infinity}";
    +    String query = String.format("select * from dfs.`%s`",file.getAbsolutePath());
    +    try {
    +      FileUtils.writeStringToFile(file, json);
    +      test("alter session set `store.json.reader.non_numeric_numbers` = true");
    +      testBuilder()
    +        .sqlQuery(query)
    +        .unOrdered()
    +        .baselineColumns("nan", "inf")
    +        .baselineValues(Double.NaN, Double.POSITIVE_INFINITY)
    +        .build()
    +        .run();
    +    } finally {
    +      test("alter session reset `store.json.reader.non_numeric_numbers`");
    +      FileUtils.deleteQuietly(file);
    +    }
    +  }
    +
    +  @Test(expected = UserRemoteException.class)
    +  public void testNonNumericFailure() throws Exception {
    +    File file = new File(getTempDir(""), "nan_test.json");
    +    test("alter session set `store.json.reader.non_numeric_numbers` = false");
    +    String json = "{\"nan\":NaN, \"inf\":Infinity}";
    +    try {
    +      FileUtils.writeStringToFile(file, json);
    +      test("select * from dfs.`%s`;", file.getAbsolutePath());
    +    } catch (UserRemoteException e) {
    +      assertThat(e.getMessage(), containsString("Error parsing JSON"));
    +      throw e;
    +    } finally {
    +      test("alter session reset `store.json.reader.non_numeric_numbers`");
    +      FileUtils.deleteQuietly(file);
    +    }
    +  }
    +
    +  @Test
    +  public void testCreateTableNonNumerics() throws Exception {
    +    File file = new File(getTempDir(""), "nan_test.json");
    +    String json = "{\"nan\":NaN, \"inf\":Infinity}";
    +    String tableName = "ctas_test";
    +    try {
    +      FileUtils.writeStringToFile(file, json);
    +      test("alter session set `store.json.reader.non_numeric_numbers` = true");
    +      test("alter session set `store.json.writer.non_numeric_numbers` = true");
    +      test("alter session set `store.format`='json'");
    +      test("create table dfs_test.tmp.`%s` as select * from dfs.`%s`;", tableName, file.getAbsolutePath());
    +
    +      // ensuring that `NaN` and `Infinity` tokens ARE NOT enclosed with double quotes
    +      File resultFile = new File(new File(getDfsTestTmpSchemaLocation(),tableName),"0_0_0.json");
    +      String resultJson = FileUtils.readFileToString(resultFile);
    +      int nanIndex = resultJson.indexOf("NaN");
    +      assertFalse("`NaN` must not be enclosed with \"\" ", resultJson.charAt(nanIndex - 1) == '"');
    +      assertFalse("`NaN` must not be enclosed with \"\" ", resultJson.charAt(nanIndex + "NaN".length()) == '"');
    +      int infIndex = resultJson.indexOf("Infinity");
    +      assertFalse("`Infinity` must not be enclosed with \"\" ", resultJson.charAt(infIndex - 1) == '"');
    +      assertFalse("`Infinity` must not be enclosed with \"\" ", resultJson.charAt(infIndex + "Infinity".length()) == '"');
    +    } finally {
    +      test("drop table if exists dfs_test.tmp.`%s`", tableName);
    +      test("alter session reset `store.json.reader.non_numeric_numbers`");
    +      test("alter session reset `store.json.writer.non_numeric_numbers`");
    +      FileUtils.deleteQuietly(file);
    +    }
    +  }
    +
    +  @Test
    +  public void testConvertFromJsonFunction() throws Exception {
    +    File file = new File(getTempDir(""), "nan_test.csv");
    +    String csv = "col_0, {\"nan\":NaN}";
    +    FileUtils.writeStringToFile(file, csv);
    --- End diff --
    
    Should be in try-catch.


---

[GitHub] drill pull request #1026: DRILL-5919: Add non-numeric support for JSON proce...

Posted by vladimirtkach <gi...@git.apache.org>.
Github user vladimirtkach commented on a diff in the pull request:

    https://github.com/apache/drill/pull/1026#discussion_r150500913
  
    --- Diff: exec/java-exec/src/main/resources/drill-module.conf ---
    @@ -502,6 +502,8 @@ drill.exec.options: {
         store.format: "parquet",
         store.hive.optimize_scan_with_native_readers: false,
         store.json.all_text_mode: false,
    +    store.json.writer.non_numeric_numbers: false,
    +    store.json.reader.non_numeric_numbers: false,
    --- End diff --
    
    I think we should stick to json standard and leave them switched off by default. If user get an exception we may show the option name which he want to switch on.  


---

[GitHub] drill pull request #1026: DRILL-5919: Add non-numeric support for JSON proce...

Posted by asfgit <gi...@git.apache.org>.
Github user asfgit closed the pull request at:

    https://github.com/apache/drill/pull/1026


---

[GitHub] drill pull request #1026: DRILL-5919: Add non-numeric support for JSON proce...

Posted by paul-rogers <gi...@git.apache.org>.
Github user paul-rogers commented on a diff in the pull request:

    https://github.com/apache/drill/pull/1026#discussion_r150360262
  
    --- Diff: exec/java-exec/src/main/resources/drill-module.conf ---
    @@ -502,6 +502,8 @@ drill.exec.options: {
         store.format: "parquet",
         store.hive.optimize_scan_with_native_readers: false,
         store.json.all_text_mode: false,
    +    store.json.writer.non_numeric_numbers: false,
    +    store.json.reader.non_numeric_numbers: false,
    --- End diff --
    
    Any reason these are not enabled by default?


---

[GitHub] drill issue #1026: DRILL-5919: Add non-numeric support for JSON processing

Posted by arina-ielchiieva <gi...@git.apache.org>.
Github user arina-ielchiieva commented on the issue:

    https://github.com/apache/drill/pull/1026
  
    Thanks, +1, LGTM.


---

[GitHub] drill issue #1026: DRILL-5919: Add non-numeric support for JSON processing

Posted by arina-ielchiieva <gi...@git.apache.org>.
Github user arina-ielchiieva commented on the issue:

    https://github.com/apache/drill/pull/1026
  
    @vladimirtkach thanks for adding tests for math functions. New changes look good to me.
    @paul-rogers please take a look.


---

[GitHub] drill pull request #1026: DRILL-5919: Add non-numeric support for JSON proce...

Posted by paul-rogers <gi...@git.apache.org>.
Github user paul-rogers commented on a diff in the pull request:

    https://github.com/apache/drill/pull/1026#discussion_r150359992
  
    --- Diff: contrib/storage-mongo/src/main/java/org/apache/drill/exec/store/mongo/MongoRecordReader.java ---
    @@ -73,6 +73,7 @@
       private final MongoStoragePlugin plugin;
     
       private final boolean enableAllTextMode;
    +  private final boolean enableNonNumericNumbers;
    --- End diff --
    
    Would recommend: `enableNanInf`.
    
    "Non-numeric numbers" sounds like we might allow "foo" or "thirteen".


---

[GitHub] drill pull request #1026: DRILL-5919: Add non-numeric support for JSON proce...

Posted by paul-rogers <gi...@git.apache.org>.
Github user paul-rogers commented on a diff in the pull request:

    https://github.com/apache/drill/pull/1026#discussion_r150689617
  
    --- Diff: contrib/storage-mongo/src/main/java/org/apache/drill/exec/store/mongo/MongoRecordReader.java ---
    @@ -73,6 +73,7 @@
       private final MongoStoragePlugin plugin;
     
       private final boolean enableAllTextMode;
    +  private final boolean enableNonNumericNumbers;
    --- End diff --
    
    The JSON parser we use does use the term "non-numeric numbers", but not sure we want to keep that naming in Drill itself.


---

[GitHub] drill pull request #1026: DRILL-5919: Add non-numeric support for JSON proce...

Posted by vladimirtkach <gi...@git.apache.org>.
Github user vladimirtkach commented on a diff in the pull request:

    https://github.com/apache/drill/pull/1026#discussion_r150500059
  
    --- Diff: exec/java-exec/src/main/resources/drill-module.conf ---
    @@ -502,6 +502,8 @@ drill.exec.options: {
         store.format: "parquet",
         store.hive.optimize_scan_with_native_readers: false,
         store.json.all_text_mode: false,
    +    store.json.writer.non_numeric_numbers: false,
    +    store.json.reader.non_numeric_numbers: false,
    --- End diff --
    
    No, we don't and we should test them. I think we have to create separate jira for testing math functions, because code changes from this PR doesn't affect logic of any math function, and while testing math function there will be other issues not connected with with functionality (like BigInteger constructor doesn't accept NaN, Infinity and others)


---

[GitHub] drill issue #1026: DRILL-5919: Add non-numeric support for JSON processing

Posted by vladimirtkach <gi...@git.apache.org>.
Github user vladimirtkach commented on the issue:

    https://github.com/apache/drill/pull/1026
  
    @arina-ielchiieva made code changes according to your comments.


---

[GitHub] drill pull request #1026: DRILL-5919: Add non-numeric support for JSON proce...

Posted by paul-rogers <gi...@git.apache.org>.
Github user paul-rogers commented on a diff in the pull request:

    https://github.com/apache/drill/pull/1026#discussion_r150360475
  
    --- Diff: exec/java-exec/src/test/java/org/apache/drill/exec/vector/complex/writer/TestJsonNonNumerics.java ---
    @@ -0,0 +1,167 @@
    +/*
    +* 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.vector.complex.writer;
    +
    +import com.google.common.collect.ImmutableMap;
    +import org.apache.commons.io.FileUtils;
    +import org.apache.drill.BaseTestQuery;
    +import org.apache.drill.common.exceptions.UserRemoteException;
    +import org.apache.drill.common.expression.SchemaPath;
    +import org.apache.drill.exec.record.RecordBatchLoader;
    +import org.apache.drill.exec.record.VectorWrapper;
    +import org.apache.drill.exec.rpc.user.QueryDataBatch;
    +import org.apache.drill.exec.vector.VarCharVector;
    +import org.junit.Test;
    +
    +import java.io.File;
    +import java.util.List;
    +
    +import static org.hamcrest.CoreMatchers.containsString;
    +import static org.junit.Assert.*;
    +
    +public class TestJsonNonNumerics extends BaseTestQuery {
    +
    +  @Test
    +  public void testNonNumericSelect() throws Exception {
    +    File file = new File(getTempDir("nan_test"), "nan_test.json");
    +    String json = "{\"nan\":NaN, \"inf\":Infinity}";
    +    String query = String.format("select * from dfs.`%s`",file.getAbsolutePath());
    +    try {
    +      FileUtils.writeStringToFile(file, json);
    +      test("alter session set `store.json.reader.non_numeric_numbers` = true");
    +      testBuilder()
    +        .sqlQuery(query)
    +        .unOrdered()
    +        .baselineColumns("nan", "inf")
    +        .baselineValues(Double.NaN, Double.POSITIVE_INFINITY)
    +        .build()
    +        .run();
    +    } finally {
    +      test("alter session reset `store.json.reader.non_numeric_numbers`");
    +      FileUtils.deleteQuietly(file);
    +    }
    +  }
    +
    +  @Test(expected = UserRemoteException.class)
    +  public void testNonNumericFailure() throws Exception {
    +    File file = new File(getTempDir("nan_test"), "nan_test.json");
    +    test("alter session set `store.json.reader.non_numeric_numbers` = false");
    +    String json = "{\"nan\":NaN, \"inf\":Infinity}";
    +    try {
    +      FileUtils.writeStringToFile(file, json);
    +      test("select * from dfs.`%s`;", file.getAbsolutePath());
    +    } catch (UserRemoteException e) {
    +      assertThat(e.getMessage(), containsString("Error parsing JSON"));
    +      throw e;
    +    } finally {
    +      test("alter session reset `store.json.reader.non_numeric_numbers`");
    +      FileUtils.deleteQuietly(file);
    +    }
    +  }
    +
    +  @Test
    +  public void testCreateTableNonNumerics() throws Exception {
    +    File file = new File(getTempDir("nan_test"), "nan_test.json");
    +    String json = "{\"nan\":NaN, \"inf\":Infinity}";
    +    String tableName = "ctas_test";
    +    try {
    +      FileUtils.writeStringToFile(file, json);
    +      test("alter session set `store.json.reader.non_numeric_numbers` = true");
    +      test("alter session set `store.json.writer.non_numeric_numbers` = true");
    +      test("alter session set `store.format`='json'");
    +      test("create table dfs_test.tmp.`%s` as select * from dfs.`%s`;", tableName, file.getAbsolutePath());
    +
    +      // ensuring that `NaN` and `Infinity` tokens ARE NOT enclosed with double quotes
    +      File resultFile = new File(new File(getDfsTestTmpSchemaLocation(),tableName),"0_0_0.json");
    +      String resultJson = FileUtils.readFileToString(resultFile);
    +      int nanIndex = resultJson.indexOf("NaN");
    +      assertFalse("`NaN` must not be enclosed with \"\" ", resultJson.charAt(nanIndex - 1) == '"');
    +      assertFalse("`NaN` must not be enclosed with \"\" ", resultJson.charAt(nanIndex + "NaN".length()) == '"');
    +      int infIndex = resultJson.indexOf("Infinity");
    +      assertFalse("`Infinity` must not be enclosed with \"\" ", resultJson.charAt(infIndex - 1) == '"');
    +      assertFalse("`Infinity` must not be enclosed with \"\" ", resultJson.charAt(infIndex + "Infinity".length()) == '"');
    +    } finally {
    +      test("drop table if exists dfs_test.tmp.`%s`", tableName);
    +      test("alter session reset `store.json.reader.non_numeric_numbers`");
    --- End diff --
    
    Please consider using the new `alterSession()` and `resetSession()` methods once they are available.


---

[GitHub] drill pull request #1026: DRILL-5919: Add non-numeric support for JSON proce...

Posted by paul-rogers <gi...@git.apache.org>.
Github user paul-rogers commented on a diff in the pull request:

    https://github.com/apache/drill/pull/1026#discussion_r150688949
  
    --- Diff: exec/java-exec/src/main/resources/drill-module.conf ---
    @@ -502,6 +502,8 @@ drill.exec.options: {
         store.format: "parquet",
         store.hive.optimize_scan_with_native_readers: false,
         store.json.all_text_mode: false,
    +    store.json.writer.non_numeric_numbers: false,
    +    store.json.reader.non_numeric_numbers: false,
    --- End diff --
    
    See discussion below. If they are off, then queries that use NaN and Infinity will fail until the user turns them on. Queries that don't use NaN and Infinity won't care. So, what is the advantage to failing queries unnecessarily?


---

[GitHub] drill issue #1026: DRILL-5919: Add non-numeric support for JSON processing

Posted by paul-rogers <gi...@git.apache.org>.
Github user paul-rogers commented on the issue:

    https://github.com/apache/drill/pull/1026
  
    Further, is the extra option to `convertFromJSON` really needed? Can't we just accept `NaN` and `Infinity` by default?
    
    Consider. If the option is off by default, users without `NaN` or `Infinity` data will see no difference. But, users will this data will get an error and have to hunt down the option to make their data work.
    
    If the option is on by default, users without `NaN` or `Infinity` data will see no difference. But, users will this data will also have their queries work by default.
    
    So, seems no harm in making the `NaN` and `Infinity` support turned on by default.


---

[GitHub] drill pull request #1026: DRILL-5919: Add non-numeric support for JSON proce...

Posted by paul-rogers <gi...@git.apache.org>.
Github user paul-rogers commented on a diff in the pull request:

    https://github.com/apache/drill/pull/1026#discussion_r150360424
  
    --- Diff: exec/java-exec/src/test/java/org/apache/drill/exec/vector/complex/writer/TestJsonNonNumerics.java ---
    @@ -0,0 +1,167 @@
    +/*
    +* 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.vector.complex.writer;
    +
    +import com.google.common.collect.ImmutableMap;
    +import org.apache.commons.io.FileUtils;
    +import org.apache.drill.BaseTestQuery;
    +import org.apache.drill.common.exceptions.UserRemoteException;
    +import org.apache.drill.common.expression.SchemaPath;
    +import org.apache.drill.exec.record.RecordBatchLoader;
    +import org.apache.drill.exec.record.VectorWrapper;
    +import org.apache.drill.exec.rpc.user.QueryDataBatch;
    +import org.apache.drill.exec.vector.VarCharVector;
    +import org.junit.Test;
    +
    +import java.io.File;
    +import java.util.List;
    +
    +import static org.hamcrest.CoreMatchers.containsString;
    +import static org.junit.Assert.*;
    +
    +public class TestJsonNonNumerics extends BaseTestQuery {
    +
    +  @Test
    +  public void testNonNumericSelect() throws Exception {
    +    File file = new File(getTempDir("nan_test"), "nan_test.json");
    +    String json = "{\"nan\":NaN, \"inf\":Infinity}";
    +    String query = String.format("select * from dfs.`%s`",file.getAbsolutePath());
    +    try {
    +      FileUtils.writeStringToFile(file, json);
    +      test("alter session set `store.json.reader.non_numeric_numbers` = true");
    +      testBuilder()
    +        .sqlQuery(query)
    +        .unOrdered()
    +        .baselineColumns("nan", "inf")
    +        .baselineValues(Double.NaN, Double.POSITIVE_INFINITY)
    +        .build()
    +        .run();
    +    } finally {
    +      test("alter session reset `store.json.reader.non_numeric_numbers`");
    +      FileUtils.deleteQuietly(file);
    +    }
    +  }
    +
    +  @Test(expected = UserRemoteException.class)
    +  public void testNonNumericFailure() throws Exception {
    +    File file = new File(getTempDir("nan_test"), "nan_test.json");
    +    test("alter session set `store.json.reader.non_numeric_numbers` = false");
    +    String json = "{\"nan\":NaN, \"inf\":Infinity}";
    +    try {
    +      FileUtils.writeStringToFile(file, json);
    +      test("select * from dfs.`%s`;", file.getAbsolutePath());
    +    } catch (UserRemoteException e) {
    +      assertThat(e.getMessage(), containsString("Error parsing JSON"));
    +      throw e;
    +    } finally {
    +      test("alter session reset `store.json.reader.non_numeric_numbers`");
    +      FileUtils.deleteQuietly(file);
    +    }
    +  }
    +
    +  @Test
    +  public void testCreateTableNonNumerics() throws Exception {
    +    File file = new File(getTempDir("nan_test"), "nan_test.json");
    +    String json = "{\"nan\":NaN, \"inf\":Infinity}";
    +    String tableName = "ctas_test";
    +    try {
    +      FileUtils.writeStringToFile(file, json);
    +      test("alter session set `store.json.reader.non_numeric_numbers` = true");
    +      test("alter session set `store.json.writer.non_numeric_numbers` = true");
    +      test("alter session set `store.format`='json'");
    +      test("create table dfs_test.tmp.`%s` as select * from dfs.`%s`;", tableName, file.getAbsolutePath());
    +
    +      // ensuring that `NaN` and `Infinity` tokens ARE NOT enclosed with double quotes
    +      File resultFile = new File(new File(getDfsTestTmpSchemaLocation(),tableName),"0_0_0.json");
    +      String resultJson = FileUtils.readFileToString(resultFile);
    +      int nanIndex = resultJson.indexOf("NaN");
    +      assertFalse("`NaN` must not be enclosed with \"\" ", resultJson.charAt(nanIndex - 1) == '"');
    +      assertFalse("`NaN` must not be enclosed with \"\" ", resultJson.charAt(nanIndex + "NaN".length()) == '"');
    +      int infIndex = resultJson.indexOf("Infinity");
    +      assertFalse("`Infinity` must not be enclosed with \"\" ", resultJson.charAt(infIndex - 1) == '"');
    +      assertFalse("`Infinity` must not be enclosed with \"\" ", resultJson.charAt(infIndex + "Infinity".length()) == '"');
    +    } finally {
    +      test("drop table if exists dfs_test.tmp.`%s`", tableName);
    +      test("alter session reset `store.json.reader.non_numeric_numbers`");
    --- End diff --
    
    Should be done in a finally block to ensure these get reset even if the test fails.


---

[GitHub] drill pull request #1026: DRILL-5919: Add non-numeric support for JSON proce...

Posted by paul-rogers <gi...@git.apache.org>.
Github user paul-rogers commented on a diff in the pull request:

    https://github.com/apache/drill/pull/1026#discussion_r150360192
  
    --- Diff: exec/java-exec/src/main/resources/drill-module.conf ---
    @@ -502,6 +502,8 @@ drill.exec.options: {
         store.format: "parquet",
         store.hive.optimize_scan_with_native_readers: false,
         store.json.all_text_mode: false,
    +    store.json.writer.non_numeric_numbers: false,
    +    store.json.reader.non_numeric_numbers: false,
    --- End diff --
    
    `allow_nan_inf`?


---

[GitHub] drill pull request #1026: DRILL-5919: Add non-numeric support for JSON proce...

Posted by paul-rogers <gi...@git.apache.org>.
Github user paul-rogers commented on a diff in the pull request:

    https://github.com/apache/drill/pull/1026#discussion_r150360309
  
    --- Diff: exec/java-exec/src/main/resources/drill-module.conf ---
    @@ -502,6 +502,8 @@ drill.exec.options: {
         store.format: "parquet",
         store.hive.optimize_scan_with_native_readers: false,
         store.json.all_text_mode: false,
    +    store.json.writer.non_numeric_numbers: false,
    +    store.json.reader.non_numeric_numbers: false,
    --- End diff --
    
    Have we tested all Drill's floating point methods to ensure that they correctly handle NaN and INF?


---

[GitHub] drill pull request #1026: DRILL-5919: Add non-numeric support for JSON proce...

Posted by paul-rogers <gi...@git.apache.org>.
Github user paul-rogers commented on a diff in the pull request:

    https://github.com/apache/drill/pull/1026#discussion_r150688725
  
    --- Diff: exec/java-exec/src/main/resources/drill-module.conf ---
    @@ -502,6 +502,8 @@ drill.exec.options: {
         store.format: "parquet",
         store.hive.optimize_scan_with_native_readers: false,
         store.json.all_text_mode: false,
    +    store.json.writer.non_numeric_numbers: false,
    +    store.json.reader.non_numeric_numbers: false,
    --- End diff --
    
    As noted in DRILL-5949, options like this should be part of the plugin config (as in CSV), not session options. For now, let's leave them as session options and I'll migrate them to plugin config options along with the others.


---

[GitHub] drill pull request #1026: DRILL-5919: Add non-numeric support for JSON proce...

Posted by arina-ielchiieva <gi...@git.apache.org>.
Github user arina-ielchiieva commented on a diff in the pull request:

    https://github.com/apache/drill/pull/1026#discussion_r149903705
  
    --- Diff: exec/java-exec/src/test/java/org/apache/drill/exec/vector/complex/writer/TestJsonNonNumerics.java ---
    @@ -0,0 +1,167 @@
    +/*
    +* 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.vector.complex.writer;
    +
    +import com.google.common.collect.ImmutableMap;
    +import org.apache.commons.io.FileUtils;
    +import org.apache.drill.BaseTestQuery;
    +import org.apache.drill.common.exceptions.UserRemoteException;
    +import org.apache.drill.common.expression.SchemaPath;
    +import org.apache.drill.exec.record.RecordBatchLoader;
    +import org.apache.drill.exec.record.VectorWrapper;
    +import org.apache.drill.exec.rpc.user.QueryDataBatch;
    +import org.apache.drill.exec.vector.VarCharVector;
    +import org.junit.Test;
    +
    +import java.io.File;
    +import java.util.List;
    +
    +import static org.hamcrest.CoreMatchers.containsString;
    +import static org.junit.Assert.*;
    +
    +public class TestJsonNonNumerics extends BaseTestQuery {
    +
    +  @Test
    +  public void testNonNumericSelect() throws Exception {
    +    File file = new File(getTempDir(""), "nan_test.json");
    --- End diff --
    
    It's better to pass dir name as well, rather than emptiness. Ex: `getTempDir("test_nan")`


---

[GitHub] drill pull request #1026: DRILL-5919: Add non-numeric support for JSON proce...

Posted by paul-rogers <gi...@git.apache.org>.
Github user paul-rogers commented on a diff in the pull request:

    https://github.com/apache/drill/pull/1026#discussion_r150359565
  
    --- Diff: exec/java-exec/src/main/java/org/apache/drill/exec/expr/fn/impl/conv/JsonConvertFrom.java ---
    @@ -91,4 +92,60 @@ public void eval(){
         }
       }
     
    +  @FunctionTemplate(name = "convert_fromJSON", scope = FunctionScope.SIMPLE, nulls = NullHandling.NULL_IF_NULL, isRandom = true)
    +  public static class ConvertFromJsonVarcharNonNumerics implements DrillSimpleFunc{
    +
    +    @Param VarCharHolder in;
    +    @Param BitHolder enableNonNumeric;
    +    @Inject DrillBuf buffer;
    +    @Workspace org.apache.drill.exec.vector.complex.fn.JsonReader jsonReader;
    +
    +    @Output ComplexWriter writer;
    +
    +    public void setup(){
    +      jsonReader = new org.apache.drill.exec.vector.complex.fn.JsonReader(buffer, false, false, false,/* do not read numbers as doubles */
    +          enableNonNumeric.value == 1);
    +    }
    +
    +    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 ConvertFromJsonNonNumerics implements DrillSimpleFunc{
    +
    +    @Param VarBinaryHolder in;
    +    @Param BitHolder enableNonNumeric;
    +    @Inject DrillBuf buffer;
    +    @Workspace org.apache.drill.exec.vector.complex.fn.JsonReader jsonReader;
    +
    +    @Output ComplexWriter writer;
    +
    +    public void setup(){
    +      jsonReader = new org.apache.drill.exec.vector.complex.fn.JsonReader(buffer, false, false, false, /* do not read numbers as doubles */
    --- End diff --
    
    See above. We really don't want all these duplicate copies.


---

[GitHub] drill pull request #1026: DRILL-5919: Add non-numeric support for JSON proce...

Posted by paul-rogers <gi...@git.apache.org>.
Github user paul-rogers commented on a diff in the pull request:

    https://github.com/apache/drill/pull/1026#discussion_r150359510
  
    --- Diff: exec/java-exec/src/main/java/org/apache/drill/exec/expr/fn/impl/conv/JsonConvertFrom.java ---
    @@ -91,4 +92,60 @@ public void eval(){
         }
       }
     
    +  @FunctionTemplate(name = "convert_fromJSON", scope = FunctionScope.SIMPLE, nulls = NullHandling.NULL_IF_NULL, isRandom = true)
    +  public static class ConvertFromJsonVarcharNonNumerics implements DrillSimpleFunc{
    +
    +    @Param VarCharHolder in;
    +    @Param BitHolder enableNonNumeric;
    +    @Inject DrillBuf buffer;
    +    @Workspace org.apache.drill.exec.vector.complex.fn.JsonReader jsonReader;
    +
    +    @Output ComplexWriter writer;
    +
    +    public void setup(){
    +      jsonReader = new org.apache.drill.exec.vector.complex.fn.JsonReader(buffer, false, false, false,/* do not read numbers as doubles */
    --- End diff --
    
    See above.


---

[GitHub] drill pull request #1026: DRILL-5919: Add non-numeric support for JSON proce...

Posted by paul-rogers <gi...@git.apache.org>.
Github user paul-rogers commented on a diff in the pull request:

    https://github.com/apache/drill/pull/1026#discussion_r150689236
  
    --- Diff: exec/java-exec/src/main/resources/drill-module.conf ---
    @@ -502,6 +502,8 @@ drill.exec.options: {
         store.format: "parquet",
         store.hive.optimize_scan_with_native_readers: false,
         store.json.all_text_mode: false,
    +    store.json.writer.non_numeric_numbers: false,
    +    store.json.reader.non_numeric_numbers: false,
    --- End diff --
    
    OK. So, if the rest of Drill either does not support (or we don't know if it supports) NaN and Inf, should we introduce the change here that potentially leads to failures elsewhere?
    
    Do we know if JDBC and ODBC support these values? (I suppose they do as they are features of Java's float/double primitives...)


---

[GitHub] drill pull request #1026: DRILL-5919: Add non-numeric support for JSON proce...

Posted by vladimirtkach <gi...@git.apache.org>.
Github user vladimirtkach commented on a diff in the pull request:

    https://github.com/apache/drill/pull/1026#discussion_r150500204
  
    --- Diff: exec/java-exec/src/main/resources/drill-module.conf ---
    @@ -502,6 +502,8 @@ drill.exec.options: {
         store.format: "parquet",
         store.hive.optimize_scan_with_native_readers: false,
         store.json.all_text_mode: false,
    +    store.json.writer.non_numeric_numbers: false,
    +    store.json.reader.non_numeric_numbers: false,
    --- End diff --
    
    thanks, allow_nan_inf definitely better.


---

[GitHub] drill issue #1026: DRILL-5919: Add non-numeric support for JSON processing

Posted by vladimirtkach <gi...@git.apache.org>.
Github user vladimirtkach commented on the issue:

    https://github.com/apache/drill/pull/1026
  
    what do you think about having two functions instead: convertFromJSON and convertFromJSON+some suffix. Second will be able to convert  NaN, Infinity


---

[GitHub] drill pull request #1026: DRILL-5919: Add non-numeric support for JSON proce...

Posted by paul-rogers <gi...@git.apache.org>.
Github user paul-rogers commented on a diff in the pull request:

    https://github.com/apache/drill/pull/1026#discussion_r150359452
  
    --- Diff: exec/java-exec/src/main/java/org/apache/drill/exec/expr/fn/impl/conv/JsonConvertFrom.java ---
    @@ -76,7 +77,7 @@ public void eval(){
         @Output ComplexWriter writer;
     
         public void setup(){
    -      jsonReader = new org.apache.drill.exec.vector.complex.fn.JsonReader(buffer, false, false, false /* do not read numbers as doubles */);
    +      jsonReader = new org.apache.drill.exec.vector.complex.fn.JsonReader(buffer, false, false, false, false /* do not read numbers as doubles */);
    --- End diff --
    
    See above. Why do we have two copies? Can we have a function that returns the reader using default configs?


---

[GitHub] drill pull request #1026: DRILL-5919: Add non-numeric support for JSON proce...

Posted by paul-rogers <gi...@git.apache.org>.
Github user paul-rogers commented on a diff in the pull request:

    https://github.com/apache/drill/pull/1026#discussion_r150359625
  
    --- Diff: exec/java-exec/src/main/java/org/apache/drill/exec/expr/fn/impl/conv/JsonConvertTo.java ---
    @@ -90,7 +91,71 @@ public void eval(){
     
           java.io.ByteArrayOutputStream stream = new java.io.ByteArrayOutputStream();
           try {
    -        org.apache.drill.exec.vector.complex.fn.JsonWriter jsonWriter = new org.apache.drill.exec.vector.complex.fn.JsonWriter(stream, true, true);
    +        org.apache.drill.exec.vector.complex.fn.JsonWriter jsonWriter = new org.apache.drill.exec.vector.complex.fn.JsonWriter(stream, true, true, false);
    +
    +        jsonWriter.write(input);
    +      } catch (Exception e) {
    +        throw new RuntimeException(e);
    +      }
    +
    +      byte [] bytea = stream.toByteArray();
    +
    +      out.buffer = buffer = buffer.reallocIfNeeded(bytea.length);
    +      out.buffer.setBytes(0, bytea);
    +      out.end = bytea.length;
    +    }
    +  }
    +
    +  @FunctionTemplate(names = { "convert_toJSON", "convert_toSIMPLEJSON" } , scope = FunctionScope.SIMPLE, nulls = NullHandling.NULL_IF_NULL)
    +  public static class ConvertToJsonNonNumeric implements DrillSimpleFunc{
    +
    +    @Param FieldReader input;
    +    @Param BitHolder nonNumeric;
    +    @Output VarBinaryHolder out;
    +    @Inject DrillBuf buffer;
    +
    +    public void setup(){
    +    }
    +
    +    public void eval(){
    +      out.start = 0;
    +
    +      java.io.ByteArrayOutputStream stream = new java.io.ByteArrayOutputStream();
    +      try {
    +        org.apache.drill.exec.vector.complex.fn.JsonWriter jsonWriter = new org.apache.drill.exec.vector.complex.fn.JsonWriter(stream, true, false,
    --- End diff --
    
    More copies.


---

[GitHub] drill issue #1026: DRILL-5919: Add non-numeric support for JSON processing

Posted by paul-rogers <gi...@git.apache.org>.
Github user paul-rogers commented on the issue:

    https://github.com/apache/drill/pull/1026
  
    On the two functions... Maybe just have one function that handles the Nan/Infinity case. As noted earlier, no matter what we do, JSON without these symbols will work. So, we need only consider JSON with the symbols. Either:
    
    * The NaN/Infinity cases always work, or
    * The NaN/Infinity cases work sometimes, fail others, depending on some option or argument.
    
    I would vote for the first case: it is simpler. I just can't see how anyone would use Drill to valid JSON and would want a query to fail if it contains NaN or Infinity. Can you suggest a case where failing the query would be of help to the user?


---

[GitHub] drill pull request #1026: DRILL-5919: Add non-numeric support for JSON proce...

Posted by paul-rogers <gi...@git.apache.org>.
Github user paul-rogers commented on a diff in the pull request:

    https://github.com/apache/drill/pull/1026#discussion_r150359353
  
    --- Diff: exec/java-exec/src/main/java/org/apache/drill/exec/expr/fn/impl/conv/JsonConvertFrom.java ---
    @@ -50,7 +51,7 @@ private JsonConvertFrom(){}
         @Output ComplexWriter writer;
     
         public void setup(){
    -      jsonReader = new org.apache.drill.exec.vector.complex.fn.JsonReader(buffer, false, false, false /* do not read numbers as doubles */);
    +      jsonReader = new org.apache.drill.exec.vector.complex.fn.JsonReader(buffer, false, false, false, false /* do not read numbers as doubles */);
    --- End diff --
    
    Here, the comment refers to the second-to-last item. Consider this:
    
    ```
    false, // What is the first one?
    false, // What is the second one?
    false, // do not read numbers as doubles
    false // Do not allow Nan, INF
    ```


---

[GitHub] drill issue #1026: DRILL-5919: Add non-numeric support for JSON processing

Posted by vladimirtkach <gi...@git.apache.org>.
Github user vladimirtkach commented on the issue:

    https://github.com/apache/drill/pull/1026
  
    @paul-rogers Made changes, please review. 


---

[GitHub] drill pull request #1026: DRILL-5919: Add non-numeric support for JSON proce...

Posted by paul-rogers <gi...@git.apache.org>.
Github user paul-rogers commented on a diff in the pull request:

    https://github.com/apache/drill/pull/1026#discussion_r150361681
  
    --- Diff: exec/java-exec/src/test/java/org/apache/drill/exec/vector/complex/writer/TestJsonNonNumerics.java ---
    @@ -0,0 +1,167 @@
    +/*
    +* 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.vector.complex.writer;
    +
    +import com.google.common.collect.ImmutableMap;
    +import org.apache.commons.io.FileUtils;
    +import org.apache.drill.BaseTestQuery;
    +import org.apache.drill.common.exceptions.UserRemoteException;
    +import org.apache.drill.common.expression.SchemaPath;
    +import org.apache.drill.exec.record.RecordBatchLoader;
    +import org.apache.drill.exec.record.VectorWrapper;
    +import org.apache.drill.exec.rpc.user.QueryDataBatch;
    +import org.apache.drill.exec.vector.VarCharVector;
    +import org.junit.Test;
    +
    +import java.io.File;
    +import java.util.List;
    +
    +import static org.hamcrest.CoreMatchers.containsString;
    +import static org.junit.Assert.*;
    +
    +public class TestJsonNonNumerics extends BaseTestQuery {
    +
    +  @Test
    +  public void testNonNumericSelect() throws Exception {
    +    File file = new File(getTempDir("nan_test"), "nan_test.json");
    +    String json = "{\"nan\":NaN, \"inf\":Infinity}";
    +    String query = String.format("select * from dfs.`%s`",file.getAbsolutePath());
    +    try {
    +      FileUtils.writeStringToFile(file, json);
    +      test("alter session set `store.json.reader.non_numeric_numbers` = true");
    +      testBuilder()
    +        .sqlQuery(query)
    +        .unOrdered()
    +        .baselineColumns("nan", "inf")
    +        .baselineValues(Double.NaN, Double.POSITIVE_INFINITY)
    +        .build()
    +        .run();
    +    } finally {
    +      test("alter session reset `store.json.reader.non_numeric_numbers`");
    +      FileUtils.deleteQuietly(file);
    +    }
    +  }
    +
    +  @Test(expected = UserRemoteException.class)
    +  public void testNonNumericFailure() throws Exception {
    +    File file = new File(getTempDir("nan_test"), "nan_test.json");
    +    test("alter session set `store.json.reader.non_numeric_numbers` = false");
    +    String json = "{\"nan\":NaN, \"inf\":Infinity}";
    +    try {
    +      FileUtils.writeStringToFile(file, json);
    +      test("select * from dfs.`%s`;", file.getAbsolutePath());
    +    } catch (UserRemoteException e) {
    +      assertThat(e.getMessage(), containsString("Error parsing JSON"));
    +      throw e;
    +    } finally {
    +      test("alter session reset `store.json.reader.non_numeric_numbers`");
    +      FileUtils.deleteQuietly(file);
    +    }
    +  }
    +
    +  @Test
    +  public void testCreateTableNonNumerics() throws Exception {
    +    File file = new File(getTempDir("nan_test"), "nan_test.json");
    +    String json = "{\"nan\":NaN, \"inf\":Infinity}";
    +    String tableName = "ctas_test";
    +    try {
    +      FileUtils.writeStringToFile(file, json);
    +      test("alter session set `store.json.reader.non_numeric_numbers` = true");
    +      test("alter session set `store.json.writer.non_numeric_numbers` = true");
    +      test("alter session set `store.format`='json'");
    +      test("create table dfs_test.tmp.`%s` as select * from dfs.`%s`;", tableName, file.getAbsolutePath());
    +
    +      // ensuring that `NaN` and `Infinity` tokens ARE NOT enclosed with double quotes
    +      File resultFile = new File(new File(getDfsTestTmpSchemaLocation(),tableName),"0_0_0.json");
    +      String resultJson = FileUtils.readFileToString(resultFile);
    +      int nanIndex = resultJson.indexOf("NaN");
    +      assertFalse("`NaN` must not be enclosed with \"\" ", resultJson.charAt(nanIndex - 1) == '"');
    +      assertFalse("`NaN` must not be enclosed with \"\" ", resultJson.charAt(nanIndex + "NaN".length()) == '"');
    +      int infIndex = resultJson.indexOf("Infinity");
    +      assertFalse("`Infinity` must not be enclosed with \"\" ", resultJson.charAt(infIndex - 1) == '"');
    +      assertFalse("`Infinity` must not be enclosed with \"\" ", resultJson.charAt(infIndex + "Infinity".length()) == '"');
    +    } finally {
    +      test("drop table if exists dfs_test.tmp.`%s`", tableName);
    +      test("alter session reset `store.json.reader.non_numeric_numbers`");
    --- End diff --
    
    Consider referencing the constant for the symbol, rather than the option name itself:
    
    ```
    resetSystem(ExecConstants.YOUR_OPTION_NAME);
    ```


---