You are viewing a plain text version of this content. The canonical link for it is here.
Posted to issues@drill.apache.org by "ASF GitHub Bot (Jira)" <ji...@apache.org> on 2020/01/02 19:47:00 UTC

[jira] [Commented] (DRILL-7502) Incorrect/invalid codegen for typeof() with UNION

    [ https://issues.apache.org/jira/browse/DRILL-7502?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=17007058#comment-17007058 ] 

ASF GitHub Bot commented on DRILL-7502:
---------------------------------------

vvysotskyi commented on pull request #1945: DRILL-7502: Invalid codegen for typeof() with UNION
URL: https://github.com/apache/drill/pull/1945#discussion_r362594843
 
 

 ##########
 File path: exec/java-exec/src/main/java/org/apache/drill/exec/expr/ClassGenerator.java
 ##########
 @@ -97,31 +98,37 @@
   private JVar innerClassField;
 
   /**
-   * Assumed that field has 3 indexes within the constant pull: index of the CONSTANT_Fieldref_info +
-   * CONSTANT_Fieldref_info.name_and_type_index + CONSTANT_NameAndType_info.name_index.
-   * CONSTANT_NameAndType_info.descriptor_index has limited range of values, CONSTANT_Fieldref_info.class_index is
-   * the same for a single class, they will be taken into account later.
+   * Assumed that field has 3 indexes within the constant pull: index of the
+   * CONSTANT_Fieldref_info + CONSTANT_Fieldref_info.name_and_type_index +
+   * CONSTANT_NameAndType_info.name_index.
+   * CONSTANT_NameAndType_info.descriptor_index has limited range of values,
+   * CONSTANT_Fieldref_info.class_index is the same for a single class, they
+   * will be taken into account later.
    * <p>
    * Local variable has 1 index within the constant pool.
    * {@link org.objectweb.asm.MethodWriter#visitLocalVariable(String, String, String, Label, Label, int)}
    * <p>
-   * For upper estimation of max index value, suppose that each field and local variable uses different literal
-   * values that have two indexes, then the number of occupied indexes within the constant pull is
-   * fieldCount * 3 + fieldCount * 2 + (index - fieldCount) * 3 => fieldCount * 2 + index * 3
+   * For upper estimation of max index value, suppose that each field and local
+   * variable uses different literal values that have two indexes, then the
+   * number of occupied indexes within the constant pull is fieldCount * 3 +
+   * fieldCount * 2 + (index - fieldCount) * 3 => fieldCount * 2 + index * 3
    * <p>
-   * Assumed that method has 3 indexes within the constant pull: index of the CONSTANT_Methodref_info +
-   * CONSTANT_Methodref_info.name_and_type_index + CONSTANT_NameAndType_info.name_index.
+   * Assumed that method has 3 indexes within the constant pull: index of the
+   * CONSTANT_Methodref_info + CONSTANT_Methodref_info.name_and_type_index +
+   * CONSTANT_NameAndType_info.name_index.
    * <p>
-   * For the upper estimation of number of split methods suppose that each expression in the method uses single variable.
-   * Suppose that the max number of indexes within the constant pull occupied by fields and local variables is M,
-   * the number of split methods is N, number of abstract methods in the template is A, then splitted methods count is
-   * N = (M - A * N * 3) / 50 => N = M / (50 + A * 3)
+   * For the upper estimation of number of split methods suppose that each
+   * expression in the method uses single variable. Suppose that the max number
+   * of indexes within the constant pull occupied by fields and local variables
+   * is M, the number of split methods is N, number of abstract methods in the
+   * template is A, then splitted methods count is N = (M - A * N * 3) / 50 => N
 
 Review comment:
   Please leave formulas to start from the new line.
 
----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


> Incorrect/invalid codegen for typeof() with UNION
> -------------------------------------------------
>
>                 Key: DRILL-7502
>                 URL: https://issues.apache.org/jira/browse/DRILL-7502
>             Project: Apache Drill
>          Issue Type: Bug
>    Affects Versions: 1.17.0
>            Reporter: Paul Rogers
>            Assignee: Paul Rogers
>            Priority: Major
>             Fix For: 1.18.0
>
>
> The {{typeof()}} function is defined as follows:
> {code:java}
>   @FunctionTemplate(names = {"typeOf"},
>           scope = FunctionTemplate.FunctionScope.SIMPLE,
>           nulls = NullHandling.INTERNAL)
>   public static class GetType implements DrillSimpleFunc {
>     @Param
>     FieldReader input;
>     @Output
>     VarCharHolder out;
>     @Inject
>     DrillBuf buf;
>     @Override
>     public void setup() {}
>     @Override
>     public void eval() {
>       String typeName = input.getTypeString();
>       byte[] type = typeName.getBytes();
>       buf = buf.reallocIfNeeded(type.length);
>       buf.setBytes(0, type);
>       out.buffer = buf;
>       out.start = 0;
>       out.end = type.length;
>     }
>   }
> {code}
> Note that the {{input}} field is defined as {{FieldReader}} which has a method called {{getTypeString()}}. As a result, the code works fine in all existing tests in {{TestTypeFns}}.
> I tried to add a function to use {{typeof()}} on a column of type {{UNION}}. When I did, the query failed with a compile error in generated code:
> {noformat}
> SYSTEM ERROR: CompileException: Line 42, Column 43: 
>   A method named "getTypeString" is not declared in any enclosing class nor any supertype, nor through a static import
> {noformat}
> The stack trace shows the generated code; Note that the type of {{input}} changes from a reader to a holder, causing code to be invalid:
> {code:java}
> public class ProjectorGen0 {
>     DrillBuf work0;
>     UnionVector vv1;
>     VarCharVector vv6;
>     DrillBuf work9;
>     VarCharVector vv11;
>     DrillBuf work14;
>     VarCharVector vv16;
>     public void doEval(int inIndex, int outIndex)
>         throws SchemaChangeException
>     {
>         {
>             UnionHolder out4 = new UnionHolder();
>             {
>                 out4 .isSet = vv1 .getAccessor().isSet((inIndex));
>                 if (out4 .isSet == 1) {
>                     vv1 .getAccessor().get((inIndex), out4);
>                 }
>             }
>             //---- start of eval portion of typeOf function. ----//
>             VarCharHolder out5 = new VarCharHolder();
>             {
>                 final VarCharHolder out = new VarCharHolder();
>                 UnionHolder input = out4;
>                 DrillBuf buf = work0;
>                 UnionFunctions$GetType_eval:
> {
>     String typeName = input.getTypeString();
>     byte[] type = typeName.getBytes();
>     buf = buf.reallocIfNeeded(type.length);
>     buf.setBytes(0, type);
>     out.buffer = buf;
>     out.start = 0;
>     out.end = type.length;
> }
> {code}
> By contrast, here is the generated code for one of the existing {{TestTypeFns}} tests where things work:
> {code:java}
> public class ProjectorGen0
>     extends ProjectorTemplate
> {
>     DrillBuf work0;
>     NullableBigIntVector vv1;
>     VarCharVector vv7;
>     public ProjectorGen0() {
>         try {
>             __DRILL_INIT__();
>         } catch (SchemaChangeException e) {
>             throw new UnsupportedOperationException(e);
>         }
>     }
>     public void doEval(int inIndex, int outIndex)
>         throws SchemaChangeException
>     {
>         {
>            ..
>             //---- start of eval portion of typeOf function. ----//
>             VarCharHolder out6 = new VarCharHolder();
>             {
>                 final VarCharHolder out = new VarCharHolder();
>                 FieldReader input = new NullableIntHolderReaderImpl(out5);
>                 DrillBuf buf = work0;
>                 UnionFunctions$GetType_eval:
> {
>     String typeName = input.getTypeString();
>     byte[] type = typeName.getBytes();
>     buf = buf.reallocIfNeeded(type.length);
>     buf.setBytes(0, type);
>     out.buffer = buf;
>     out.start = 0;
>     out.end = type.length;
> }
>                 work0 = buf;
>                 out6 .start = out.start;
>                 out6 .end = out.end;
>                 out6 .buffer = out.buffer;
>             }
>             //---- end of eval portion of typeOf function. ----//
> {code}
> Notice that the {{input}} variable is of type {{FieldReader}} as expected.
> Queries that work:
> {code:java}
>     String sql = "SELECT typeof(CAST(a AS " + castType + ")) FROM (VALUES (1)) AS T(a)";
>     sql = "SELECT typeof(CAST(a AS " + castType + ")) FROM cp.`functions/null.json`";
>     String sql = "SELECT typeof(" + expr + ") FROM (VALUES (" + value + ")) AS T(a)";
> {code}
> Query that fails:
> {code:java}
>     String sql ="SELECT typeof(a) AS t, modeof(a) as m, drilltypeof(a) AS dt\n" +
>                 "FROM cp.`jsoninput/union/c.json`";
> {code}
> The queries that work all include either a CAST or constant values. The query that fails works with data read from a file. Also, the queries that work use scalar types, the query that fails uses the UNION type.



--
This message was sent by Atlassian Jira
(v8.3.4#803005)