You are viewing a plain text version of this content. The canonical link for it is here.
Posted to dev@drill.apache.org by jinfengni <gi...@git.apache.org> on 2017/08/11 21:23:04 UTC

[GitHub] drill pull request #906: DRILL-5546: Handle schema change exception failure ...

GitHub user jinfengni opened a pull request:

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

    DRILL-5546: Handle schema change exception failure caused by empty in…

    …put or empty batche.
    
    1. Modify ScanBatch's logic when it iterates list of RecordReader.
       1) Skip RecordReader if it returns 0 row && present same schema. A new schema (by calling Mutator.isNewSchema() ) means either a new top level field is added, or a field in a nested field is added, or an existing field type is changed.
       2) Implicit columns are added and populated only when the input is not empty, i.e. the batch contains > 0 row or rowCount == 0 && new schema.
       3) ScanBatch will return NONE directly (called as "fast NONE"), if all its RecordReaders haver empty input and thus are skipped, in stead of returing OK_NEW_SCHEMA first.
    
    2. Modify IteratorValidatorBatchIterator to allow
       1) fast NONE ( before seeing a OK_NEW_SCHEMA)
       2) batch with empty list of columns.
    
    2. Modify JsonRecordReader when it get 0 row. Do not insert a nullable-int column for 0 row input. Together with ScanBatch, Drill will skip empty json files.
    
    3. Modify binary operators such as join, union to handle fast none for either one side or both sides. Abstract the logic in AbstractBinaryRecordBatch, except for MergeJoin as its implementation is quite different from others.
    
    4. Fix and refactor union all operator.
      1) Correct union operator hanndling 0 input rows. Previously, it will ignore inputs with 0 row and put nullable-int into output schema, which causes various of schema change issue in down-stream operator. The new behavior is to take schema with 0 into account
      in determining the output schema, in the same way with > 0 input rows. By doing that, we ensure Union operator will not behave like a schema-lossy operator.
      2) Add a UnionInputIterator to simplify the logic to iterate the left/right inputs, removing significant chunk of duplicate codes in previous implementation.
      The new union all operator reduces the code size into half, comparing the old one.
    
    5. Introduce UntypedNullVector to handle convertFromJson() function, when the input batch contains 0 row.
      Problem: The function convertFromJSon() is different from other regular functions in that it only knows the output schema after evaluation is performed. When input has 0 row, Drill essentially does not have
      a way to know the output type, and previously will assume Map type. That works under the assumption other operators like Union would ignore batch with 0 row, which is no longer
      the case in the current implementation.
      Solution: Use MinorType.NULL at the output type for convertFromJSON() when input contains 0 row. The new UntypedNullVector is used to represent a column with MinorType.NULL.
    
    6. HBaseGroupScan convert star column into list of row_key and column family. HBaseRecordReader should reject column star since it expectes star has been converted somewhere else.
      In HBase a column family always has map type, and a non-rowkey column always has nullable varbinary type, this ensures that HBaseRecordReader across different HBase regions will have the same top level schema, even if the region is
      empty or prune all the rows due to filter pushdown optimization. In other words, we will not see different top level schema from different HBaseRecordReader for the same table.
      However, such change will not be able to handle hard schema change : c1 exists in cf1 in one region, but not in another region. Further work is required to handle hard schema change.
    
    7. Modify scan cost estimation when the query involves * column. This is to remove the planning randomness since previously two different operators could have same cost.
    
    8. Add a new flag 'outputProj' to Project operator, to indicate if Project is for the query's final output. Such Project is added by TopProjectVisitor, to handle fast NONE when all the inputs to the query are empty
    and are skipped.
      1) column star is replaced with empty list
      2) regular column reference is replaced with nullable-int column
      3) An expression will go through ExpressionTreeMaterializer, and use the type of materialized expression as the output type
      4) Return an OK_NEW_SCHEMA with the schema using the above logic, then return a NONE to down-stream operator.
    
    9. Add unit test to test operators handling empty input.
    
    10. Add unit test to test query when inputs are all empty.

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

    $ git pull https://github.com/jinfengni/incubator-drill DRILL-5546

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

    https://github.com/apache/drill/pull/906.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 #906
    
----
commit b0110140f8375809af3deddf9881e64dc1242886
Author: Jinfeng Ni <jn...@apache.org>
Date:   2017-05-17T23:08:00Z

    DRILL-5546: Handle schema change exception failure caused by empty input or empty batche.
    
    1. Modify ScanBatch's logic when it iterates list of RecordReader.
       1) Skip RecordReader if it returns 0 row && present same schema. A new schema (by calling Mutator.isNewSchema() ) means either a new top level field is added, or a field in a nested field is added, or an existing field type is changed.
       2) Implicit columns are added and populated only when the input is not empty, i.e. the batch contains > 0 row or rowCount == 0 && new schema.
       3) ScanBatch will return NONE directly (called as "fast NONE"), if all its RecordReaders haver empty input and thus are skipped, in stead of returing OK_NEW_SCHEMA first.
    
    2. Modify IteratorValidatorBatchIterator to allow
       1) fast NONE ( before seeing a OK_NEW_SCHEMA)
       2) batch with empty list of columns.
    
    2. Modify JsonRecordReader when it get 0 row. Do not insert a nullable-int column for 0 row input. Together with ScanBatch, Drill will skip empty json files.
    
    3. Modify binary operators such as join, union to handle fast none for either one side or both sides. Abstract the logic in AbstractBinaryRecordBatch, except for MergeJoin as its implementation is quite different from others.
    
    4. Fix and refactor union all operator.
      1) Correct union operator hanndling 0 input rows. Previously, it will ignore inputs with 0 row and put nullable-int into output schema, which causes various of schema change issue in down-stream operator. The new behavior is to take schema with 0 into account
      in determining the output schema, in the same way with > 0 input rows. By doing that, we ensure Union operator will not behave like a schema-lossy operator.
      2) Add a UnionInputIterator to simplify the logic to iterate the left/right inputs, removing significant chunk of duplicate codes in previous implementation.
      The new union all operator reduces the code size into half, comparing the old one.
    
    5. Introduce UntypedNullVector to handle convertFromJson() function, when the input batch contains 0 row.
      Problem: The function convertFromJSon() is different from other regular functions in that it only knows the output schema after evaluation is performed. When input has 0 row, Drill essentially does not have
      a way to know the output type, and previously will assume Map type. That works under the assumption other operators like Union would ignore batch with 0 row, which is no longer
      the case in the current implementation.
      Solution: Use MinorType.NULL at the output type for convertFromJSON() when input contains 0 row. The new UntypedNullVector is used to represent a column with MinorType.NULL.
    
    6. HBaseGroupScan convert star column into list of row_key and column family. HBaseRecordReader should reject column star since it expectes star has been converted somewhere else.
      In HBase a column family always has map type, and a non-rowkey column always has nullable varbinary type, this ensures that HBaseRecordReader across different HBase regions will have the same top level schema, even if the region is
      empty or prune all the rows due to filter pushdown optimization. In other words, we will not see different top level schema from different HBaseRecordReader for the same table.
      However, such change will not be able to handle hard schema change : c1 exists in cf1 in one region, but not in another region. Further work is required to handle hard schema change.
    
    7. Modify scan cost estimation when the query involves * column. This is to remove the planning randomness since previously two different operators could have same cost.
    
    8. Add a new flag 'outputProj' to Project operator, to indicate if Project is for the query's final output. Such Project is added by TopProjectVisitor, to handle fast NONE when all the inputs to the query are empty
    and are skipped.
      1) column star is replaced with empty list
      2) regular column reference is replaced with nullable-int column
      3) An expression will go through ExpressionTreeMaterializer, and use the type of materialized expression as the output type
      4) Return an OK_NEW_SCHEMA with the schema using the above logic, then return a NONE to down-stream operator.
    
    9. Add unit test to test operators handling empty input.
    
    10. Add unit test to test query when inputs are all empty.

----


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] drill pull request #906: DRILL-5546: Handle schema change exception failure ...

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

    https://github.com/apache/drill/pull/906#discussion_r135974107
  
    --- Diff: exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/project/ProjectRecordBatch.java ---
    @@ -768,4 +765,73 @@ else if (exprHasPrefix && refHasPrefix) {
           }
         }
       }
    +
    +  /**
    +   * handle FAST NONE specially when Project for query output. This happens when input returns a
    +   * FAST NONE directly ( input does not return any batch with schema/data).
    +   *
    +   * Project operator has to return a batch with schema derived using the following 3 rules:
    +   *  Case 1:  *  ==>  expand into an empty list of columns.
    +   *  Case 2:  regular column reference ==> treat as nullable-int column
    +   *  Case 3:  expressions => Call ExpressionTreeMaterialization over an empty vector contain.
    --- End diff --
    
    I managed to find a way to share the existing code path. With the revised patch, both of the first scenario and second one would call same method to setup a new schema based on incoming batch. The only difference for the second scenario is we have to simulate a record batch with empty vector container, as the code for set up new schema requires it.  That is done by leveraging an existing class (with some extension). 



---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] drill pull request #906: DRILL-5546: Handle schema change exception failure ...

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

    https://github.com/apache/drill/pull/906#discussion_r135710183
  
    --- Diff: exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/ScanBatch.java ---
    @@ -329,9 +326,11 @@ public TypedFieldId getValueVectorId(SchemaPath path) {
     
       @VisibleForTesting
       public static class Mutator implements OutputMutator {
    -    /** Whether schema has changed since last inquiry (via #isNewSchema}).  Is
    -     *  true before first inquiry. */
    -    private boolean schemaChanged = true;
    +    /** Flag keeping track whether top-level schema has changed since last inquiry (via #isNewSchema}).
    +     * It's initialized to false, or reset to false after #isNewSchema or after #clear, until a new value vector
    +     * or a value vector with different type is added to fieldVectorMap.
    +     **/
    +    private boolean schemaChanged;
    --- End diff --
    
    No comments with respect to "flag vs counter". But the flag is in the existing code, and the patch I have simply 1) change the initial value of flag, 2) add comments. 


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] drill pull request #906: DRILL-5546: Handle schema change exception failure ...

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/906#discussion_r134627882
  
    --- Diff: exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/project/ProjectRecordBatch.java ---
    @@ -768,4 +765,73 @@ else if (exprHasPrefix && refHasPrefix) {
           }
         }
       }
    +
    +  /**
    +   * handle FAST NONE specially when Project for query output. This happens when input returns a
    +   * FAST NONE directly ( input does not return any batch with schema/data).
    +   *
    +   * Project operator has to return a batch with schema derived using the following 3 rules:
    +   *  Case 1:  *  ==>  expand into an empty list of columns.
    +   *  Case 2:  regular column reference ==> treat as nullable-int column
    +   *  Case 3:  expressions => Call ExpressionTreeMaterialization over an empty vector contain.
    +   *           Once the expression is materialized without error, use the output type of materialized
    +   *           expression.
    +   * The batch is constructed with the above rules, and recordCount = 0.
    +   * Returned with OK_NEW_SCHEMA to down-stream operator.
    --- End diff --
    
    Here, the only downstream operator is Screen, right?


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] drill pull request #906: DRILL-5546: Handle schema change exception failure ...

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

    https://github.com/apache/drill/pull/906#discussion_r136186438
  
    --- Diff: exec/vector/src/main/java/org/apache/drill/exec/vector/UntypedNullHolder.java ---
    @@ -0,0 +1,50 @@
    +/*
    + * 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;
    +
    +import org.apache.drill.common.types.TypeProtos;
    +import org.apache.drill.common.types.Types;
    +import org.apache.drill.exec.expr.holders.ValueHolder;
    +
    +public class UntypedNullHolder implements ValueHolder {
    +  public static final TypeProtos.MajorType TYPE = Types.optional(TypeProtos.MinorType.NULL);
    +
    +  public TypeProtos.MajorType getType() {return TYPE;}
    --- End diff --
    
    Done.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] drill pull request #906: DRILL-5546: Handle schema change exception failure ...

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/906#discussion_r135574741
  
    --- Diff: exec/java-exec/src/main/java/org/apache/drill/exec/planner/logical/DrillScanRel.java ---
    @@ -160,12 +161,7 @@ public RelOptCost computeSelfCost(final RelOptPlanner planner, RelMetadataQuery
         final ScanStats stats = groupScan.getScanStats(settings);
         int columnCount = getRowType().getFieldCount();
         double ioCost = 0;
    -    boolean isStarQuery = Iterables.tryFind(getRowType().getFieldNames(), new Predicate<String>() {
    -      @Override
    -      public boolean apply(String input) {
    -        return Preconditions.checkNotNull(input).equals("*");
    -      }
    -    }).isPresent();
    +    boolean isStarQuery = AbstractRecordReader.isStarQuery(columns);
    --- End diff --
    
    Should this planning-time class depend on a specific execution-time class? Or, would it be better to move the method from the execution class onto something common between plan and execution?


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] drill pull request #906: DRILL-5546: Handle schema change exception failure ...

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

    https://github.com/apache/drill/pull/906#discussion_r136159563
  
    --- Diff: exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/union/UnionAllRecordBatch.java ---
    @@ -130,562 +145,248 @@ public IterOutcome innerNext() {
       }
     
       @Override
    -  public WritableBatch getWritableBatch() {
    -    return WritableBatch.get(this);
    +  public int getRecordCount() {
    +    return recordCount;
       }
     
    -  private void setValueCount(int count) {
    -    for (ValueVector v : allocationVectors) {
    -      ValueVector.Mutator m = v.getMutator();
    -      m.setValueCount(count);
    -    }
    -  }
     
    -  private boolean doAlloc() {
    -    for (ValueVector v : allocationVectors) {
    -      try {
    -        AllocationHelper.allocateNew(v, current.getRecordCount());
    -      } catch (OutOfMemoryException ex) {
    -        return false;
    -      }
    +  @SuppressWarnings("resource")
    +  private IterOutcome doWork(RecordBatch inputBatch, boolean newSchema) throws ClassTransformationException, IOException, SchemaChangeException {
    +    if (inputBatch.getSchema().getFieldCount() != container.getSchema().getFieldCount()) {
    +      // wrong.
         }
    -    return true;
    -  }
     
    -  @SuppressWarnings("resource")
    -  private IterOutcome doWork() throws ClassTransformationException, IOException, SchemaChangeException {
    -    if (allocationVectors != null) {
    -      for (ValueVector v : allocationVectors) {
    -        v.clear();
    -      }
    +    if (newSchema) {
    +      createUnionAller(inputBatch);
         }
     
    -    allocationVectors = Lists.newArrayList();
    -    transfers.clear();
    +    container.zeroVectors();
     
    -    // If both sides of Union-All are empty
    -    if(unionAllInput.isBothSideEmpty()) {
    -      for(int i = 0; i < outputFields.size(); ++i) {
    -        final String colName = outputFields.get(i).getPath();
    -        final MajorType majorType = MajorType.newBuilder()
    -            .setMinorType(MinorType.INT)
    -            .setMode(DataMode.OPTIONAL)
    -            .build();
    -
    -        MaterializedField outputField = MaterializedField.create(colName, majorType);
    -        ValueVector vv = container.addOrGet(outputField, callBack);
    -        allocationVectors.add(vv);
    -      }
    +    for (final ValueVector v : this.allocationVectors) {
    +      AllocationHelper.allocateNew(v, inputBatch.getRecordCount());
    +    }
     
    -      container.buildSchema(BatchSchema.SelectionVectorMode.NONE);
    +    recordCount = unionall.unionRecords(0, inputBatch.getRecordCount(), 0);
    +    for (final ValueVector v : allocationVectors) {
    +      final ValueVector.Mutator m = v.getMutator();
    +      m.setValueCount(recordCount);
    +    }
    +
    +    if (callBack.getSchemaChangedAndReset()) {
           return IterOutcome.OK_NEW_SCHEMA;
    +    } else {
    +      return IterOutcome.OK;
         }
    +  }
    +
    +  private void createUnionAller(RecordBatch inputBatch) throws ClassTransformationException, IOException, SchemaChangeException {
    +    transfers.clear();
    +    allocationVectors.clear();;
     
         final ClassGenerator<UnionAller> cg = CodeGenerator.getRoot(UnionAller.TEMPLATE_DEFINITION, context.getFunctionRegistry(), context.getOptions());
         cg.getCodeGenerator().plainJavaCapable(true);
         // Uncomment out this line to debug the generated code.
    -//    cg.getCodeGenerator().saveCodeForDebugging(true);
    +    //    cg.getCodeGenerator().saveCodeForDebugging(true);
    +
         int index = 0;
    -    for(VectorWrapper<?> vw : current) {
    -       ValueVector vvIn = vw.getValueVector();
    -      // get the original input column names
    -      SchemaPath inputPath = SchemaPath.getSimplePath(vvIn.getField().getPath());
    -      // get the renamed column names
    -      SchemaPath outputPath = SchemaPath.getSimplePath(outputFields.get(index).getPath());
    +    for(VectorWrapper<?> vw : inputBatch) {
    +      ValueVector vvIn = vw.getValueVector();
    +      ValueVector vvOut = container.getValueVector(index).getValueVector();
     
           final ErrorCollector collector = new ErrorCollectorImpl();
           // According to input data names, Minortypes, Datamodes, choose to
           // transfer directly,
           // rename columns or
           // cast data types (Minortype or DataMode)
    -      if (hasSameTypeAndMode(outputFields.get(index), vw.getValueVector().getField())) {
    +      if (hasSameTypeAndMode(container.getSchema().getColumn(index), vvIn.getField())
    +          && vvIn.getField().getType().getMinorType() != TypeProtos.MinorType.MAP // Per DRILL-5521, existing bug for map transfer
    +          ) {
             // Transfer column
    -
    -        MajorType outputFieldType = outputFields.get(index).getType();
    -        MaterializedField outputField = MaterializedField.create(outputPath.getAsUnescapedPath(), outputFieldType);
    -
    -        /*
    -          todo: Fix if condition when DRILL-4824 is merged
    -          If condition should be changed to:
    -          `if (outputFields.get(index).getPath().equals(inputPath.getAsUnescapedPath())) {`
    -          DRILL-5419 has changed condition to correct one but this caused regression (DRILL-5521).
    -          Root cause is missing indication of child column in map types when it is null.
    -          DRILL-4824 is re-working json reader implementation, including map types and will fix this problem.
    -          Reverting condition to previous one to avoid regression till DRILL-4824 is merged.
    -          Unit test - TestJsonReader.testKvgenWithUnionAll().
    -         */
    -        if (outputFields.get(index).getPath().equals(inputPath)) {
    -          ValueVector vvOut = container.addOrGet(outputField);
    -          TransferPair tp = vvIn.makeTransferPair(vvOut);
    -          transfers.add(tp);
    +        TransferPair tp = vvIn.makeTransferPair(vvOut);
    +        transfers.add(tp);
             // Copy data in order to rename the column
    -        } else {
    -          final LogicalExpression expr = ExpressionTreeMaterializer.materialize(inputPath, current, collector, context.getFunctionRegistry() );
    -          if (collector.hasErrors()) {
    -            throw new SchemaChangeException(String.format("Failure while trying to materialize incoming schema.  Errors:\n %s.", collector.toErrorString()));
    -          }
    -
    -          ValueVector vv = container.addOrGet(outputField, callBack);
    -          allocationVectors.add(vv);
    -          TypedFieldId fid = container.getValueVectorId(SchemaPath.getSimplePath(outputField.getPath()));
    -          ValueVectorWriteExpression write = new ValueVectorWriteExpression(fid, expr, true);
    -          cg.addExpr(write);
    -        }
    -      // Cast is necessary
    +      } else if (vvIn.getField().getType().getMinorType() == TypeProtos.MinorType.NULL) {
    +        continue;
           } else {
    -        LogicalExpression expr = ExpressionTreeMaterializer.materialize(inputPath, current, collector, context.getFunctionRegistry());
    +        SchemaPath inputPath = SchemaPath.getSimplePath(vvIn.getField().getPath());
    +        MaterializedField inField = vvIn.getField();
    +        MaterializedField outputField = vvOut.getField();
    +
    +        LogicalExpression expr = ExpressionTreeMaterializer.materialize(inputPath, inputBatch, collector, context.getFunctionRegistry());
    +
             if (collector.hasErrors()) {
               throw new SchemaChangeException(String.format("Failure while trying to materialize incoming schema.  Errors:\n %s.", collector.toErrorString()));
             }
     
             // If the inputs' DataMode is required and the outputs' DataMode is not required
             // cast to the one with the least restriction
    -        if(vvIn.getField().getType().getMode() == DataMode.REQUIRED
    -            && outputFields.get(index).getType().getMode() != DataMode.REQUIRED) {
    -          expr = ExpressionTreeMaterializer.convertToNullableType(expr, vvIn.getField().getType().getMinorType(), context.getFunctionRegistry(), collector);
    +        if(inField.getType().getMode() == TypeProtos.DataMode.REQUIRED
    +            && outputField.getType().getMode() != TypeProtos.DataMode.REQUIRED) {
    +          expr = ExpressionTreeMaterializer.convertToNullableType(expr, inField.getType().getMinorType(), context.getFunctionRegistry(), collector);
               if (collector.hasErrors()) {
                 throw new SchemaChangeException(String.format("Failure while trying to materialize incoming schema.  Errors:\n %s.", collector.toErrorString()));
               }
             }
     
             // If two inputs' MinorTypes are different,
             // Insert a cast before the Union operation
    -        if(vvIn.getField().getType().getMinorType() != outputFields.get(index).getType().getMinorType()) {
    -          expr = ExpressionTreeMaterializer.addCastExpression(expr, outputFields.get(index).getType(), context.getFunctionRegistry(), collector);
    +        if(inField.getType().getMinorType() != outputField.getType().getMinorType()) {
    +          expr = ExpressionTreeMaterializer.addCastExpression(expr, outputField.getType(), context.getFunctionRegistry(), collector);
               if (collector.hasErrors()) {
                 throw new SchemaChangeException(String.format("Failure while trying to materialize incoming schema.  Errors:\n %s.", collector.toErrorString()));
               }
             }
     
    -        final MaterializedField outputField = MaterializedField.create(outputPath.getAsUnescapedPath(), expr.getMajorType());
    -        ValueVector vector = container.addOrGet(outputField, callBack);
    -        allocationVectors.add(vector);
             TypedFieldId fid = container.getValueVectorId(SchemaPath.getSimplePath(outputField.getPath()));
     
    -        boolean useSetSafe = !(vector instanceof FixedWidthVector);
    +        boolean useSetSafe = !(vvOut instanceof FixedWidthVector);
             ValueVectorWriteExpression write = new ValueVectorWriteExpression(fid, expr, useSetSafe);
             cg.addExpr(write);
    +
    +        allocationVectors.add(vvOut);
           }
           ++index;
         }
     
         unionall = context.getImplementationClass(cg.getCodeGenerator());
    -    unionall.setup(context, current, this, transfers);
    -
    -    if(!schemaAvailable) {
    -      container.buildSchema(BatchSchema.SelectionVectorMode.NONE);
    -      schemaAvailable = true;
    -    }
    -
    -    if(!doAlloc()) {
    -      return IterOutcome.OUT_OF_MEMORY;
    -    }
    -
    -    recordCount = unionall.unionRecords(0, current.getRecordCount(), 0);
    -    setValueCount(recordCount);
    -    return IterOutcome.OK;
    -  }
    -
    -  public static boolean hasSameTypeAndMode(MaterializedField leftField, MaterializedField rightField) {
    -    return (leftField.getType().getMinorType() == rightField.getType().getMinorType())
    -        && (leftField.getType().getMode() == rightField.getType().getMode());
    -  }
    -
    -  // This method is used by inner class to point the reference `current` to the correct record batch
    -  private void setCurrentRecordBatch(RecordBatch target) {
    -    this.current = target;
    -  }
    -
    -  // This method is used by inner class to clear the current record batch
    -  private void clearCurrentRecordBatch() {
    -    for(VectorWrapper<?> v: current) {
    -      v.clear();
    -    }
    +    unionall.setup(context, inputBatch, this, transfers);
       }
     
    -  public static class UnionAllInput {
    -    private UnionAllRecordBatch unionAllRecordBatch;
    -    private List<MaterializedField> outputFields;
    -    private OneSideInput leftSide;
    -    private OneSideInput rightSide;
    -    private IterOutcome upstream = IterOutcome.NOT_YET;
    -    private boolean leftIsFinish = false;
    -    private boolean rightIsFinish = false;
    -
    -    // These two schemas are obtained from the first record batches of the left and right inputs
    -    // They are used to check if the schema is changed between recordbatches
    -    private BatchSchema leftSchema;
    -    private BatchSchema rightSchema;
    -    private boolean bothEmpty = false;
    -
    -    public UnionAllInput(UnionAllRecordBatch unionAllRecordBatch, RecordBatch left, RecordBatch right) {
    -      this.unionAllRecordBatch = unionAllRecordBatch;
    -      leftSide = new OneSideInput(left);
    -      rightSide = new OneSideInput(right);
    -    }
    -
    -    private void setBothSideEmpty(boolean bothEmpty) {
    -      this.bothEmpty = bothEmpty;
    -    }
    -
    -    private boolean isBothSideEmpty() {
    -      return bothEmpty;
    -    }
    -
    -    public IterOutcome nextBatch() throws SchemaChangeException {
    -      if(upstream == RecordBatch.IterOutcome.NOT_YET) {
    -        IterOutcome iterLeft = leftSide.nextBatch();
    -        switch(iterLeft) {
    -          case OK_NEW_SCHEMA:
    -            /*
    -             * If the first few record batches are all empty,
    -             * there is no way to tell whether these empty batches are coming from empty files.
    -             * It is incorrect to infer output types when either side could be coming from empty.
    -             *
    -             * Thus, while-loop is necessary to skip those empty batches.
    -             */
    -            whileLoop:
    -            while(leftSide.getRecordBatch().getRecordCount() == 0) {
    -              iterLeft = leftSide.nextBatch();
    -
    -              switch(iterLeft) {
    -                case STOP:
    -                case OUT_OF_MEMORY:
    -                  return iterLeft;
    -
    -                case NONE:
    -                  // Special Case: The left side was an empty input.
    -                  leftIsFinish = true;
    -                  break whileLoop;
    -
    -                case NOT_YET:
    -                case OK_NEW_SCHEMA:
    -                case OK:
    -                  continue whileLoop;
    -
    -                default:
    -                  throw new IllegalStateException(
    -                      String.format("Unexpected state %s.", iterLeft));
    -              }
    -            }
    -
    -            break;
    -          case STOP:
    -          case OUT_OF_MEMORY:
    -            return iterLeft;
    -
    -          default:
    -            throw new IllegalStateException(
    -                String.format("Unexpected state %s.", iterLeft));
    -        }
    -
    -        IterOutcome iterRight = rightSide.nextBatch();
    -        switch(iterRight) {
    -          case OK_NEW_SCHEMA:
    -            // Unless there is no record batch on the left side of the inputs,
    -            // always start processing from the left side.
    -            if(leftIsFinish) {
    -              unionAllRecordBatch.setCurrentRecordBatch(rightSide.getRecordBatch());
    -            } else {
    -              unionAllRecordBatch.setCurrentRecordBatch(leftSide.getRecordBatch());
    -            }
    -            // If the record count of the first batch from right input is zero,
    -            // there are two possibilities:
    -            // 1. The right side is an empty input (e.g., file).
    -            // 2. There will be more records carried by later batches.
    -
    -            /*
    -             * If the first few record batches are all empty,
    -             * there is no way to tell whether these empty batches are coming from empty files.
    -             * It is incorrect to infer output types when either side could be coming from empty.
    -             *
    -             * Thus, while-loop is necessary to skip those empty batches.
    -             */
    -            whileLoop:
    -            while(rightSide.getRecordBatch().getRecordCount() == 0) {
    -              iterRight = rightSide.nextBatch();
    -              switch(iterRight) {
    -                case STOP:
    -                case OUT_OF_MEMORY:
    -                  return iterRight;
    -
    -                case NONE:
    -                  // Special Case: The right side was an empty input.
    -                  rightIsFinish = true;
    -                  break whileLoop;
    -
    -                case NOT_YET:
    -                case OK_NEW_SCHEMA:
    -                case OK:
    -                  continue whileLoop;
    -
    -                default:
    -                  throw new IllegalStateException(
    -                      String.format("Unexpected state %s.", iterRight));
    -              }
    -            }
    -
    -            if(leftIsFinish && rightIsFinish) {
    -              setBothSideEmpty(true);
    -            }
    -
    -            inferOutputFields();
    -            break;
    -
    -          case STOP:
    -          case OUT_OF_MEMORY:
    -            return iterRight;
    -
    -          default:
    -            throw new IllegalStateException(
    -                String.format("Unexpected state %s.", iterRight));
    -        }
    -
    -
    -
    -        upstream = IterOutcome.OK_NEW_SCHEMA;
    -        return upstream;
    -      } else {
    -        if(isBothSideEmpty()) {
    -          return IterOutcome.NONE;
    -        }
    -
    -        unionAllRecordBatch.clearCurrentRecordBatch();
    -
    -        if(leftIsFinish && rightIsFinish) {
    -          upstream = IterOutcome.NONE;
    -          return upstream;
    -        } else if(leftIsFinish) {
    -          IterOutcome iterOutcome = rightSide.nextBatch();
    -
    -          switch(iterOutcome) {
    -            case NONE:
    -              rightIsFinish = true;
    -              // fall through
    -            case STOP:
    -            case OUT_OF_MEMORY:
    -              upstream = iterOutcome;
    -              return upstream;
    -
    -            case OK_NEW_SCHEMA:
    -              if(!rightSide.getRecordBatch().getSchema().equals(rightSchema)) {
    -                throw new SchemaChangeException("Schema change detected in the right input of Union-All. This is not currently supported");
    -              }
    -              iterOutcome = IterOutcome.OK;
    -              // fall through
    -            case OK:
    -              unionAllRecordBatch.setCurrentRecordBatch(rightSide.getRecordBatch());
    -              upstream = iterOutcome;
    -              return upstream;
    -
    -            default:
    -              throw new IllegalStateException(String.format("Unknown state %s.", upstream));
    -          }
    -        } else if(rightIsFinish) {
    -          IterOutcome iterOutcome = leftSide.nextBatch();
    -          switch(iterOutcome) {
    -            case STOP:
    -            case OUT_OF_MEMORY:
    -            case NONE:
    -              upstream = iterOutcome;
    -              return upstream;
    -
    -            case OK:
    -              unionAllRecordBatch.setCurrentRecordBatch(leftSide.getRecordBatch());
    -              upstream = iterOutcome;
    -              return upstream;
    -
    -            default:
    -              throw new IllegalStateException(String.format("Unknown state %s.", iterOutcome));
    -          }
    -        } else {
    -          IterOutcome iterOutcome = leftSide.nextBatch();
    -
    -          switch(iterOutcome) {
    -            case STOP:
    -            case OUT_OF_MEMORY:
    -              upstream = iterOutcome;
    -              return upstream;
    -
    -            case OK_NEW_SCHEMA:
    -              if(!leftSide.getRecordBatch().getSchema().equals(leftSchema)) {
    -                throw new SchemaChangeException("Schema change detected in the left input of Union-All. This is not currently supported");
    -              }
    -
    -              iterOutcome = IterOutcome.OK;
    -              // fall through
    -            case OK:
    -              unionAllRecordBatch.setCurrentRecordBatch(leftSide.getRecordBatch());
    -              upstream = iterOutcome;
    -              return upstream;
    -
    -            case NONE:
    -              unionAllRecordBatch.setCurrentRecordBatch(rightSide.getRecordBatch());
    -              upstream = IterOutcome.OK;
    -              leftIsFinish = true;
    -              return upstream;
    -
    -            default:
    -              throw new IllegalStateException(String.format("Unknown state %s.", upstream));
    -          }
    -        }
    -      }
    -    }
     
    -    /**
    -     *
    -     * Summarize the inference in the four different situations:
    -     * First of all, the field names are always determined by the left side
    -     * (Even when the left side is from an empty file, we have the column names.)
    -     *
    -     * Cases:
    -     * 1. Left: non-empty; Right: non-empty
    -     *      types determined by both sides with implicit casting involved
    -     * 2. Left: empty; Right: non-empty
    -     *      type from the right
    -     * 3. Left: non-empty; Right: empty
    -     *      types from the left
    -     * 4. Left: empty; Right: empty
    -     *      types are nullable integer
    -     */
    -    private void inferOutputFields() {
    -      if(!leftIsFinish && !rightIsFinish) {
    -        // Both sides are non-empty
    -        inferOutputFieldsBothSide();
    -      } else if(!rightIsFinish) {
    -        // Left side is non-empty
    -        // While use left side's column names as output column names,
    -        // use right side's column types as output column types.
    -        inferOutputFieldsFromSingleSide(
    -            leftSide.getRecordBatch().getSchema(),
    -            rightSide.getRecordBatch().getSchema());
    +  // The output table's column names always follow the left table,
    +  // where the output type is chosen based on DRILL's implicit casting rules
    +  private void inferOutputFieldsBothSide(final BatchSchema leftSchema, final BatchSchema rightSchema) {
    +//    outputFields = Lists.newArrayList();
    +    final Iterator<MaterializedField> leftIter = leftSchema.iterator();
    +    final Iterator<MaterializedField> rightIter = rightSchema.iterator();
    +
    +    int index = 1;
    +    while (leftIter.hasNext() && rightIter.hasNext()) {
    +      MaterializedField leftField  = leftIter.next();
    +      MaterializedField rightField = rightIter.next();
    +
    +      if (hasSameTypeAndMode(leftField, rightField)) {
    +        TypeProtos.MajorType.Builder builder = TypeProtos.MajorType.newBuilder().setMinorType(leftField.getType().getMinorType()).setMode(leftField.getDataMode());
    +        builder = Types.calculateTypePrecisionAndScale(leftField.getType(), rightField.getType(), builder);
    +        container.addOrGet(MaterializedField.create(leftField.getPath(), builder.build()), callBack);
    +      } else if (Types.isUntypedNull(rightField.getType())) {
    +        container.addOrGet(leftField, callBack);
    +      } else if (Types.isUntypedNull(leftField.getType())) {
    +        container.addOrGet(MaterializedField.create(leftField.getPath(), rightField.getType()), callBack);
           } else {
    -        // Either right side is empty or both are empty
    -        // Using left side's schema is sufficient
    -        inferOutputFieldsFromSingleSide(
    -            leftSide.getRecordBatch().getSchema(),
    -            leftSide.getRecordBatch().getSchema());
    -      }
    -    }
    -
    -    // The output table's column names always follow the left table,
    -    // where the output type is chosen based on DRILL's implicit casting rules
    -    private void inferOutputFieldsBothSide() {
    -      outputFields = Lists.newArrayList();
    -      leftSchema = leftSide.getRecordBatch().getSchema();
    -      rightSchema = rightSide.getRecordBatch().getSchema();
    -      Iterator<MaterializedField> leftIter = leftSchema.iterator();
    -      Iterator<MaterializedField> rightIter = rightSchema.iterator();
    -
    -      int index = 1;
    -      while (leftIter.hasNext() && rightIter.hasNext()) {
    -        MaterializedField leftField  = leftIter.next();
    -        MaterializedField rightField = rightIter.next();
    -
    -        if (hasSameTypeAndMode(leftField, rightField)) {
    -          MajorType.Builder builder = MajorType.newBuilder().setMinorType(leftField.getType().getMinorType()).setMode(leftField.getDataMode());
    +        // If the output type is not the same,
    +        // cast the column of one of the table to a data type which is the Least Restrictive
    +        TypeProtos.MajorType.Builder builder = TypeProtos.MajorType.newBuilder();
    +        if (leftField.getType().getMinorType() == rightField.getType().getMinorType()) {
    +          builder.setMinorType(leftField.getType().getMinorType());
               builder = Types.calculateTypePrecisionAndScale(leftField.getType(), rightField.getType(), builder);
    -          outputFields.add(MaterializedField.create(leftField.getPath(), builder.build()));
             } else {
    -          // If the output type is not the same,
    -          // cast the column of one of the table to a data type which is the Least Restrictive
    -          MajorType.Builder builder = MajorType.newBuilder();
    -          if (leftField.getType().getMinorType() == rightField.getType().getMinorType()) {
    -            builder.setMinorType(leftField.getType().getMinorType());
    -            builder = Types.calculateTypePrecisionAndScale(leftField.getType(), rightField.getType(), builder);
    -          } else {
    -            List<MinorType> types = Lists.newLinkedList();
    -            types.add(leftField.getType().getMinorType());
    -            types.add(rightField.getType().getMinorType());
    -            MinorType outputMinorType = TypeCastRules.getLeastRestrictiveType(types);
    -            if (outputMinorType == null) {
    -              throw new DrillRuntimeException("Type mismatch between " + leftField.getType().getMinorType().toString() +
    -                  " on the left side and " + rightField.getType().getMinorType().toString() +
    -                  " on the right side in column " + index + " of UNION ALL");
    -            }
    -            builder.setMinorType(outputMinorType);
    +          List<TypeProtos.MinorType> types = Lists.newLinkedList();
    +          types.add(leftField.getType().getMinorType());
    +          types.add(rightField.getType().getMinorType());
    +          TypeProtos.MinorType outputMinorType = TypeCastRules.getLeastRestrictiveType(types);
    +          if (outputMinorType == null) {
    +            throw new DrillRuntimeException("Type mismatch between " + leftField.getType().getMinorType().toString() +
    +                " on the left side and " + rightField.getType().getMinorType().toString() +
    +                " on the right side in column " + index + " of UNION ALL");
               }
    -
    -          // The output data mode should be as flexible as the more flexible one from the two input tables
    -          List<DataMode> dataModes = Lists.newLinkedList();
    -          dataModes.add(leftField.getType().getMode());
    -          dataModes.add(rightField.getType().getMode());
    -          builder.setMode(TypeCastRules.getLeastRestrictiveDataMode(dataModes));
    -
    -          outputFields.add(MaterializedField.create(leftField.getPath(), builder.build()));
    +          builder.setMinorType(outputMinorType);
             }
    -        ++index;
    -      }
    -
    -      assert !leftIter.hasNext() && ! rightIter.hasNext() : "Mis-match of column count should have been detected when validating sqlNode at planning";
    -    }
     
    -    private void inferOutputFieldsFromSingleSide(final BatchSchema schemaForNames, final BatchSchema schemaForTypes) {
    -      outputFields = Lists.newArrayList();
    +        // The output data mode should be as flexible as the more flexible one from the two input tables
    +        List<TypeProtos.DataMode> dataModes = Lists.newLinkedList();
    +        dataModes.add(leftField.getType().getMode());
    +        dataModes.add(rightField.getType().getMode());
    +        builder.setMode(TypeCastRules.getLeastRestrictiveDataMode(dataModes));
     
    -      final List<String> outputColumnNames = Lists.newArrayList();
    -      final Iterator<MaterializedField> iterForNames = schemaForNames.iterator();
    -      while(iterForNames.hasNext()) {
    -        outputColumnNames.add(iterForNames.next().getPath());
    -      }
    -
    -      final Iterator<MaterializedField> iterForTypes = schemaForTypes.iterator();
    -      for(int i = 0; iterForTypes.hasNext(); ++i) {
    -        MaterializedField field = iterForTypes.next();
    -        outputFields.add(MaterializedField.create(outputColumnNames.get(i), field.getType()));
    +        container.addOrGet(MaterializedField.create(leftField.getPath(), builder.build()), callBack);
           }
    +      ++index;
         }
     
    -    public List<MaterializedField> getOutputFields() {
    -      if(outputFields == null) {
    -        throw new NullPointerException("Output fields have not been inferred");
    -      }
    -
    -      return outputFields;
    -    }
    +    assert !leftIter.hasNext() && ! rightIter.hasNext() : "Mis-match of column count should have been detected when validating sqlNode at planning";
    +  }
     
    -    public void killIncoming(boolean sendUpstream) {
    -      leftSide.getRecordBatch().kill(sendUpstream);
    -      rightSide.getRecordBatch().kill(sendUpstream);
    +  private void inferOutputFieldsOneSide(final BatchSchema schema) {
    +    for (MaterializedField field : schema) {
    +      container.addOrGet(field, callBack);
         }
    +  }
     
    -    public RecordBatch getLeftRecordBatch() {
    -      return leftSide.getRecordBatch();
    -    }
    +  private static boolean hasSameTypeAndMode(MaterializedField leftField, MaterializedField rightField) {
    --- End diff --
    
    Move the method to `MaterializedField`. 


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] drill pull request #906: DRILL-5546: Handle schema change exception failure ...

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

    https://github.com/apache/drill/pull/906#discussion_r135924386
  
    --- Diff: exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/project/ProjectRecordBatch.java ---
    @@ -768,4 +765,73 @@ else if (exprHasPrefix && refHasPrefix) {
           }
         }
       }
    +
    +  /**
    +   * handle FAST NONE specially when Project for query output. This happens when input returns a
    --- End diff --
    
    Done. 


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] drill pull request #906: DRILL-5546: Handle schema change exception failure ...

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/906#discussion_r135577415
  
    --- Diff: exec/java-exec/src/main/java/org/apache/drill/exec/record/AbstractSingleRecordBatch.java ---
    @@ -125,4 +131,19 @@ public BatchSchema getSchema() {
     
       protected abstract boolean setupNewSchema() throws SchemaChangeException;
       protected abstract IterOutcome doWork();
    +
    +  /**
    +   * Default behavior to handle fast NONE (incoming's first next() return NONE, in stead of OK_NEW_SCHEMA):
    +   * FAST NONE could happen when the underneath Scan operators do not produce any batch with schema.
    +   *
    +   * This behavior could be override in each individual operator, if the operator's semantics is to
    +   * inject a batch with schema.
    +   *
    +   * @return IterOutcome.NONE.
    +   */
    +  protected IterOutcome handleFastNone() {
    +    container.buildSchema(SelectionVectorMode.NONE);
    +    return IterOutcome.NONE;
    +  };
    --- End diff --
    
    Delete the semi-colon.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] drill pull request #906: DRILL-5546: Handle schema change exception failure ...

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/906#discussion_r134637327
  
    --- Diff: exec/java-exec/src/test/java/org/apache/drill/exec/physical/unit/TestEmptyBatchMiniPlan.java ---
    @@ -0,0 +1,453 @@
    +/*
    + * 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
    + * <p/>
    + * http://www.apache.org/licenses/LICENSE-2.0
    + * <p/>
    + * 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.physical.unit;
    +
    +import com.google.common.collect.Lists;
    +import org.apache.calcite.rel.RelFieldCollation;
    +import org.apache.calcite.rel.core.JoinRelType;
    +import org.apache.drill.common.expression.SchemaPath;
    +import org.apache.drill.common.types.TypeProtos;
    +import org.apache.drill.common.util.FileUtils;
    +import org.apache.drill.exec.physical.base.PhysicalOperator;
    +import org.apache.drill.exec.physical.config.ExternalSort;
    +import org.apache.drill.exec.physical.config.Filter;
    +import org.apache.drill.exec.physical.config.FlattenPOP;
    +import org.apache.drill.exec.physical.config.HashAggregate;
    +import org.apache.drill.exec.physical.config.HashJoinPOP;
    +import org.apache.drill.exec.physical.config.Limit;
    +import org.apache.drill.exec.physical.config.MergeJoinPOP;
    +import org.apache.drill.exec.physical.config.Project;
    +import org.apache.drill.exec.physical.config.StreamingAggregate;
    +import org.apache.drill.exec.physical.config.UnionAll;
    +import org.apache.drill.exec.planner.physical.AggPrelBase;
    +import org.apache.drill.exec.record.BatchSchema;
    +import org.apache.drill.exec.record.RecordBatch;
    +import org.apache.drill.exec.store.dfs.DrillFileSystem;
    +import org.apache.drill.test.rowSet.SchemaBuilder;
    +import org.apache.hadoop.conf.Configuration;
    +import org.apache.hadoop.fs.FileSystem;
    +import org.junit.BeforeClass;
    +import org.junit.Ignore;
    +import org.junit.Test;
    +
    +import java.util.Collections;
    +import java.util.List;
    +
    +public class TestEmptyBatchMiniPlan extends MiniPlanUnitTestBase{
    +  protected static DrillFileSystem fs;
    +
    +  @BeforeClass
    +  public static void initFS() throws Exception {
    +    Configuration conf = new Configuration();
    +    conf.set(FileSystem.FS_DEFAULT_NAME_KEY, FileSystem.DEFAULT_FS);
    +    fs = new DrillFileSystem(conf);
    +  }
    +
    +  @Test
    +  public void testEmptyJsonInput() throws Exception {
    +    RecordBatch scanBatch = createEmptyBatchFromJson();
    +
    +    new MiniPlanTestBuilder()
    +        .root(scanBatch)
    +        .expectNullBatch(true)
    +        .go();
    +  }
    +
    +  @Test
    +  public void testProjectEmpty() throws Exception {
    +    final PhysicalOperator project = new Project(parseExprs("x+5", "x"), null);
    +    testSingleInputEmptyBatchHandling(project);
    +  }
    +
    +  @Test
    +  public void testFilterEmpty() throws Exception {
    +    final PhysicalOperator filter = new Filter(null, parseExpr("a=5"), 1.0f);
    +    testSingleInputEmptyBatchHandling(filter);
    +  }
    +
    +  @Test
    +  public void testHashAggEmpty() throws Exception {
    +    final PhysicalOperator hashAgg = new HashAggregate(null, AggPrelBase.OperatorPhase.PHASE_1of1, parseExprs("a", "a"), parseExprs("sum(b)", "b_sum"), 1.0f);
    +    testSingleInputEmptyBatchHandling(hashAgg);
    +  }
    +
    +  @Test
    +  public void testStreamingAggEmpty() throws Exception {
    +    final PhysicalOperator hashAgg = new StreamingAggregate(null, parseExprs("a", "a"), parseExprs("sum(b)", "b_sum"), 1.0f);
    +    testSingleInputEmptyBatchHandling(hashAgg);
    +  }
    +
    +  @Test
    +  public void testSortEmpty() throws Exception {
    +    final PhysicalOperator sort = new ExternalSort(null,
    +        Lists.newArrayList(ordering("b", RelFieldCollation.Direction.ASCENDING, RelFieldCollation.NullDirection.FIRST)), false);
    +    testSingleInputEmptyBatchHandling(sort);
    +  }
    +
    +  @Test
    +  public void testLimitEmpty() throws Exception {
    +    final PhysicalOperator limit = new Limit(null, 10, 5);
    +    testSingleInputEmptyBatchHandling(limit);
    +  }
    +
    +  @Test
    +  public void testFlattenEmpty() throws Exception {
    +    final PhysicalOperator flatten = new FlattenPOP(null, SchemaPath.getSimplePath("col1"));
    +    testSingleInputEmptyBatchHandling(flatten);
    +  }
    +
    +  @Test
    +  public void testUnionEmptyBoth() throws Exception {
    +    final PhysicalOperator unionAll = new UnionAll(Collections.EMPTY_LIST); // Children list is provided through RecordBatch
    +    testTwoInputEmptyBatchHandling(unionAll);
    +  }
    +
    +  @Test
    +  public void testHashJoinEmptyBoth() throws Exception {
    +   final PhysicalOperator join = new HashJoinPOP(null, null, Lists.newArrayList(joinCond("a", "EQUALS", "b")), JoinRelType.INNER);
    +    testTwoInputEmptyBatchHandling(join);
    +  }
    +
    +  @Test
    +  public void testLeftHashJoinEmptyBoth() throws Exception {
    +    final PhysicalOperator join = new HashJoinPOP(null, null, Lists.newArrayList(joinCond("a", "EQUALS", "b")), JoinRelType.LEFT);
    +    testTwoInputEmptyBatchHandling(join);
    +  }
    +
    +  @Test
    +  public void testRightHashJoinEmptyBoth() throws Exception {
    +    final PhysicalOperator join = new HashJoinPOP(null, null, Lists.newArrayList(joinCond("a", "EQUALS", "b")), JoinRelType.RIGHT);
    +    testTwoInputEmptyBatchHandling(join);
    +  }
    +
    +  @Test
    +  public void testFullHashJoinEmptyBoth() throws Exception {
    +    final PhysicalOperator join = new HashJoinPOP(null, null, Lists.newArrayList(joinCond("a", "EQUALS", "b")), JoinRelType.FULL);
    +    testTwoInputEmptyBatchHandling(join);
    +  }
    +
    +  @Test
    +  public void testMergeJoinEmptyBoth() throws Exception {
    +    final PhysicalOperator join = new MergeJoinPOP(null, null, Lists.newArrayList(joinCond("a", "EQUALS", "b")), JoinRelType.INNER);
    +    testTwoInputEmptyBatchHandling(join);
    +  }
    +
    +  @Test
    +  public void testLeftMergeJoinEmptyBoth() throws Exception {
    +    final PhysicalOperator join = new MergeJoinPOP(null, null, Lists.newArrayList(joinCond("a", "EQUALS", "b")), JoinRelType.LEFT);
    +    testTwoInputEmptyBatchHandling(join);
    +  }
    +
    +  @Test
    +  public void testRightMergeJoinEmptyBoth() throws Exception {
    +    final PhysicalOperator join = new MergeJoinPOP(null, null, Lists.newArrayList(joinCond("a", "EQUALS", "b")), JoinRelType.RIGHT);
    +    testTwoInputEmptyBatchHandling(join);
    +  }
    +
    +  @Test
    +  @Ignore("Full Merge join is not supported.")
    +  public void testFullMergeJoinEmptyBoth() throws Exception {
    +    final PhysicalOperator join = new MergeJoinPOP(null, null, Lists.newArrayList(joinCond("a", "EQUALS", "b")), JoinRelType.FULL);
    +    testTwoInputEmptyBatchHandling(join);
    +  }
    +
    +  @Test
    +  public void testUnionLeftEmtpy() throws Exception {
    +    final PhysicalOperator unionAll = new UnionAll(Collections.EMPTY_LIST); // Children list is provided through RecordBatch
    +
    +    RecordBatch left = createEmptyBatchFromJson();
    +
    +    String file = FileUtils.getResourceAsFile("/tpchmulti/region/01.parquet").toURI().toString();
    +
    +    RecordBatch scanBatch = new ParquetScanBuilder()
    +        .fileSystem(fs)
    +        .columnsToRead("R_REGIONKEY")
    +        .inputPaths(Lists.newArrayList(file))
    +        .build();
    +
    +    RecordBatch projectBatch = new PopBuilder()
    +        .physicalOperator(new Project(parseExprs("R_REGIONKEY+10", "regionkey"), null))
    +        .addInput(scanBatch)
    +        .build();
    +
    +    RecordBatch unionBatch = new PopBuilder()
    +        .physicalOperator(unionAll)
    +        .addInput(left)
    +        .addInput(projectBatch)
    +        .build();
    +
    +    BatchSchema expectedSchema = new SchemaBuilder()
    +        .add("regionkey", TypeProtos.MinorType.BIGINT)
    +        .build();
    +
    +    new MiniPlanTestBuilder()
    +        .root(unionBatch)
    +        .expectSchema(expectedSchema)
    +        .baselineValues(10L)
    +        .baselineValues(11L)
    +        .go();
    +  }
    +
    +
    +  @Test
    +  public void testHashJoinLeftEmpty() throws Exception {
    +    RecordBatch left = createEmptyBatchFromJson();
    +
    +    List<String> rightJsonBatches = Lists.newArrayList(
    +        "[{\"a\": 50, \"b\" : 10 }]");
    +
    +    RecordBatch rightScan = new JsonScanBuilder()
    +        .jsonBatches(rightJsonBatches)
    +        .columnsToRead("a", "b")
    +        .build();
    +
    +    RecordBatch joinBatch = new PopBuilder()
    +        .physicalOperator(new HashJoinPOP(null, null, Lists.newArrayList(joinCond("a", "EQUALS", "a2")), JoinRelType.INNER))
    +        .addInput(left)
    +        .addInput(rightScan)
    +        .build();
    +
    +    BatchSchema expectedSchema = new SchemaBuilder()
    +        .addNullable("a", TypeProtos.MinorType.BIGINT)
    +        .addNullable("b", TypeProtos.MinorType.BIGINT)
    +        .withSVMode(BatchSchema.SelectionVectorMode.NONE)
    +        .build();
    +
    +    new MiniPlanTestBuilder()
    +        .root(joinBatch)
    +        .expectSchema(expectedSchema)
    +        .expectZeroRow(true)
    +        .go();
    +  }
    +
    +  @Test
    +  public void testHashJoinRightEmpty() throws Exception {
    +    List<String> leftJsonBatches = Lists.newArrayList(
    +        "[{\"a\": 50, \"b\" : 10 }]");
    +
    +    RecordBatch leftScan = new JsonScanBuilder()
    +        .jsonBatches(leftJsonBatches)
    +        .columnsToRead("a", "b")
    +        .build();
    +
    +    RecordBatch right = createEmptyBatchFromJson();
    +
    +    RecordBatch joinBatch = new PopBuilder()
    +        .physicalOperator(new HashJoinPOP(null, null, Lists.newArrayList(joinCond("a", "EQUALS", "a2")), JoinRelType.INNER))
    +        .addInput(leftScan)
    +        .addInput(right)
    +        .build();
    +
    +    BatchSchema expectedSchema = new SchemaBuilder()
    +        .addNullable("a", TypeProtos.MinorType.BIGINT)
    +        .addNullable("b", TypeProtos.MinorType.BIGINT)
    +        .withSVMode(BatchSchema.SelectionVectorMode.NONE)
    +        .build();
    +
    +    new MiniPlanTestBuilder()
    +        .root(joinBatch)
    +        .expectSchema(expectedSchema)
    +        .expectZeroRow(true)
    +        .go();
    +  }
    +
    +
    +  @Test
    +  public void testLeftHashJoinLeftEmpty() throws Exception {
    +    RecordBatch left = createEmptyBatchFromJson();
    +
    +    List<String> rightJsonBatches = Lists.newArrayList(
    +        "[{\"a\": 50, \"b\" : 10 }]");
    +
    +    RecordBatch rightScan = new JsonScanBuilder()
    +        .jsonBatches(rightJsonBatches)
    +        .columnsToRead("a", "b")
    +        .build();
    +
    +    RecordBatch joinBatch = new PopBuilder()
    +        .physicalOperator(new HashJoinPOP(null, null, Lists.newArrayList(joinCond("a", "EQUALS", "a2")), JoinRelType.LEFT))
    +        .addInput(left)
    +        .addInput(rightScan)
    +        .build();
    +
    +    BatchSchema expectedSchema = new SchemaBuilder()
    +        .addNullable("a", TypeProtos.MinorType.BIGINT)
    +        .addNullable("b", TypeProtos.MinorType.BIGINT)
    +        .withSVMode(BatchSchema.SelectionVectorMode.NONE)
    +        .build();
    +
    +    new MiniPlanTestBuilder()
    +        .root(joinBatch)
    +        .expectSchema(expectedSchema)
    +        .expectZeroRow(true)
    +        .go();
    +  }
    +
    +  @Test
    +  public void testLeftHashJoinRightEmpty() throws Exception {
    +    List<String> leftJsonBatches = Lists.newArrayList(
    +        "[{\"a\": 50, \"b\" : 10 }]");
    +
    +    RecordBatch leftScan = new JsonScanBuilder()
    +        .jsonBatches(leftJsonBatches)
    +        .columnsToRead("a", "b")
    +        .build();
    +
    +    RecordBatch right = createEmptyBatchFromJson();
    +
    +    RecordBatch joinBatch = new PopBuilder()
    +        .physicalOperator(new HashJoinPOP(null, null, Lists.newArrayList(joinCond("a", "EQUALS", "a2")), JoinRelType.LEFT))
    +        .addInput(leftScan)
    +        .addInput(right)
    +        .build();
    +
    +    BatchSchema expectedSchema = new SchemaBuilder()
    +        .addNullable("a", TypeProtos.MinorType.BIGINT)
    +        .addNullable("b", TypeProtos.MinorType.BIGINT)
    +        .withSVMode(BatchSchema.SelectionVectorMode.NONE)
    +        .build();
    +
    +    new MiniPlanTestBuilder()
    +        .root(joinBatch)
    +        .expectSchema(expectedSchema)
    +        .baselineValues(50L, 10L)
    +        .go();
    +  }
    +
    +  @Test
    +  public void testUnionFilterAll() throws Exception {
    +    List<String> leftJsonBatches = Lists.newArrayList(
    +        "[{\"a\": 5, \"b\" : 1 }]");
    +
    +    List<String> rightJsonBatches = Lists.newArrayList(
    +        "[{\"a\": 50, \"b\" : 10 }]");
    +
    +    RecordBatch leftScan = new JsonScanBuilder()
    +        .jsonBatches(leftJsonBatches)
    +        .columnsToRead("a", "b")
    +        .build();
    +
    +    RecordBatch leftFilter = new PopBuilder()
    +        .physicalOperator(new Filter(null, parseExpr("a < 0"), 1.0f))
    +        .addInput(leftScan)
    +        .build();
    +
    +    RecordBatch rightScan = new JsonScanBuilder()
    +        .jsonBatches(rightJsonBatches)
    +        .columnsToRead("a", "b")
    +        .build();
    +
    +    RecordBatch rightFilter = new PopBuilder()
    +        .physicalOperator(new Filter(null, parseExpr("a < 0"), 1.0f))
    +        .addInput(rightScan)
    +        .build();
    +
    +    RecordBatch batch = new PopBuilder()
    +        .physicalOperator(new UnionAll(Collections.EMPTY_LIST)) // Children list is provided through RecordBatch
    +        .addInput(leftFilter)
    +        .addInput(rightFilter)
    +        .build();
    +
    +    BatchSchema expectedSchema = new SchemaBuilder()
    +        .addNullable("a", TypeProtos.MinorType.BIGINT)
    +        .addNullable("b", TypeProtos.MinorType.BIGINT)
    +        .withSVMode(BatchSchema.SelectionVectorMode.NONE)
    +        .build();
    +
    +    new MiniPlanTestBuilder()
    +        .root(batch)
    +        .expectSchema(expectedSchema)
    +        .expectZeroRow(true)
    +        .go();
    +  }
    +
    +  @Test
    +  public void testOutputProjectEmpty() throws Exception {
    --- End diff --
    
    This test does a great job of handling a wide variety of expressions.
    
    But, looking at the code under test, it does not exercise the following code paths:
    
    * `SELECT *`
    * `SELECT foo.*, bar.*`
    * Error in the expression tree materializer
    * Map column references: `myMap.innerMap.foo`
    * Array references: `columns[5]`


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] drill pull request #906: DRILL-5546: Handle schema change exception failure ...

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/906#discussion_r135370581
  
    --- Diff: exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/union/UnionAllRecordBatch.java ---
    @@ -130,562 +145,248 @@ public IterOutcome innerNext() {
       }
     
       @Override
    -  public WritableBatch getWritableBatch() {
    -    return WritableBatch.get(this);
    +  public int getRecordCount() {
    +    return recordCount;
       }
     
    -  private void setValueCount(int count) {
    -    for (ValueVector v : allocationVectors) {
    -      ValueVector.Mutator m = v.getMutator();
    -      m.setValueCount(count);
    -    }
    -  }
     
    -  private boolean doAlloc() {
    -    for (ValueVector v : allocationVectors) {
    -      try {
    -        AllocationHelper.allocateNew(v, current.getRecordCount());
    -      } catch (OutOfMemoryException ex) {
    -        return false;
    -      }
    +  @SuppressWarnings("resource")
    +  private IterOutcome doWork(RecordBatch inputBatch, boolean newSchema) throws ClassTransformationException, IOException, SchemaChangeException {
    +    if (inputBatch.getSchema().getFieldCount() != container.getSchema().getFieldCount()) {
    +      // wrong.
         }
    -    return true;
    -  }
     
    -  @SuppressWarnings("resource")
    -  private IterOutcome doWork() throws ClassTransformationException, IOException, SchemaChangeException {
    -    if (allocationVectors != null) {
    -      for (ValueVector v : allocationVectors) {
    -        v.clear();
    -      }
    +    if (newSchema) {
    +      createUnionAller(inputBatch);
         }
     
    -    allocationVectors = Lists.newArrayList();
    -    transfers.clear();
    +    container.zeroVectors();
     
    -    // If both sides of Union-All are empty
    -    if(unionAllInput.isBothSideEmpty()) {
    -      for(int i = 0; i < outputFields.size(); ++i) {
    -        final String colName = outputFields.get(i).getPath();
    -        final MajorType majorType = MajorType.newBuilder()
    -            .setMinorType(MinorType.INT)
    -            .setMode(DataMode.OPTIONAL)
    -            .build();
    -
    -        MaterializedField outputField = MaterializedField.create(colName, majorType);
    -        ValueVector vv = container.addOrGet(outputField, callBack);
    -        allocationVectors.add(vv);
    -      }
    +    for (final ValueVector v : this.allocationVectors) {
    +      AllocationHelper.allocateNew(v, inputBatch.getRecordCount());
    +    }
     
    -      container.buildSchema(BatchSchema.SelectionVectorMode.NONE);
    +    recordCount = unionall.unionRecords(0, inputBatch.getRecordCount(), 0);
    +    for (final ValueVector v : allocationVectors) {
    +      final ValueVector.Mutator m = v.getMutator();
    +      m.setValueCount(recordCount);
    +    }
    +
    +    if (callBack.getSchemaChangedAndReset()) {
           return IterOutcome.OK_NEW_SCHEMA;
    +    } else {
    +      return IterOutcome.OK;
         }
    +  }
    +
    +  private void createUnionAller(RecordBatch inputBatch) throws ClassTransformationException, IOException, SchemaChangeException {
    +    transfers.clear();
    +    allocationVectors.clear();;
     
         final ClassGenerator<UnionAller> cg = CodeGenerator.getRoot(UnionAller.TEMPLATE_DEFINITION, context.getFunctionRegistry(), context.getOptions());
         cg.getCodeGenerator().plainJavaCapable(true);
         // Uncomment out this line to debug the generated code.
    -//    cg.getCodeGenerator().saveCodeForDebugging(true);
    +    //    cg.getCodeGenerator().saveCodeForDebugging(true);
    +
         int index = 0;
    -    for(VectorWrapper<?> vw : current) {
    -       ValueVector vvIn = vw.getValueVector();
    -      // get the original input column names
    -      SchemaPath inputPath = SchemaPath.getSimplePath(vvIn.getField().getPath());
    -      // get the renamed column names
    -      SchemaPath outputPath = SchemaPath.getSimplePath(outputFields.get(index).getPath());
    +    for(VectorWrapper<?> vw : inputBatch) {
    +      ValueVector vvIn = vw.getValueVector();
    +      ValueVector vvOut = container.getValueVector(index).getValueVector();
     
           final ErrorCollector collector = new ErrorCollectorImpl();
           // According to input data names, Minortypes, Datamodes, choose to
           // transfer directly,
           // rename columns or
           // cast data types (Minortype or DataMode)
    -      if (hasSameTypeAndMode(outputFields.get(index), vw.getValueVector().getField())) {
    +      if (hasSameTypeAndMode(container.getSchema().getColumn(index), vvIn.getField())
    +          && vvIn.getField().getType().getMinorType() != TypeProtos.MinorType.MAP // Per DRILL-5521, existing bug for map transfer
    +          ) {
             // Transfer column
    -
    -        MajorType outputFieldType = outputFields.get(index).getType();
    -        MaterializedField outputField = MaterializedField.create(outputPath.getAsUnescapedPath(), outputFieldType);
    -
    -        /*
    -          todo: Fix if condition when DRILL-4824 is merged
    -          If condition should be changed to:
    -          `if (outputFields.get(index).getPath().equals(inputPath.getAsUnescapedPath())) {`
    -          DRILL-5419 has changed condition to correct one but this caused regression (DRILL-5521).
    -          Root cause is missing indication of child column in map types when it is null.
    -          DRILL-4824 is re-working json reader implementation, including map types and will fix this problem.
    -          Reverting condition to previous one to avoid regression till DRILL-4824 is merged.
    -          Unit test - TestJsonReader.testKvgenWithUnionAll().
    -         */
    -        if (outputFields.get(index).getPath().equals(inputPath)) {
    -          ValueVector vvOut = container.addOrGet(outputField);
    -          TransferPair tp = vvIn.makeTransferPair(vvOut);
    -          transfers.add(tp);
    +        TransferPair tp = vvIn.makeTransferPair(vvOut);
    +        transfers.add(tp);
             // Copy data in order to rename the column
    -        } else {
    -          final LogicalExpression expr = ExpressionTreeMaterializer.materialize(inputPath, current, collector, context.getFunctionRegistry() );
    -          if (collector.hasErrors()) {
    -            throw new SchemaChangeException(String.format("Failure while trying to materialize incoming schema.  Errors:\n %s.", collector.toErrorString()));
    -          }
    -
    -          ValueVector vv = container.addOrGet(outputField, callBack);
    -          allocationVectors.add(vv);
    -          TypedFieldId fid = container.getValueVectorId(SchemaPath.getSimplePath(outputField.getPath()));
    -          ValueVectorWriteExpression write = new ValueVectorWriteExpression(fid, expr, true);
    -          cg.addExpr(write);
    -        }
    -      // Cast is necessary
    +      } else if (vvIn.getField().getType().getMinorType() == TypeProtos.MinorType.NULL) {
    +        continue;
           } else {
    -        LogicalExpression expr = ExpressionTreeMaterializer.materialize(inputPath, current, collector, context.getFunctionRegistry());
    +        SchemaPath inputPath = SchemaPath.getSimplePath(vvIn.getField().getPath());
    +        MaterializedField inField = vvIn.getField();
    +        MaterializedField outputField = vvOut.getField();
    +
    +        LogicalExpression expr = ExpressionTreeMaterializer.materialize(inputPath, inputBatch, collector, context.getFunctionRegistry());
    +
             if (collector.hasErrors()) {
               throw new SchemaChangeException(String.format("Failure while trying to materialize incoming schema.  Errors:\n %s.", collector.toErrorString()));
             }
     
             // If the inputs' DataMode is required and the outputs' DataMode is not required
             // cast to the one with the least restriction
    -        if(vvIn.getField().getType().getMode() == DataMode.REQUIRED
    -            && outputFields.get(index).getType().getMode() != DataMode.REQUIRED) {
    -          expr = ExpressionTreeMaterializer.convertToNullableType(expr, vvIn.getField().getType().getMinorType(), context.getFunctionRegistry(), collector);
    +        if(inField.getType().getMode() == TypeProtos.DataMode.REQUIRED
    +            && outputField.getType().getMode() != TypeProtos.DataMode.REQUIRED) {
    +          expr = ExpressionTreeMaterializer.convertToNullableType(expr, inField.getType().getMinorType(), context.getFunctionRegistry(), collector);
               if (collector.hasErrors()) {
                 throw new SchemaChangeException(String.format("Failure while trying to materialize incoming schema.  Errors:\n %s.", collector.toErrorString()));
               }
             }
     
             // If two inputs' MinorTypes are different,
             // Insert a cast before the Union operation
    -        if(vvIn.getField().getType().getMinorType() != outputFields.get(index).getType().getMinorType()) {
    -          expr = ExpressionTreeMaterializer.addCastExpression(expr, outputFields.get(index).getType(), context.getFunctionRegistry(), collector);
    +        if(inField.getType().getMinorType() != outputField.getType().getMinorType()) {
    +          expr = ExpressionTreeMaterializer.addCastExpression(expr, outputField.getType(), context.getFunctionRegistry(), collector);
               if (collector.hasErrors()) {
                 throw new SchemaChangeException(String.format("Failure while trying to materialize incoming schema.  Errors:\n %s.", collector.toErrorString()));
               }
             }
     
    -        final MaterializedField outputField = MaterializedField.create(outputPath.getAsUnescapedPath(), expr.getMajorType());
    -        ValueVector vector = container.addOrGet(outputField, callBack);
    -        allocationVectors.add(vector);
             TypedFieldId fid = container.getValueVectorId(SchemaPath.getSimplePath(outputField.getPath()));
     
    -        boolean useSetSafe = !(vector instanceof FixedWidthVector);
    +        boolean useSetSafe = !(vvOut instanceof FixedWidthVector);
             ValueVectorWriteExpression write = new ValueVectorWriteExpression(fid, expr, useSetSafe);
             cg.addExpr(write);
    +
    +        allocationVectors.add(vvOut);
           }
           ++index;
         }
     
         unionall = context.getImplementationClass(cg.getCodeGenerator());
    -    unionall.setup(context, current, this, transfers);
    -
    -    if(!schemaAvailable) {
    -      container.buildSchema(BatchSchema.SelectionVectorMode.NONE);
    -      schemaAvailable = true;
    -    }
    -
    -    if(!doAlloc()) {
    -      return IterOutcome.OUT_OF_MEMORY;
    -    }
    -
    -    recordCount = unionall.unionRecords(0, current.getRecordCount(), 0);
    -    setValueCount(recordCount);
    -    return IterOutcome.OK;
    -  }
    -
    -  public static boolean hasSameTypeAndMode(MaterializedField leftField, MaterializedField rightField) {
    -    return (leftField.getType().getMinorType() == rightField.getType().getMinorType())
    -        && (leftField.getType().getMode() == rightField.getType().getMode());
    -  }
    -
    -  // This method is used by inner class to point the reference `current` to the correct record batch
    -  private void setCurrentRecordBatch(RecordBatch target) {
    -    this.current = target;
    -  }
    -
    -  // This method is used by inner class to clear the current record batch
    -  private void clearCurrentRecordBatch() {
    -    for(VectorWrapper<?> v: current) {
    -      v.clear();
    -    }
    +    unionall.setup(context, inputBatch, this, transfers);
       }
     
    -  public static class UnionAllInput {
    -    private UnionAllRecordBatch unionAllRecordBatch;
    -    private List<MaterializedField> outputFields;
    -    private OneSideInput leftSide;
    -    private OneSideInput rightSide;
    -    private IterOutcome upstream = IterOutcome.NOT_YET;
    -    private boolean leftIsFinish = false;
    -    private boolean rightIsFinish = false;
    -
    -    // These two schemas are obtained from the first record batches of the left and right inputs
    -    // They are used to check if the schema is changed between recordbatches
    -    private BatchSchema leftSchema;
    -    private BatchSchema rightSchema;
    -    private boolean bothEmpty = false;
    -
    -    public UnionAllInput(UnionAllRecordBatch unionAllRecordBatch, RecordBatch left, RecordBatch right) {
    -      this.unionAllRecordBatch = unionAllRecordBatch;
    -      leftSide = new OneSideInput(left);
    -      rightSide = new OneSideInput(right);
    -    }
    -
    -    private void setBothSideEmpty(boolean bothEmpty) {
    -      this.bothEmpty = bothEmpty;
    -    }
    -
    -    private boolean isBothSideEmpty() {
    -      return bothEmpty;
    -    }
    -
    -    public IterOutcome nextBatch() throws SchemaChangeException {
    -      if(upstream == RecordBatch.IterOutcome.NOT_YET) {
    -        IterOutcome iterLeft = leftSide.nextBatch();
    -        switch(iterLeft) {
    -          case OK_NEW_SCHEMA:
    -            /*
    -             * If the first few record batches are all empty,
    -             * there is no way to tell whether these empty batches are coming from empty files.
    -             * It is incorrect to infer output types when either side could be coming from empty.
    -             *
    -             * Thus, while-loop is necessary to skip those empty batches.
    -             */
    -            whileLoop:
    -            while(leftSide.getRecordBatch().getRecordCount() == 0) {
    -              iterLeft = leftSide.nextBatch();
    -
    -              switch(iterLeft) {
    -                case STOP:
    -                case OUT_OF_MEMORY:
    -                  return iterLeft;
    -
    -                case NONE:
    -                  // Special Case: The left side was an empty input.
    -                  leftIsFinish = true;
    -                  break whileLoop;
    -
    -                case NOT_YET:
    -                case OK_NEW_SCHEMA:
    -                case OK:
    -                  continue whileLoop;
    -
    -                default:
    -                  throw new IllegalStateException(
    -                      String.format("Unexpected state %s.", iterLeft));
    -              }
    -            }
    -
    -            break;
    -          case STOP:
    -          case OUT_OF_MEMORY:
    -            return iterLeft;
    -
    -          default:
    -            throw new IllegalStateException(
    -                String.format("Unexpected state %s.", iterLeft));
    -        }
    -
    -        IterOutcome iterRight = rightSide.nextBatch();
    -        switch(iterRight) {
    -          case OK_NEW_SCHEMA:
    -            // Unless there is no record batch on the left side of the inputs,
    -            // always start processing from the left side.
    -            if(leftIsFinish) {
    -              unionAllRecordBatch.setCurrentRecordBatch(rightSide.getRecordBatch());
    -            } else {
    -              unionAllRecordBatch.setCurrentRecordBatch(leftSide.getRecordBatch());
    -            }
    -            // If the record count of the first batch from right input is zero,
    -            // there are two possibilities:
    -            // 1. The right side is an empty input (e.g., file).
    -            // 2. There will be more records carried by later batches.
    -
    -            /*
    -             * If the first few record batches are all empty,
    -             * there is no way to tell whether these empty batches are coming from empty files.
    -             * It is incorrect to infer output types when either side could be coming from empty.
    -             *
    -             * Thus, while-loop is necessary to skip those empty batches.
    -             */
    -            whileLoop:
    -            while(rightSide.getRecordBatch().getRecordCount() == 0) {
    -              iterRight = rightSide.nextBatch();
    -              switch(iterRight) {
    -                case STOP:
    -                case OUT_OF_MEMORY:
    -                  return iterRight;
    -
    -                case NONE:
    -                  // Special Case: The right side was an empty input.
    -                  rightIsFinish = true;
    -                  break whileLoop;
    -
    -                case NOT_YET:
    -                case OK_NEW_SCHEMA:
    -                case OK:
    -                  continue whileLoop;
    -
    -                default:
    -                  throw new IllegalStateException(
    -                      String.format("Unexpected state %s.", iterRight));
    -              }
    -            }
    -
    -            if(leftIsFinish && rightIsFinish) {
    -              setBothSideEmpty(true);
    -            }
    -
    -            inferOutputFields();
    -            break;
    -
    -          case STOP:
    -          case OUT_OF_MEMORY:
    -            return iterRight;
    -
    -          default:
    -            throw new IllegalStateException(
    -                String.format("Unexpected state %s.", iterRight));
    -        }
    -
    -
    -
    -        upstream = IterOutcome.OK_NEW_SCHEMA;
    -        return upstream;
    -      } else {
    -        if(isBothSideEmpty()) {
    -          return IterOutcome.NONE;
    -        }
    -
    -        unionAllRecordBatch.clearCurrentRecordBatch();
    -
    -        if(leftIsFinish && rightIsFinish) {
    -          upstream = IterOutcome.NONE;
    -          return upstream;
    -        } else if(leftIsFinish) {
    -          IterOutcome iterOutcome = rightSide.nextBatch();
    -
    -          switch(iterOutcome) {
    -            case NONE:
    -              rightIsFinish = true;
    -              // fall through
    -            case STOP:
    -            case OUT_OF_MEMORY:
    -              upstream = iterOutcome;
    -              return upstream;
    -
    -            case OK_NEW_SCHEMA:
    -              if(!rightSide.getRecordBatch().getSchema().equals(rightSchema)) {
    -                throw new SchemaChangeException("Schema change detected in the right input of Union-All. This is not currently supported");
    -              }
    -              iterOutcome = IterOutcome.OK;
    -              // fall through
    -            case OK:
    -              unionAllRecordBatch.setCurrentRecordBatch(rightSide.getRecordBatch());
    -              upstream = iterOutcome;
    -              return upstream;
    -
    -            default:
    -              throw new IllegalStateException(String.format("Unknown state %s.", upstream));
    -          }
    -        } else if(rightIsFinish) {
    -          IterOutcome iterOutcome = leftSide.nextBatch();
    -          switch(iterOutcome) {
    -            case STOP:
    -            case OUT_OF_MEMORY:
    -            case NONE:
    -              upstream = iterOutcome;
    -              return upstream;
    -
    -            case OK:
    -              unionAllRecordBatch.setCurrentRecordBatch(leftSide.getRecordBatch());
    -              upstream = iterOutcome;
    -              return upstream;
    -
    -            default:
    -              throw new IllegalStateException(String.format("Unknown state %s.", iterOutcome));
    -          }
    -        } else {
    -          IterOutcome iterOutcome = leftSide.nextBatch();
    -
    -          switch(iterOutcome) {
    -            case STOP:
    -            case OUT_OF_MEMORY:
    -              upstream = iterOutcome;
    -              return upstream;
    -
    -            case OK_NEW_SCHEMA:
    -              if(!leftSide.getRecordBatch().getSchema().equals(leftSchema)) {
    -                throw new SchemaChangeException("Schema change detected in the left input of Union-All. This is not currently supported");
    -              }
    -
    -              iterOutcome = IterOutcome.OK;
    -              // fall through
    -            case OK:
    -              unionAllRecordBatch.setCurrentRecordBatch(leftSide.getRecordBatch());
    -              upstream = iterOutcome;
    -              return upstream;
    -
    -            case NONE:
    -              unionAllRecordBatch.setCurrentRecordBatch(rightSide.getRecordBatch());
    -              upstream = IterOutcome.OK;
    -              leftIsFinish = true;
    -              return upstream;
    -
    -            default:
    -              throw new IllegalStateException(String.format("Unknown state %s.", upstream));
    -          }
    -        }
    -      }
    -    }
     
    -    /**
    -     *
    -     * Summarize the inference in the four different situations:
    -     * First of all, the field names are always determined by the left side
    -     * (Even when the left side is from an empty file, we have the column names.)
    -     *
    -     * Cases:
    -     * 1. Left: non-empty; Right: non-empty
    -     *      types determined by both sides with implicit casting involved
    -     * 2. Left: empty; Right: non-empty
    -     *      type from the right
    -     * 3. Left: non-empty; Right: empty
    -     *      types from the left
    -     * 4. Left: empty; Right: empty
    -     *      types are nullable integer
    -     */
    -    private void inferOutputFields() {
    -      if(!leftIsFinish && !rightIsFinish) {
    -        // Both sides are non-empty
    -        inferOutputFieldsBothSide();
    -      } else if(!rightIsFinish) {
    -        // Left side is non-empty
    -        // While use left side's column names as output column names,
    -        // use right side's column types as output column types.
    -        inferOutputFieldsFromSingleSide(
    -            leftSide.getRecordBatch().getSchema(),
    -            rightSide.getRecordBatch().getSchema());
    +  // The output table's column names always follow the left table,
    --- End diff --
    
    Please clarify just a bit more. We match columns by name, correct? So, the only name conflicts might be by case? "A" on the left, "a" on the right? So, is it more accurate to say that the case of column names always follow the left table?
    
    Or, are the columns correlated by position, and so both the name and type follow the left table?


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] drill pull request #906: DRILL-5546: Handle schema change exception failure ...

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/906#discussion_r134305681
  
    --- Diff: exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/ScanBatch.java ---
    @@ -152,97 +157,75 @@ public void kill(boolean sendUpstream) {
         }
       }
     
    -  private void releaseAssets() {
    -    container.zeroVectors();
    -  }
    -
    -  private void clearFieldVectorMap() {
    -    for (final ValueVector v : mutator.fieldVectorMap().values()) {
    -      v.clear();
    -    }
    -  }
    -
       @Override
       public IterOutcome next() {
         if (done) {
           return IterOutcome.NONE;
         }
         oContext.getStats().startProcessing();
         try {
    -      try {
    -        injector.injectChecked(context.getExecutionControls(), "next-allocate", OutOfMemoryException.class);
    -
    -        currentReader.allocate(mutator.fieldVectorMap());
    -      } catch (OutOfMemoryException e) {
    -        clearFieldVectorMap();
    -        throw UserException.memoryError(e).build(logger);
    -      }
    -      while ((recordCount = currentReader.next()) == 0) {
    +      while (true) {
             try {
    -          if (!readers.hasNext()) {
    -            // We're on the last reader, and it has no (more) rows.
    -            currentReader.close();
    -            releaseAssets();
    -            done = true;  // have any future call to next() return NONE
    -
    -            if (mutator.isNewSchema()) {
    -              // This last reader has a new schema (e.g., we have a zero-row
    -              // file or other source).  (Note that some sources have a non-
    -              // null/non-trivial schema even when there are no rows.)
    +          injector.injectChecked(context.getExecutionControls(), "next-allocate", OutOfMemoryException.class);
    --- End diff --
    
    From the PR description:
    
    > 1. Skip RecordReader if it returns 0 row && present same schema. A new schema (by calling Mutator.isNewSchema() ) means either a new top level field is added, or a field in a nested field is added, or an existing field type is changed.
    
    The code, however, adds an additional condition: if implicit fields change. (But, as noted below, that should never occur in practice.)
    
    What happens on the first reader? There is no schema, so *any* schema is a new schema. Suppose the file is JSON and the schema is built on the fly. Does the code handle the case that we have no schema (first reader), and that reader adds no columns?
    
    Or, according to the logic that the downstream wants to know the schema, even if there are no records, do we send an empty schema (schema with no columns) downstream, because that is an accurate representation of an empty JSON file?
    
    What happens in the case of an empty JSON file following a non-empty file? In this case, do we consider the empty schema as a schema change relative to a non-empty change?
    
    In short, can we generalize this first rule a bit?
    
    > 2. Implicit columns are added and populated only when the input is not empty, i.e. the batch contains > 0 row or rowCount == 0 && new schema.
    
    How does this interact with a scan batch that has only one file, and that file is empty? Would we return the empty schema downstream? With the implicit columns?
    
    > 3. ScanBatch will return NONE directly (called as "fast NONE"), if all its RecordReaders haver empty input and thus are skipped, in stead of returing OK_NEW_SCHEMA first.
    
    
    This is just a bit ambiguous. If the reader is JSON, then an empty file has an empty schema (for reasons cited above.)
    
    But, if the input is CSV, then we *always* have a schema. If the file has column headers, then we know that the schema is, say, (a, b, c) because those are the headers. Or, if the file has no headers, the schema is always the `columns` array. So, should we send that schema downstream? If so, should it include the implicit columns?
    
    This, in fact, raises another issue (out of scope for this PR): if we return an empty batch with non-empty schema, we have no place to attach the implicit columns that will allow the user to figure out that, say, "foo.csv" is empty.
    
    On the other hand, if we say that an empty CSV file has no schema, then we can skip that file. The same might be true of JSON. What about Parquet? We'd have a schema even if there are no rows. Same with JDBC. Should we return this schema, even if the data is empty?
    
    Finally, do we need special handling for "null" files: a file that no longer exists on disk and so has a completely undefined schema? An undefined schema is different than an empty schema. Undefined = "we just don't know what the schema might be." Empty = "we know there is a schema and that schema is empty." A missing CSV file has an undefined schema. We could argue that a 0-length (or, a one-byte file with the only content being a newline) is a valid file, but it is a file with no columns, hence an empty schema.
    
    Or, should we have a rule that says something like:
    
    1. If a file is missing, or has an undefined schema, skip it.
    2. If a file has a schema (empty or not) and no rows, then:
      * If this is the first file, remember the schema and skip to the next.
      * If this file follows a non-empty file, trigger a schema change.
    4. If a file has a rows, then:
      * If the previous schema was empty, ignore that empty schema.
      * If the previous schema was non-empty, and different, return OK_NEW_SCHEMA
      * If the previous schema was the same, return OK.
    5. At EOF:
      * If the previous schema was empty, and the previous file had no rows, return that schema.
      * If the previous schema was empty, or no non-empty files, return NONE.
    
    We can reduce these to a simpler set of rules:
    
    * We ignore all empty or missing schemas.
    * We ignore empty files unless they are the only file(s) or they have a schema different than the next non-empty file.
    
    Other rules are possible. One is simply to ignore all 0-length files regardless of whether the schema is the same or different than surrounding files. If all files return 0 records, just return NONE. (These are the rules adopted, as of now, by the new scan operator; though we will change them to match the rules here one we pin them down.)


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] drill pull request #906: DRILL-5546: Handle schema change exception failure ...

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/906#discussion_r136440319
  
    --- Diff: exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/union/UnionAllRecordBatch.java ---
    @@ -130,562 +145,248 @@ public IterOutcome innerNext() {
       }
     
       @Override
    -  public WritableBatch getWritableBatch() {
    -    return WritableBatch.get(this);
    +  public int getRecordCount() {
    +    return recordCount;
       }
     
    -  private void setValueCount(int count) {
    -    for (ValueVector v : allocationVectors) {
    -      ValueVector.Mutator m = v.getMutator();
    -      m.setValueCount(count);
    -    }
    -  }
     
    -  private boolean doAlloc() {
    -    for (ValueVector v : allocationVectors) {
    -      try {
    -        AllocationHelper.allocateNew(v, current.getRecordCount());
    -      } catch (OutOfMemoryException ex) {
    -        return false;
    -      }
    +  @SuppressWarnings("resource")
    +  private IterOutcome doWork(RecordBatch inputBatch, boolean newSchema) throws ClassTransformationException, IOException, SchemaChangeException {
    +    if (inputBatch.getSchema().getFieldCount() != container.getSchema().getFieldCount()) {
    +      // wrong.
         }
    -    return true;
    -  }
     
    -  @SuppressWarnings("resource")
    -  private IterOutcome doWork() throws ClassTransformationException, IOException, SchemaChangeException {
    -    if (allocationVectors != null) {
    -      for (ValueVector v : allocationVectors) {
    -        v.clear();
    -      }
    +    if (newSchema) {
    +      createUnionAller(inputBatch);
         }
     
    -    allocationVectors = Lists.newArrayList();
    -    transfers.clear();
    +    container.zeroVectors();
     
    -    // If both sides of Union-All are empty
    -    if(unionAllInput.isBothSideEmpty()) {
    -      for(int i = 0; i < outputFields.size(); ++i) {
    -        final String colName = outputFields.get(i).getPath();
    -        final MajorType majorType = MajorType.newBuilder()
    -            .setMinorType(MinorType.INT)
    -            .setMode(DataMode.OPTIONAL)
    -            .build();
    -
    -        MaterializedField outputField = MaterializedField.create(colName, majorType);
    -        ValueVector vv = container.addOrGet(outputField, callBack);
    -        allocationVectors.add(vv);
    -      }
    +    for (final ValueVector v : this.allocationVectors) {
    +      AllocationHelper.allocateNew(v, inputBatch.getRecordCount());
    +    }
     
    -      container.buildSchema(BatchSchema.SelectionVectorMode.NONE);
    +    recordCount = unionall.unionRecords(0, inputBatch.getRecordCount(), 0);
    +    for (final ValueVector v : allocationVectors) {
    +      final ValueVector.Mutator m = v.getMutator();
    +      m.setValueCount(recordCount);
    +    }
    +
    +    if (callBack.getSchemaChangedAndReset()) {
           return IterOutcome.OK_NEW_SCHEMA;
    +    } else {
    +      return IterOutcome.OK;
         }
    +  }
    +
    +  private void createUnionAller(RecordBatch inputBatch) throws ClassTransformationException, IOException, SchemaChangeException {
    +    transfers.clear();
    +    allocationVectors.clear();;
     
         final ClassGenerator<UnionAller> cg = CodeGenerator.getRoot(UnionAller.TEMPLATE_DEFINITION, context.getFunctionRegistry(), context.getOptions());
         cg.getCodeGenerator().plainJavaCapable(true);
         // Uncomment out this line to debug the generated code.
    -//    cg.getCodeGenerator().saveCodeForDebugging(true);
    +    //    cg.getCodeGenerator().saveCodeForDebugging(true);
    +
         int index = 0;
    -    for(VectorWrapper<?> vw : current) {
    -       ValueVector vvIn = vw.getValueVector();
    -      // get the original input column names
    -      SchemaPath inputPath = SchemaPath.getSimplePath(vvIn.getField().getPath());
    -      // get the renamed column names
    -      SchemaPath outputPath = SchemaPath.getSimplePath(outputFields.get(index).getPath());
    +    for(VectorWrapper<?> vw : inputBatch) {
    +      ValueVector vvIn = vw.getValueVector();
    +      ValueVector vvOut = container.getValueVector(index).getValueVector();
     
           final ErrorCollector collector = new ErrorCollectorImpl();
           // According to input data names, Minortypes, Datamodes, choose to
           // transfer directly,
           // rename columns or
           // cast data types (Minortype or DataMode)
    -      if (hasSameTypeAndMode(outputFields.get(index), vw.getValueVector().getField())) {
    +      if (hasSameTypeAndMode(container.getSchema().getColumn(index), vvIn.getField())
    +          && vvIn.getField().getType().getMinorType() != TypeProtos.MinorType.MAP // Per DRILL-5521, existing bug for map transfer
    +          ) {
             // Transfer column
    -
    -        MajorType outputFieldType = outputFields.get(index).getType();
    -        MaterializedField outputField = MaterializedField.create(outputPath.getAsUnescapedPath(), outputFieldType);
    -
    -        /*
    -          todo: Fix if condition when DRILL-4824 is merged
    -          If condition should be changed to:
    -          `if (outputFields.get(index).getPath().equals(inputPath.getAsUnescapedPath())) {`
    -          DRILL-5419 has changed condition to correct one but this caused regression (DRILL-5521).
    -          Root cause is missing indication of child column in map types when it is null.
    -          DRILL-4824 is re-working json reader implementation, including map types and will fix this problem.
    -          Reverting condition to previous one to avoid regression till DRILL-4824 is merged.
    -          Unit test - TestJsonReader.testKvgenWithUnionAll().
    -         */
    -        if (outputFields.get(index).getPath().equals(inputPath)) {
    -          ValueVector vvOut = container.addOrGet(outputField);
    -          TransferPair tp = vvIn.makeTransferPair(vvOut);
    -          transfers.add(tp);
    +        TransferPair tp = vvIn.makeTransferPair(vvOut);
    +        transfers.add(tp);
             // Copy data in order to rename the column
    -        } else {
    -          final LogicalExpression expr = ExpressionTreeMaterializer.materialize(inputPath, current, collector, context.getFunctionRegistry() );
    -          if (collector.hasErrors()) {
    -            throw new SchemaChangeException(String.format("Failure while trying to materialize incoming schema.  Errors:\n %s.", collector.toErrorString()));
    -          }
    -
    -          ValueVector vv = container.addOrGet(outputField, callBack);
    -          allocationVectors.add(vv);
    -          TypedFieldId fid = container.getValueVectorId(SchemaPath.getSimplePath(outputField.getPath()));
    -          ValueVectorWriteExpression write = new ValueVectorWriteExpression(fid, expr, true);
    -          cg.addExpr(write);
    -        }
    -      // Cast is necessary
    +      } else if (vvIn.getField().getType().getMinorType() == TypeProtos.MinorType.NULL) {
    +        continue;
           } else {
    -        LogicalExpression expr = ExpressionTreeMaterializer.materialize(inputPath, current, collector, context.getFunctionRegistry());
    +        SchemaPath inputPath = SchemaPath.getSimplePath(vvIn.getField().getPath());
    +        MaterializedField inField = vvIn.getField();
    +        MaterializedField outputField = vvOut.getField();
    +
    +        LogicalExpression expr = ExpressionTreeMaterializer.materialize(inputPath, inputBatch, collector, context.getFunctionRegistry());
    +
             if (collector.hasErrors()) {
               throw new SchemaChangeException(String.format("Failure while trying to materialize incoming schema.  Errors:\n %s.", collector.toErrorString()));
             }
     
             // If the inputs' DataMode is required and the outputs' DataMode is not required
             // cast to the one with the least restriction
    -        if(vvIn.getField().getType().getMode() == DataMode.REQUIRED
    -            && outputFields.get(index).getType().getMode() != DataMode.REQUIRED) {
    -          expr = ExpressionTreeMaterializer.convertToNullableType(expr, vvIn.getField().getType().getMinorType(), context.getFunctionRegistry(), collector);
    +        if(inField.getType().getMode() == TypeProtos.DataMode.REQUIRED
    +            && outputField.getType().getMode() != TypeProtos.DataMode.REQUIRED) {
    +          expr = ExpressionTreeMaterializer.convertToNullableType(expr, inField.getType().getMinorType(), context.getFunctionRegistry(), collector);
               if (collector.hasErrors()) {
                 throw new SchemaChangeException(String.format("Failure while trying to materialize incoming schema.  Errors:\n %s.", collector.toErrorString()));
               }
             }
     
             // If two inputs' MinorTypes are different,
             // Insert a cast before the Union operation
    -        if(vvIn.getField().getType().getMinorType() != outputFields.get(index).getType().getMinorType()) {
    -          expr = ExpressionTreeMaterializer.addCastExpression(expr, outputFields.get(index).getType(), context.getFunctionRegistry(), collector);
    +        if(inField.getType().getMinorType() != outputField.getType().getMinorType()) {
    +          expr = ExpressionTreeMaterializer.addCastExpression(expr, outputField.getType(), context.getFunctionRegistry(), collector);
               if (collector.hasErrors()) {
                 throw new SchemaChangeException(String.format("Failure while trying to materialize incoming schema.  Errors:\n %s.", collector.toErrorString()));
               }
             }
     
    -        final MaterializedField outputField = MaterializedField.create(outputPath.getAsUnescapedPath(), expr.getMajorType());
    -        ValueVector vector = container.addOrGet(outputField, callBack);
    -        allocationVectors.add(vector);
             TypedFieldId fid = container.getValueVectorId(SchemaPath.getSimplePath(outputField.getPath()));
     
    -        boolean useSetSafe = !(vector instanceof FixedWidthVector);
    +        boolean useSetSafe = !(vvOut instanceof FixedWidthVector);
             ValueVectorWriteExpression write = new ValueVectorWriteExpression(fid, expr, useSetSafe);
             cg.addExpr(write);
    +
    +        allocationVectors.add(vvOut);
           }
           ++index;
         }
     
         unionall = context.getImplementationClass(cg.getCodeGenerator());
    -    unionall.setup(context, current, this, transfers);
    -
    -    if(!schemaAvailable) {
    -      container.buildSchema(BatchSchema.SelectionVectorMode.NONE);
    -      schemaAvailable = true;
    -    }
    -
    -    if(!doAlloc()) {
    -      return IterOutcome.OUT_OF_MEMORY;
    -    }
    -
    -    recordCount = unionall.unionRecords(0, current.getRecordCount(), 0);
    -    setValueCount(recordCount);
    -    return IterOutcome.OK;
    -  }
    -
    -  public static boolean hasSameTypeAndMode(MaterializedField leftField, MaterializedField rightField) {
    -    return (leftField.getType().getMinorType() == rightField.getType().getMinorType())
    -        && (leftField.getType().getMode() == rightField.getType().getMode());
    -  }
    -
    -  // This method is used by inner class to point the reference `current` to the correct record batch
    -  private void setCurrentRecordBatch(RecordBatch target) {
    -    this.current = target;
    -  }
    -
    -  // This method is used by inner class to clear the current record batch
    -  private void clearCurrentRecordBatch() {
    -    for(VectorWrapper<?> v: current) {
    -      v.clear();
    -    }
    +    unionall.setup(context, inputBatch, this, transfers);
       }
     
    -  public static class UnionAllInput {
    -    private UnionAllRecordBatch unionAllRecordBatch;
    -    private List<MaterializedField> outputFields;
    -    private OneSideInput leftSide;
    -    private OneSideInput rightSide;
    -    private IterOutcome upstream = IterOutcome.NOT_YET;
    -    private boolean leftIsFinish = false;
    -    private boolean rightIsFinish = false;
    -
    -    // These two schemas are obtained from the first record batches of the left and right inputs
    -    // They are used to check if the schema is changed between recordbatches
    -    private BatchSchema leftSchema;
    -    private BatchSchema rightSchema;
    -    private boolean bothEmpty = false;
    -
    -    public UnionAllInput(UnionAllRecordBatch unionAllRecordBatch, RecordBatch left, RecordBatch right) {
    -      this.unionAllRecordBatch = unionAllRecordBatch;
    -      leftSide = new OneSideInput(left);
    -      rightSide = new OneSideInput(right);
    -    }
    -
    -    private void setBothSideEmpty(boolean bothEmpty) {
    -      this.bothEmpty = bothEmpty;
    -    }
    -
    -    private boolean isBothSideEmpty() {
    -      return bothEmpty;
    -    }
    -
    -    public IterOutcome nextBatch() throws SchemaChangeException {
    -      if(upstream == RecordBatch.IterOutcome.NOT_YET) {
    -        IterOutcome iterLeft = leftSide.nextBatch();
    -        switch(iterLeft) {
    -          case OK_NEW_SCHEMA:
    -            /*
    -             * If the first few record batches are all empty,
    -             * there is no way to tell whether these empty batches are coming from empty files.
    -             * It is incorrect to infer output types when either side could be coming from empty.
    -             *
    -             * Thus, while-loop is necessary to skip those empty batches.
    -             */
    -            whileLoop:
    -            while(leftSide.getRecordBatch().getRecordCount() == 0) {
    -              iterLeft = leftSide.nextBatch();
    -
    -              switch(iterLeft) {
    -                case STOP:
    -                case OUT_OF_MEMORY:
    -                  return iterLeft;
    -
    -                case NONE:
    -                  // Special Case: The left side was an empty input.
    -                  leftIsFinish = true;
    -                  break whileLoop;
    -
    -                case NOT_YET:
    -                case OK_NEW_SCHEMA:
    -                case OK:
    -                  continue whileLoop;
    -
    -                default:
    -                  throw new IllegalStateException(
    -                      String.format("Unexpected state %s.", iterLeft));
    -              }
    -            }
    -
    -            break;
    -          case STOP:
    -          case OUT_OF_MEMORY:
    -            return iterLeft;
    -
    -          default:
    -            throw new IllegalStateException(
    -                String.format("Unexpected state %s.", iterLeft));
    -        }
    -
    -        IterOutcome iterRight = rightSide.nextBatch();
    -        switch(iterRight) {
    -          case OK_NEW_SCHEMA:
    -            // Unless there is no record batch on the left side of the inputs,
    -            // always start processing from the left side.
    -            if(leftIsFinish) {
    -              unionAllRecordBatch.setCurrentRecordBatch(rightSide.getRecordBatch());
    -            } else {
    -              unionAllRecordBatch.setCurrentRecordBatch(leftSide.getRecordBatch());
    -            }
    -            // If the record count of the first batch from right input is zero,
    -            // there are two possibilities:
    -            // 1. The right side is an empty input (e.g., file).
    -            // 2. There will be more records carried by later batches.
    -
    -            /*
    -             * If the first few record batches are all empty,
    -             * there is no way to tell whether these empty batches are coming from empty files.
    -             * It is incorrect to infer output types when either side could be coming from empty.
    -             *
    -             * Thus, while-loop is necessary to skip those empty batches.
    -             */
    -            whileLoop:
    -            while(rightSide.getRecordBatch().getRecordCount() == 0) {
    -              iterRight = rightSide.nextBatch();
    -              switch(iterRight) {
    -                case STOP:
    -                case OUT_OF_MEMORY:
    -                  return iterRight;
    -
    -                case NONE:
    -                  // Special Case: The right side was an empty input.
    -                  rightIsFinish = true;
    -                  break whileLoop;
    -
    -                case NOT_YET:
    -                case OK_NEW_SCHEMA:
    -                case OK:
    -                  continue whileLoop;
    -
    -                default:
    -                  throw new IllegalStateException(
    -                      String.format("Unexpected state %s.", iterRight));
    -              }
    -            }
    -
    -            if(leftIsFinish && rightIsFinish) {
    -              setBothSideEmpty(true);
    -            }
    -
    -            inferOutputFields();
    -            break;
    -
    -          case STOP:
    -          case OUT_OF_MEMORY:
    -            return iterRight;
    -
    -          default:
    -            throw new IllegalStateException(
    -                String.format("Unexpected state %s.", iterRight));
    -        }
    -
    -
    -
    -        upstream = IterOutcome.OK_NEW_SCHEMA;
    -        return upstream;
    -      } else {
    -        if(isBothSideEmpty()) {
    -          return IterOutcome.NONE;
    -        }
    -
    -        unionAllRecordBatch.clearCurrentRecordBatch();
    -
    -        if(leftIsFinish && rightIsFinish) {
    -          upstream = IterOutcome.NONE;
    -          return upstream;
    -        } else if(leftIsFinish) {
    -          IterOutcome iterOutcome = rightSide.nextBatch();
    -
    -          switch(iterOutcome) {
    -            case NONE:
    -              rightIsFinish = true;
    -              // fall through
    -            case STOP:
    -            case OUT_OF_MEMORY:
    -              upstream = iterOutcome;
    -              return upstream;
    -
    -            case OK_NEW_SCHEMA:
    -              if(!rightSide.getRecordBatch().getSchema().equals(rightSchema)) {
    -                throw new SchemaChangeException("Schema change detected in the right input of Union-All. This is not currently supported");
    -              }
    -              iterOutcome = IterOutcome.OK;
    -              // fall through
    -            case OK:
    -              unionAllRecordBatch.setCurrentRecordBatch(rightSide.getRecordBatch());
    -              upstream = iterOutcome;
    -              return upstream;
    -
    -            default:
    -              throw new IllegalStateException(String.format("Unknown state %s.", upstream));
    -          }
    -        } else if(rightIsFinish) {
    -          IterOutcome iterOutcome = leftSide.nextBatch();
    -          switch(iterOutcome) {
    -            case STOP:
    -            case OUT_OF_MEMORY:
    -            case NONE:
    -              upstream = iterOutcome;
    -              return upstream;
    -
    -            case OK:
    -              unionAllRecordBatch.setCurrentRecordBatch(leftSide.getRecordBatch());
    -              upstream = iterOutcome;
    -              return upstream;
    -
    -            default:
    -              throw new IllegalStateException(String.format("Unknown state %s.", iterOutcome));
    -          }
    -        } else {
    -          IterOutcome iterOutcome = leftSide.nextBatch();
    -
    -          switch(iterOutcome) {
    -            case STOP:
    -            case OUT_OF_MEMORY:
    -              upstream = iterOutcome;
    -              return upstream;
    -
    -            case OK_NEW_SCHEMA:
    -              if(!leftSide.getRecordBatch().getSchema().equals(leftSchema)) {
    -                throw new SchemaChangeException("Schema change detected in the left input of Union-All. This is not currently supported");
    -              }
    -
    -              iterOutcome = IterOutcome.OK;
    -              // fall through
    -            case OK:
    -              unionAllRecordBatch.setCurrentRecordBatch(leftSide.getRecordBatch());
    -              upstream = iterOutcome;
    -              return upstream;
    -
    -            case NONE:
    -              unionAllRecordBatch.setCurrentRecordBatch(rightSide.getRecordBatch());
    -              upstream = IterOutcome.OK;
    -              leftIsFinish = true;
    -              return upstream;
    -
    -            default:
    -              throw new IllegalStateException(String.format("Unknown state %s.", upstream));
    -          }
    -        }
    -      }
    -    }
     
    -    /**
    -     *
    -     * Summarize the inference in the four different situations:
    -     * First of all, the field names are always determined by the left side
    -     * (Even when the left side is from an empty file, we have the column names.)
    -     *
    -     * Cases:
    -     * 1. Left: non-empty; Right: non-empty
    -     *      types determined by both sides with implicit casting involved
    -     * 2. Left: empty; Right: non-empty
    -     *      type from the right
    -     * 3. Left: non-empty; Right: empty
    -     *      types from the left
    -     * 4. Left: empty; Right: empty
    -     *      types are nullable integer
    -     */
    -    private void inferOutputFields() {
    -      if(!leftIsFinish && !rightIsFinish) {
    -        // Both sides are non-empty
    -        inferOutputFieldsBothSide();
    -      } else if(!rightIsFinish) {
    -        // Left side is non-empty
    -        // While use left side's column names as output column names,
    -        // use right side's column types as output column types.
    -        inferOutputFieldsFromSingleSide(
    -            leftSide.getRecordBatch().getSchema(),
    -            rightSide.getRecordBatch().getSchema());
    +  // The output table's column names always follow the left table,
    +  // where the output type is chosen based on DRILL's implicit casting rules
    +  private void inferOutputFieldsBothSide(final BatchSchema leftSchema, final BatchSchema rightSchema) {
    +//    outputFields = Lists.newArrayList();
    +    final Iterator<MaterializedField> leftIter = leftSchema.iterator();
    +    final Iterator<MaterializedField> rightIter = rightSchema.iterator();
    +
    +    int index = 1;
    +    while (leftIter.hasNext() && rightIter.hasNext()) {
    +      MaterializedField leftField  = leftIter.next();
    +      MaterializedField rightField = rightIter.next();
    +
    +      if (hasSameTypeAndMode(leftField, rightField)) {
    +        TypeProtos.MajorType.Builder builder = TypeProtos.MajorType.newBuilder().setMinorType(leftField.getType().getMinorType()).setMode(leftField.getDataMode());
    +        builder = Types.calculateTypePrecisionAndScale(leftField.getType(), rightField.getType(), builder);
    +        container.addOrGet(MaterializedField.create(leftField.getPath(), builder.build()), callBack);
    +      } else if (Types.isUntypedNull(rightField.getType())) {
    +        container.addOrGet(leftField, callBack);
    +      } else if (Types.isUntypedNull(leftField.getType())) {
    +        container.addOrGet(MaterializedField.create(leftField.getPath(), rightField.getType()), callBack);
           } else {
    -        // Either right side is empty or both are empty
    -        // Using left side's schema is sufficient
    -        inferOutputFieldsFromSingleSide(
    -            leftSide.getRecordBatch().getSchema(),
    -            leftSide.getRecordBatch().getSchema());
    -      }
    -    }
    -
    -    // The output table's column names always follow the left table,
    -    // where the output type is chosen based on DRILL's implicit casting rules
    -    private void inferOutputFieldsBothSide() {
    -      outputFields = Lists.newArrayList();
    -      leftSchema = leftSide.getRecordBatch().getSchema();
    -      rightSchema = rightSide.getRecordBatch().getSchema();
    -      Iterator<MaterializedField> leftIter = leftSchema.iterator();
    -      Iterator<MaterializedField> rightIter = rightSchema.iterator();
    -
    -      int index = 1;
    -      while (leftIter.hasNext() && rightIter.hasNext()) {
    -        MaterializedField leftField  = leftIter.next();
    -        MaterializedField rightField = rightIter.next();
    -
    -        if (hasSameTypeAndMode(leftField, rightField)) {
    -          MajorType.Builder builder = MajorType.newBuilder().setMinorType(leftField.getType().getMinorType()).setMode(leftField.getDataMode());
    +        // If the output type is not the same,
    +        // cast the column of one of the table to a data type which is the Least Restrictive
    +        TypeProtos.MajorType.Builder builder = TypeProtos.MajorType.newBuilder();
    +        if (leftField.getType().getMinorType() == rightField.getType().getMinorType()) {
    +          builder.setMinorType(leftField.getType().getMinorType());
               builder = Types.calculateTypePrecisionAndScale(leftField.getType(), rightField.getType(), builder);
    -          outputFields.add(MaterializedField.create(leftField.getPath(), builder.build()));
             } else {
    -          // If the output type is not the same,
    -          // cast the column of one of the table to a data type which is the Least Restrictive
    -          MajorType.Builder builder = MajorType.newBuilder();
    -          if (leftField.getType().getMinorType() == rightField.getType().getMinorType()) {
    -            builder.setMinorType(leftField.getType().getMinorType());
    -            builder = Types.calculateTypePrecisionAndScale(leftField.getType(), rightField.getType(), builder);
    -          } else {
    -            List<MinorType> types = Lists.newLinkedList();
    -            types.add(leftField.getType().getMinorType());
    -            types.add(rightField.getType().getMinorType());
    -            MinorType outputMinorType = TypeCastRules.getLeastRestrictiveType(types);
    -            if (outputMinorType == null) {
    -              throw new DrillRuntimeException("Type mismatch between " + leftField.getType().getMinorType().toString() +
    -                  " on the left side and " + rightField.getType().getMinorType().toString() +
    -                  " on the right side in column " + index + " of UNION ALL");
    -            }
    -            builder.setMinorType(outputMinorType);
    +          List<TypeProtos.MinorType> types = Lists.newLinkedList();
    +          types.add(leftField.getType().getMinorType());
    +          types.add(rightField.getType().getMinorType());
    +          TypeProtos.MinorType outputMinorType = TypeCastRules.getLeastRestrictiveType(types);
    +          if (outputMinorType == null) {
    +            throw new DrillRuntimeException("Type mismatch between " + leftField.getType().getMinorType().toString() +
    +                " on the left side and " + rightField.getType().getMinorType().toString() +
    +                " on the right side in column " + index + " of UNION ALL");
               }
    -
    -          // The output data mode should be as flexible as the more flexible one from the two input tables
    -          List<DataMode> dataModes = Lists.newLinkedList();
    -          dataModes.add(leftField.getType().getMode());
    -          dataModes.add(rightField.getType().getMode());
    -          builder.setMode(TypeCastRules.getLeastRestrictiveDataMode(dataModes));
    -
    -          outputFields.add(MaterializedField.create(leftField.getPath(), builder.build()));
    +          builder.setMinorType(outputMinorType);
             }
    -        ++index;
    -      }
    -
    -      assert !leftIter.hasNext() && ! rightIter.hasNext() : "Mis-match of column count should have been detected when validating sqlNode at planning";
    -    }
     
    -    private void inferOutputFieldsFromSingleSide(final BatchSchema schemaForNames, final BatchSchema schemaForTypes) {
    -      outputFields = Lists.newArrayList();
    +        // The output data mode should be as flexible as the more flexible one from the two input tables
    +        List<TypeProtos.DataMode> dataModes = Lists.newLinkedList();
    +        dataModes.add(leftField.getType().getMode());
    +        dataModes.add(rightField.getType().getMode());
    +        builder.setMode(TypeCastRules.getLeastRestrictiveDataMode(dataModes));
     
    -      final List<String> outputColumnNames = Lists.newArrayList();
    -      final Iterator<MaterializedField> iterForNames = schemaForNames.iterator();
    -      while(iterForNames.hasNext()) {
    -        outputColumnNames.add(iterForNames.next().getPath());
    -      }
    -
    -      final Iterator<MaterializedField> iterForTypes = schemaForTypes.iterator();
    -      for(int i = 0; iterForTypes.hasNext(); ++i) {
    -        MaterializedField field = iterForTypes.next();
    -        outputFields.add(MaterializedField.create(outputColumnNames.get(i), field.getType()));
    +        container.addOrGet(MaterializedField.create(leftField.getPath(), builder.build()), callBack);
           }
    +      ++index;
         }
     
    -    public List<MaterializedField> getOutputFields() {
    -      if(outputFields == null) {
    -        throw new NullPointerException("Output fields have not been inferred");
    -      }
    -
    -      return outputFields;
    -    }
    +    assert !leftIter.hasNext() && ! rightIter.hasNext() : "Mis-match of column count should have been detected when validating sqlNode at planning";
    --- End diff --
    
    Thanks for the explanation.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] drill pull request #906: DRILL-5546: Handle schema change exception failure ...

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

    https://github.com/apache/drill/pull/906#discussion_r136185442
  
    --- Diff: exec/java-exec/src/main/java/org/apache/drill/exec/record/AbstractSingleRecordBatch.java ---
    @@ -125,4 +131,19 @@ public BatchSchema getSchema() {
     
       protected abstract boolean setupNewSchema() throws SchemaChangeException;
       protected abstract IterOutcome doWork();
    +
    +  /**
    +   * Default behavior to handle fast NONE (incoming's first next() return NONE, in stead of OK_NEW_SCHEMA):
    +   * FAST NONE could happen when the underneath Scan operators do not produce any batch with schema.
    +   *
    --- End diff --
    
    done


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] drill pull request #906: DRILL-5546: Handle schema change exception failure ...

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

    https://github.com/apache/drill/pull/906#discussion_r135382803
  
    --- Diff: exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/ScanBatch.java ---
    @@ -152,97 +157,75 @@ public void kill(boolean sendUpstream) {
         }
       }
     
    -  private void releaseAssets() {
    -    container.zeroVectors();
    -  }
    -
    -  private void clearFieldVectorMap() {
    -    for (final ValueVector v : mutator.fieldVectorMap().values()) {
    -      v.clear();
    -    }
    -  }
    -
       @Override
       public IterOutcome next() {
         if (done) {
           return IterOutcome.NONE;
         }
         oContext.getStats().startProcessing();
         try {
    -      try {
    -        injector.injectChecked(context.getExecutionControls(), "next-allocate", OutOfMemoryException.class);
    -
    -        currentReader.allocate(mutator.fieldVectorMap());
    -      } catch (OutOfMemoryException e) {
    -        clearFieldVectorMap();
    -        throw UserException.memoryError(e).build(logger);
    -      }
    -      while ((recordCount = currentReader.next()) == 0) {
    +      while (true) {
             try {
    -          if (!readers.hasNext()) {
    -            // We're on the last reader, and it has no (more) rows.
    -            currentReader.close();
    -            releaseAssets();
    -            done = true;  // have any future call to next() return NONE
    -
    -            if (mutator.isNewSchema()) {
    -              // This last reader has a new schema (e.g., we have a zero-row
    -              // file or other source).  (Note that some sources have a non-
    -              // null/non-trivial schema even when there are no rows.)
    +          injector.injectChecked(context.getExecutionControls(), "next-allocate", OutOfMemoryException.class);
    --- End diff --
    
    This patch tries to decouple the logic of record reader and scanbatch:
     - Record reader is responsible to add vectors to batch (via Mutator), and populate data
     - ScanBatch is responsible to interpret the output of record reader, by checking rowCount && Mutator.isNewSchema() to decide whether return OK_NEW_SCHEMA, OK, or NONE. 
    
    > What happens on the first reader? There is no schema, so any schema is a new schema. Suppose the file is JSON and the schema is built on the fly. Does the code handle the case that we have no schema (first reader), and that reader adds no columns?
    
    It's not true "any schema is a new schema". If the first reader has no schema and adds no columns, then Mutator.isNewSchema() should return false. Mutator.isNewSchema() returns true only after the last call, one or more happens
    
    - a new top level field is added, 
    - a field in a nested field is added, 
    - an existing field type is changed
    
    You may argue a more appropriate way to represent an empty JSON file is an empty schema. However, such idea would lead to various schema conflicts in down-stream operator, if other scan thread has non-empty JSON files. This is exactly what happened before this patch. 
    
    The proposal in this patch is to **ignore** empty JSON, since 1)rowCount=0, 2)no new column were added to batch. 
     - If all the record readers for a scan thread return with rowCount = 0, and produce no new schema, then this Scan thread should return 'NONE' directly, without returning OK_NEW_SCHEMA. 
     - If at least one of reader return either with >0 row, or new schema, then Scan thread will return batch with new schema.
    - If all scan threads returns 'NONE' directly, implying the entire table does not have data/schema, this is what Project.handleFastNone() will deal with.
    
    >But, if the input is CSV, then we always have a schema. If the file has column headers, then we know that the schema is, say, (a, b, c) because those are the headers. Or, if the file has no headers, the schema is always the columns array. So, should we send that schema downstream? If so, should it include the implicit columns?
    
    If CSV always adds columns (either _a,b,c, or columns_), then ScanBatch will produce a batch with (a, b, c), or columns. It does not make sense to ignore those schema.  
      - In the case of file with header,  file with _a,b,c_ will lead to a batch with (a,b,c) while a file with _a,b,c,d_ will lead to a batch with (a,b,c,d). Those two files will cause a schema change, which is expected behavior.
      - In the case of file without header, all files will produce a batch with columns, which means there would be no schema change across different files, regardless whether they have row=0, or row > 0.
      



---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] drill pull request #906: DRILL-5546: Handle schema change exception failure ...

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/906#discussion_r134287754
  
    --- Diff: exec/java-exec/src/main/java/org/apache/drill/exec/physical/config/Project.java ---
    @@ -46,14 +56,18 @@ public Project(@JsonProperty("exprs") List<NamedExpression> exprs, @JsonProperty
         return exprs;
       }
     
    +  public boolean isOutputProj() {
    --- End diff --
    
    Maybe a comment to explain the purpose of this attribute? To quote from the PR description:
    
    > Add a new flag 'outputProj' to Project operator, to indicate if Project is for the query's final output. Such Project is added by TopProjectVisitor, to handle fast NONE when all the inputs to the query are empty
    and are skipped.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] drill pull request #906: DRILL-5546: Handle schema change exception failure ...

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/906#discussion_r134299304
  
    --- Diff: exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/ScanBatch.java ---
    @@ -252,14 +235,28 @@ public IterOutcome next() {
         }
       }
     
    +  private void releaseAssets() {
    +    container.zeroVectors();
    +  }
    +
    +  private void clearFieldVectorMap() {
    +    for (final ValueVector v : mutator.fieldVectorMap().values()) {
    +      v.clear();
    +    }
    +  }
    +
    +  private void advanceNextReader() throws ExecutionSetupException {
    +    currentReader = readers.next();
    +    implicitValues = implicitColumns.hasNext() ? implicitColumns.next() : null;
    +    currentReader.setup(oContext, mutator);
    --- End diff --
    
    This seems somewhat unreliable. In the `next()` method above, we have, say, a reader with a new schema that returned 0 rows. We want to return the container, with that new schema, downstream.
    
    Before we do, we set up the next reader, passing it the mutator. Suppose the reader decides to set up its schema in the mutator? Doesn't that add noise to the signal we want to send downstream?
    
    Conversely, do readers know to *not* touch the mutator in `setup()` and instead defer schema setup to the first call to `next()`? Doesn't that make readers rather more complicated than they need to be?
    
    Of course, I could be missing something. In that case, a bit of comment to explain the protocol would be greatly appreciated!


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] drill pull request #906: DRILL-5546: Handle schema change exception failure ...

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/906#discussion_r135579508
  
    --- Diff: exec/vector/src/main/java/org/apache/drill/exec/vector/UntypedNullVector.java ---
    @@ -0,0 +1,267 @@
    +/*
    + * 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;
    +
    +
    +import com.google.common.base.Preconditions;
    +import io.netty.buffer.DrillBuf;
    +import org.apache.drill.exec.memory.BufferAllocator;
    +import org.apache.drill.exec.proto.UserBitShared.SerializedField;
    +import org.apache.drill.exec.record.MaterializedField;
    +import org.apache.drill.exec.record.TransferPair;
    +import org.apache.drill.exec.vector.complex.reader.FieldReader;
    +
    +/** UntypedNullVector is to represent a value vector with {@link org.apache.drill.common.types.MinorType.NULL}
    + *  All values in the vector represent two semantic implications: 1) the value is unknown, 2) the type is unknown.
    + *  Because of this, we only have to keep track of the number of values in value vector,
    + *  and there is no allocated buffer to back up this value vector. Therefore, the majority of
    + *  methods in this class is either no-op, or throws {@link UnsupportedOperationException}.
    + *
    + */
    +public final class UntypedNullVector extends BaseDataValueVector implements FixedWidthVector {
    +  private static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(UntypedNullVector.class);
    +
    +  /**
    +   * Width of each fixed-width value.
    +   */
    +  public static final int VALUE_WIDTH = 0;
    +
    +  private final Accessor accessor = new Accessor();
    +  private final Mutator mutator = new Mutator();
    +  private int valueCount ;
    +
    +  public UntypedNullVector(MaterializedField field, BufferAllocator allocator) {
    +    super(field, allocator);
    +    valueCount = 0;
    +  }
    +
    +  @Override
    +  public FieldReader getReader() { throw new UnsupportedOperationException(); }
    +
    +  @Override
    +  public int getBufferSizeFor(final int valueCount) {
    +    return 0;
    +  }
    +
    +  @Override
    +  public int getValueCapacity(){
    +    return Character.MAX_VALUE;
    --- End diff --
    
    Please use `ValueVector.MAX_ROW_COUNT`. It is a coincidence that the maximum row count happens to be (one greater than) `Character.MAX_VALUE`.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] drill pull request #906: DRILL-5546: Handle schema change exception failure ...

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

    https://github.com/apache/drill/pull/906#discussion_r135974905
  
    --- Diff: exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/project/ProjectRecordBatch.java ---
    @@ -768,4 +765,73 @@ else if (exprHasPrefix && refHasPrefix) {
           }
         }
       }
    +
    +  /**
    +   * handle FAST NONE specially when Project for query output. This happens when input returns a
    +   * FAST NONE directly ( input does not return any batch with schema/data).
    +   *
    +   * Project operator has to return a batch with schema derived using the following 3 rules:
    +   *  Case 1:  *  ==>  expand into an empty list of columns.
    +   *  Case 2:  regular column reference ==> treat as nullable-int column
    +   *  Case 3:  expressions => Call ExpressionTreeMaterialization over an empty vector contain.
    +   *           Once the expression is materialized without error, use the output type of materialized
    +   *           expression.
    +   * The batch is constructed with the above rules, and recordCount = 0.
    +   * Returned with OK_NEW_SCHEMA to down-stream operator.
    +   */
    +  @Override
    +  protected IterOutcome handleFastNone() {
    +    if (! popConfig.isOutputProj()) {
    +      return super.handleFastNone();
    +    }
    +
    +    allocationVectors = new ArrayList<>();
    +    final List<NamedExpression> exprs = getExpressionList();
    +    final ErrorCollector collector = new ErrorCollectorImpl();
    +    VectorContainer fakedIncomingVC = new VectorContainer();
    +
    +    for (NamedExpression namedExpression : exprs) {
    +      if (namedExpression.getExpr() instanceof SchemaPath) {
    +        final NameSegment expr = ((SchemaPath) namedExpression.getExpr()).getRootSegment();
    +        if (expr.getPath().contains(StarColumnHelper.STAR_COLUMN)) {
    --- End diff --
    
    This code has been removed in revised patch.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] drill pull request #906: DRILL-5546: Handle schema change exception failure ...

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

    https://github.com/apache/drill/pull/906#discussion_r135925006
  
    --- Diff: exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/project/ProjectRecordBatch.java ---
    @@ -768,4 +765,73 @@ else if (exprHasPrefix && refHasPrefix) {
           }
         }
       }
    +
    +  /**
    +   * handle FAST NONE specially when Project for query output. This happens when input returns a
    --- End diff --
    
    I change to "handleNullInput" from "handleFastNone". 
    
    The concept of "NullInput" captures the scenarios where input does not produce any batch, and returning NONE directly.  The concept of "empty result set" could mean the case where we have 0 row, but still we could have a batch. I think "empty result set" is not precisely what we want to cover here. 


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] drill pull request #906: DRILL-5546: Handle schema change exception failure ...

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/906#discussion_r135577493
  
    --- Diff: exec/java-exec/src/main/java/org/apache/drill/exec/record/AbstractSingleRecordBatch.java ---
    @@ -125,4 +131,19 @@ public BatchSchema getSchema() {
     
       protected abstract boolean setupNewSchema() throws SchemaChangeException;
       protected abstract IterOutcome doWork();
    +
    +  /**
    +   * Default behavior to handle fast NONE (incoming's first next() return NONE, in stead of OK_NEW_SCHEMA):
    +   * FAST NONE could happen when the underneath Scan operators do not produce any batch with schema.
    +   *
    --- End diff --
    
    Javadoc is HTML, so insert \<p>


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] drill pull request #906: DRILL-5546: Handle schema change exception failure ...

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/906#discussion_r134630528
  
    --- Diff: exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/project/ProjectRecordBatch.java ---
    @@ -768,4 +765,73 @@ else if (exprHasPrefix && refHasPrefix) {
           }
         }
       }
    +
    +  /**
    +   * handle FAST NONE specially when Project for query output. This happens when input returns a
    +   * FAST NONE directly ( input does not return any batch with schema/data).
    +   *
    +   * Project operator has to return a batch with schema derived using the following 3 rules:
    +   *  Case 1:  *  ==>  expand into an empty list of columns.
    +   *  Case 2:  regular column reference ==> treat as nullable-int column
    +   *  Case 3:  expressions => Call ExpressionTreeMaterialization over an empty vector contain.
    +   *           Once the expression is materialized without error, use the output type of materialized
    +   *           expression.
    +   * The batch is constructed with the above rules, and recordCount = 0.
    +   * Returned with OK_NEW_SCHEMA to down-stream operator.
    +   */
    +  @Override
    +  protected IterOutcome handleFastNone() {
    +    if (! popConfig.isOutputProj()) {
    +      return super.handleFastNone();
    +    }
    +
    +    allocationVectors = new ArrayList<>();
    +    final List<NamedExpression> exprs = getExpressionList();
    +    final ErrorCollector collector = new ErrorCollectorImpl();
    +    VectorContainer fakedIncomingVC = new VectorContainer();
    +
    +    for (NamedExpression namedExpression : exprs) {
    +      if (namedExpression.getExpr() instanceof SchemaPath) {
    +        final NameSegment expr = ((SchemaPath) namedExpression.getExpr()).getRootSegment();
    +        if (expr.getPath().contains(StarColumnHelper.STAR_COLUMN)) {
    --- End diff --
    
    `expr.getPath().contains(StarColumnHelper.STAR_COLUMN)` --> `isWildcard(expr)`?


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] drill pull request #906: DRILL-5546: Handle schema change exception failure ...

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/906#discussion_r135368337
  
    --- Diff: exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/union/UnionAllRecordBatch.java ---
    @@ -39,88 +35,107 @@
     import org.apache.drill.exec.expr.ValueVectorWriteExpression;
     import org.apache.drill.exec.ops.FragmentContext;
     import org.apache.drill.exec.physical.config.UnionAll;
    -import org.apache.drill.exec.record.AbstractRecordBatch;
    +import org.apache.drill.exec.record.AbstractBinaryRecordBatch;
     import org.apache.drill.exec.record.BatchSchema;
     import org.apache.drill.exec.record.MaterializedField;
     import org.apache.drill.exec.record.RecordBatch;
     import org.apache.drill.exec.record.TransferPair;
     import org.apache.drill.exec.record.TypedFieldId;
     import org.apache.drill.exec.record.VectorWrapper;
    -import org.apache.drill.exec.record.WritableBatch;
    -import org.apache.drill.exec.record.selection.SelectionVector2;
    -import org.apache.drill.exec.record.selection.SelectionVector4;
     import org.apache.drill.exec.resolver.TypeCastRules;
     import org.apache.drill.exec.vector.AllocationHelper;
     import org.apache.drill.exec.vector.FixedWidthVector;
     import org.apache.drill.exec.vector.SchemaChangeCallBack;
     import org.apache.drill.exec.vector.ValueVector;
     
    -import com.google.common.collect.Lists;
    +import java.io.IOException;
    +import java.util.Iterator;
    +import java.util.List;
    +import java.util.NoSuchElementException;
    +import java.util.Stack;
     
    -public class UnionAllRecordBatch extends AbstractRecordBatch<UnionAll> {
    +public class UnionAllRecordBatch extends AbstractBinaryRecordBatch<UnionAll> {
       private static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(UnionAllRecordBatch.class);
     
    -  private List<MaterializedField> outputFields;
    +  private SchemaChangeCallBack callBack = new SchemaChangeCallBack();
       private UnionAller unionall;
    -  private UnionAllInput unionAllInput;
    -  private RecordBatch current;
    -
       private final List<TransferPair> transfers = Lists.newArrayList();
    -  private List<ValueVector> allocationVectors;
    -  protected SchemaChangeCallBack callBack = new SchemaChangeCallBack();
    +  private List<ValueVector> allocationVectors = Lists.newArrayList();
       private int recordCount = 0;
    -  private boolean schemaAvailable = false;
    +  private UnionInputIterator unionInputIterator;
     
       public UnionAllRecordBatch(UnionAll config, List<RecordBatch> children, FragmentContext context) throws OutOfMemoryException {
    -    super(config, context, false);
    -    assert (children.size() == 2) : "The number of the operands of Union must be 2";
    -    unionAllInput = new UnionAllInput(this, children.get(0), children.get(1));
    -  }
    -
    -  @Override
    -  public int getRecordCount() {
    -    return recordCount;
    +    super(config, context, true, children.get(0), children.get(1));
       }
     
       @Override
       protected void killIncoming(boolean sendUpstream) {
    -    unionAllInput.getLeftRecordBatch().kill(sendUpstream);
    -    unionAllInput.getRightRecordBatch().kill(sendUpstream);
    +    left.kill(sendUpstream);
    +    right.kill(sendUpstream);
       }
     
    -  @Override
    -  public SelectionVector2 getSelectionVector2() {
    -    throw new UnsupportedOperationException("UnionAllRecordBatch does not support selection vector");
    -  }
    +  protected void buildSchema() throws SchemaChangeException {
    +    if (! prefetchFirstBatchFromBothSides()) {
    +      return;
    +    }
     
    -  @Override
    -  public SelectionVector4 getSelectionVector4() {
    -    throw new UnsupportedOperationException("UnionAllRecordBatch does not support selection vector");
    +    unionInputIterator = new UnionInputIterator(leftUpstream, left, rightUpstream, right);
    +
    +    if (leftUpstream == IterOutcome.NONE && rightUpstream == IterOutcome.OK_NEW_SCHEMA) {
    +      inferOutputFieldsOneSide(right.getSchema());
    +    } else if (rightUpstream == IterOutcome.NONE && leftUpstream == IterOutcome.OK_NEW_SCHEMA) {
    +      inferOutputFieldsOneSide((left.getSchema()));
    +    } else if (leftUpstream == IterOutcome.OK_NEW_SCHEMA && rightUpstream == IterOutcome.OK_NEW_SCHEMA) {
    +      inferOutputFieldsBothSide(left.getSchema(), right.getSchema());
    +    }
    +
    +    container.buildSchema(BatchSchema.SelectionVectorMode.NONE);
    +
    +    for (VectorWrapper vv: container) {
    +      vv.getValueVector().allocateNew();
    +      vv.getValueVector().getMutator().setValueCount(0);
    +    }
       }
     
       @Override
       public IterOutcome innerNext() {
         try {
    -      IterOutcome upstream = unionAllInput.nextBatch();
    -      logger.debug("Upstream of Union-All: {}", upstream);
    +      if (!unionInputIterator.hasNext()) {
    +        return IterOutcome.NONE;
    +      }
    +
    +      Pair<IterOutcome, RecordBatch> nextBatch = unionInputIterator.next();
    +
    +      IterOutcome upstream = nextBatch.left;
    +      RecordBatch incoming = nextBatch.right;
    +
    +      // skip batches with same schema as the previous one yet having 0 row.
    +      if (upstream == IterOutcome.OK && incoming.getRecordCount() == 0) {
    +        do {
    +          for (final VectorWrapper<?> w : incoming) {
    +            w.clear();
    +          }
    +          if (!unionInputIterator.hasNext()) {
    +            return IterOutcome.NONE;
    +          }
    +          nextBatch = unionInputIterator.next();
    +          upstream = nextBatch.left;
    +          incoming = nextBatch.right;
    +        } while ((upstream == IterOutcome.OK) &&
    +                incoming.getRecordCount() == 0);
    +      }
    +
    --- End diff --
    
    Loop does not handle `STOP` or `OOM` conditions.
    
    The loop can be compressed to something like:
    ```
          IterOutcome upstream;
          RecordBatch incoming;
    
    for (;;) {
          // Another input batch?
          if (!unionInputIterator.hasNext()) {
            return IterOutcome.NONE;
          }
    
          IterOutcome upstream = nextBatch.left;
          RecordBatch incoming = nextBatch.right;
          // Handle error conditions
          switch(upstream) {
            case NONE: // Can this actually occur?
            case OUT_OF_MEMORY:
            case STOP:
              return upstream;
          }
    
          // skip batches with same schema as the previous one yet having 0 row.
          if (upstream == IterOutcome.OK && incoming.getRecordCount() == 0) {
              VectorAccessibleUtilities.clear(incoming);
               continue;
          }
          // Have a batch
          break;
      }
    
    ```
    
    The above can be simplified a bit, but should convey the idea.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] drill pull request #906: DRILL-5546: Handle schema change exception failure ...

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/906#discussion_r134636771
  
    --- Diff: exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/project/ProjectRecordBatch.java ---
    @@ -768,4 +765,73 @@ else if (exprHasPrefix && refHasPrefix) {
           }
         }
       }
    +
    +  /**
    +   * handle FAST NONE specially when Project for query output. This happens when input returns a
    +   * FAST NONE directly ( input does not return any batch with schema/data).
    +   *
    +   * Project operator has to return a batch with schema derived using the following 3 rules:
    +   *  Case 1:  *  ==>  expand into an empty list of columns.
    +   *  Case 2:  regular column reference ==> treat as nullable-int column
    +   *  Case 3:  expressions => Call ExpressionTreeMaterialization over an empty vector contain.
    +   *           Once the expression is materialized without error, use the output type of materialized
    +   *           expression.
    +   * The batch is constructed with the above rules, and recordCount = 0.
    +   * Returned with OK_NEW_SCHEMA to down-stream operator.
    +   */
    +  @Override
    +  protected IterOutcome handleFastNone() {
    +    if (! popConfig.isOutputProj()) {
    +      return super.handleFastNone();
    +    }
    +
    +    allocationVectors = new ArrayList<>();
    +    final List<NamedExpression> exprs = getExpressionList();
    +    final ErrorCollector collector = new ErrorCollectorImpl();
    +    VectorContainer fakedIncomingVC = new VectorContainer();
    +
    +    for (NamedExpression namedExpression : exprs) {
    +      if (namedExpression.getExpr() instanceof SchemaPath) {
    +        final NameSegment expr = ((SchemaPath) namedExpression.getExpr()).getRootSegment();
    +        if (expr.getPath().contains(StarColumnHelper.STAR_COLUMN)) {
    +          continue; // * would expand into an empty list.
    +        } else {
    +          final TypeProtos.MajorType majorType = TypeProtos.MajorType.newBuilder()
    +              .setMinorType(MinorType.INT)
    +              .setMode(TypeProtos.DataMode.OPTIONAL)
    +              .build();
    +
    +          MaterializedField outputField = MaterializedField.create(namedExpression.getRef().getRootSegment().getPath(), majorType);
    +          final ValueVector vv = container.addOrGet(outputField, callBack);
    +          allocationVectors.add(vv);
    +        }
    +        continue;
    +      }
    +
    +      final LogicalExpression materializedExpr = ExpressionTreeMaterializer.materialize(namedExpression.getExpr(),
    +          fakedIncomingVC,
    +          collector,
    +          context.getFunctionRegistry(),
    +          true,
    +          unionTypeEnabled);
    +
    +      if (collector.hasErrors()) {
    +        throw new IllegalArgumentException(String.format("Failure while trying to materialize expressions : %s.  Errors:\n %s.",
    +            namedExpression.getExpr(),
    +            collector.toErrorString()));
    +      }
    +
    +      final MaterializedField outputField = MaterializedField.create(namedExpression.getRef().getRootSegment().getPath(),
    +          materializedExpr.getMajorType());
    +      final ValueVector vv = container.addOrGet(outputField, callBack);
    +      allocationVectors.add(vv);
    +
    +    }
    +
    +    doAlloc(0);
    +    container.buildSchema(SelectionVectorMode.NONE);
    +    wasNone = true;
    +    return IterOutcome.OK_NEW_SCHEMA;
    +  }
    --- End diff --
    
    The above issues aside, a "mental execution" of the "fast none" code path suggests it should work.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] drill pull request #906: DRILL-5546: Handle schema change exception failure ...

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/906#discussion_r135369225
  
    --- Diff: exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/union/UnionAllRecordBatch.java ---
    @@ -39,88 +35,107 @@
     import org.apache.drill.exec.expr.ValueVectorWriteExpression;
     import org.apache.drill.exec.ops.FragmentContext;
     import org.apache.drill.exec.physical.config.UnionAll;
    -import org.apache.drill.exec.record.AbstractRecordBatch;
    +import org.apache.drill.exec.record.AbstractBinaryRecordBatch;
     import org.apache.drill.exec.record.BatchSchema;
     import org.apache.drill.exec.record.MaterializedField;
     import org.apache.drill.exec.record.RecordBatch;
     import org.apache.drill.exec.record.TransferPair;
     import org.apache.drill.exec.record.TypedFieldId;
     import org.apache.drill.exec.record.VectorWrapper;
    -import org.apache.drill.exec.record.WritableBatch;
    -import org.apache.drill.exec.record.selection.SelectionVector2;
    -import org.apache.drill.exec.record.selection.SelectionVector4;
     import org.apache.drill.exec.resolver.TypeCastRules;
     import org.apache.drill.exec.vector.AllocationHelper;
     import org.apache.drill.exec.vector.FixedWidthVector;
     import org.apache.drill.exec.vector.SchemaChangeCallBack;
     import org.apache.drill.exec.vector.ValueVector;
     
    -import com.google.common.collect.Lists;
    +import java.io.IOException;
    +import java.util.Iterator;
    +import java.util.List;
    +import java.util.NoSuchElementException;
    +import java.util.Stack;
     
    -public class UnionAllRecordBatch extends AbstractRecordBatch<UnionAll> {
    +public class UnionAllRecordBatch extends AbstractBinaryRecordBatch<UnionAll> {
       private static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(UnionAllRecordBatch.class);
     
    -  private List<MaterializedField> outputFields;
    +  private SchemaChangeCallBack callBack = new SchemaChangeCallBack();
       private UnionAller unionall;
    -  private UnionAllInput unionAllInput;
    -  private RecordBatch current;
    -
       private final List<TransferPair> transfers = Lists.newArrayList();
    -  private List<ValueVector> allocationVectors;
    -  protected SchemaChangeCallBack callBack = new SchemaChangeCallBack();
    +  private List<ValueVector> allocationVectors = Lists.newArrayList();
       private int recordCount = 0;
    -  private boolean schemaAvailable = false;
    +  private UnionInputIterator unionInputIterator;
     
       public UnionAllRecordBatch(UnionAll config, List<RecordBatch> children, FragmentContext context) throws OutOfMemoryException {
    -    super(config, context, false);
    -    assert (children.size() == 2) : "The number of the operands of Union must be 2";
    -    unionAllInput = new UnionAllInput(this, children.get(0), children.get(1));
    -  }
    -
    -  @Override
    -  public int getRecordCount() {
    -    return recordCount;
    +    super(config, context, true, children.get(0), children.get(1));
       }
     
       @Override
       protected void killIncoming(boolean sendUpstream) {
    -    unionAllInput.getLeftRecordBatch().kill(sendUpstream);
    -    unionAllInput.getRightRecordBatch().kill(sendUpstream);
    +    left.kill(sendUpstream);
    +    right.kill(sendUpstream);
       }
     
    -  @Override
    -  public SelectionVector2 getSelectionVector2() {
    -    throw new UnsupportedOperationException("UnionAllRecordBatch does not support selection vector");
    -  }
    +  protected void buildSchema() throws SchemaChangeException {
    +    if (! prefetchFirstBatchFromBothSides()) {
    +      return;
    +    }
     
    -  @Override
    -  public SelectionVector4 getSelectionVector4() {
    -    throw new UnsupportedOperationException("UnionAllRecordBatch does not support selection vector");
    +    unionInputIterator = new UnionInputIterator(leftUpstream, left, rightUpstream, right);
    +
    +    if (leftUpstream == IterOutcome.NONE && rightUpstream == IterOutcome.OK_NEW_SCHEMA) {
    +      inferOutputFieldsOneSide(right.getSchema());
    +    } else if (rightUpstream == IterOutcome.NONE && leftUpstream == IterOutcome.OK_NEW_SCHEMA) {
    +      inferOutputFieldsOneSide((left.getSchema()));
    +    } else if (leftUpstream == IterOutcome.OK_NEW_SCHEMA && rightUpstream == IterOutcome.OK_NEW_SCHEMA) {
    +      inferOutputFieldsBothSide(left.getSchema(), right.getSchema());
    +    }
    +
    +    container.buildSchema(BatchSchema.SelectionVectorMode.NONE);
    +
    +    for (VectorWrapper vv: container) {
    +      vv.getValueVector().allocateNew();
    +      vv.getValueVector().getMutator().setValueCount(0);
    +    }
       }
     
       @Override
       public IterOutcome innerNext() {
         try {
    -      IterOutcome upstream = unionAllInput.nextBatch();
    -      logger.debug("Upstream of Union-All: {}", upstream);
    +      if (!unionInputIterator.hasNext()) {
    +        return IterOutcome.NONE;
    +      }
    +
    +      Pair<IterOutcome, RecordBatch> nextBatch = unionInputIterator.next();
    +
    +      IterOutcome upstream = nextBatch.left;
    +      RecordBatch incoming = nextBatch.right;
    +
    +      // skip batches with same schema as the previous one yet having 0 row.
    +      if (upstream == IterOutcome.OK && incoming.getRecordCount() == 0) {
    +        do {
    +          for (final VectorWrapper<?> w : incoming) {
    +            w.clear();
    +          }
    --- End diff --
    
    `VectorAccessibleUtilities.clear(incoming)`


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] drill pull request #906: DRILL-5546: Handle schema change exception failure ...

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/906#discussion_r136445816
  
    --- Diff: exec/java-exec/src/main/java/org/apache/drill/exec/record/AbstractBinaryRecordBatch.java ---
    @@ -0,0 +1,75 @@
    +/**
    + * 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.record;
    +
    +import org.apache.drill.exec.exception.OutOfMemoryException;
    +import org.apache.drill.exec.ops.FragmentContext;
    +import org.apache.drill.exec.physical.base.PhysicalOperator;
    +
    +public abstract class AbstractBinaryRecordBatch<T extends PhysicalOperator> extends  AbstractRecordBatch<T> {
    +  protected final RecordBatch left;
    +  protected final RecordBatch right;
    +
    +  // state (IterOutcome) of the left input
    +  protected IterOutcome leftUpstream = IterOutcome.NONE;
    +
    +  // state (IterOutcome) of the right input
    +  protected IterOutcome rightUpstream = IterOutcome.NONE;
    --- End diff --
    
    Thanks for the explanation.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] drill pull request #906: DRILL-5546: Handle schema change exception failure ...

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

    https://github.com/apache/drill/pull/906#discussion_r135424090
  
    --- Diff: exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/ScanBatch.java ---
    @@ -152,97 +157,75 @@ public void kill(boolean sendUpstream) {
         }
       }
     
    -  private void releaseAssets() {
    -    container.zeroVectors();
    -  }
    -
    -  private void clearFieldVectorMap() {
    -    for (final ValueVector v : mutator.fieldVectorMap().values()) {
    -      v.clear();
    -    }
    -  }
    -
       @Override
       public IterOutcome next() {
         if (done) {
           return IterOutcome.NONE;
         }
         oContext.getStats().startProcessing();
         try {
    -      try {
    -        injector.injectChecked(context.getExecutionControls(), "next-allocate", OutOfMemoryException.class);
    -
    -        currentReader.allocate(mutator.fieldVectorMap());
    -      } catch (OutOfMemoryException e) {
    -        clearFieldVectorMap();
    -        throw UserException.memoryError(e).build(logger);
    -      }
    -      while ((recordCount = currentReader.next()) == 0) {
    +      while (true) {
             try {
    -          if (!readers.hasNext()) {
    -            // We're on the last reader, and it has no (more) rows.
    -            currentReader.close();
    -            releaseAssets();
    -            done = true;  // have any future call to next() return NONE
    -
    -            if (mutator.isNewSchema()) {
    -              // This last reader has a new schema (e.g., we have a zero-row
    -              // file or other source).  (Note that some sources have a non-
    -              // null/non-trivial schema even when there are no rows.)
    +          injector.injectChecked(context.getExecutionControls(), "next-allocate", OutOfMemoryException.class);
    +          currentReader.allocate(mutator.fieldVectorMap());
    +        } catch (OutOfMemoryException e) {
    +          clearFieldVectorMap();
    +          throw UserException.memoryError(e).build(logger);
    +        }
     
    -              container.buildSchema(SelectionVectorMode.NONE);
    -              schema = container.getSchema();
    +        recordCount = currentReader.next();
    +        Preconditions.checkArgument(recordCount >= 0,
    +            "recordCount from RecordReader.next() should not be negative");
     
    -              return IterOutcome.OK_NEW_SCHEMA;
    -            }
    -            return IterOutcome.NONE;
    -          }
    -          // At this point, the reader that hit its end is not the last reader.
    +        boolean isNewRegularSchema = mutator.isNewSchema();
    +        // We should skip the reader, when recordCount = 0 && ! isNewRegularSchema.
    +        // Add/set implicit column vectors, only when reader gets > 0 row, or
    +        // when reader gets 0 row but with a schema with new field added
    +        if (recordCount > 0 || isNewRegularSchema) {
    +          addImplicitVectors();
    +          populateImplicitVectors();
    +        }
     
    -          // If all the files we have read so far are just empty, the schema is not useful
    -          if (! hasReadNonEmptyFile) {
    -            container.clear();
    -            clearFieldVectorMap();
    -            mutator.clear();
    -          }
    +        boolean isNewImplicitSchema = mutator.isNewSchema();
    +        for (VectorWrapper<?> w : container) {
    +          w.getValueVector().getMutator().setValueCount(recordCount);
    +        }
    +        final boolean isNewSchema = isNewRegularSchema || isNewImplicitSchema;
    +        oContext.getStats().batchReceived(0, recordCount, isNewSchema);
     
    +        if (recordCount == 0) {
               currentReader.close();
    -          currentReader = readers.next();
    -          implicitValues = implicitColumns.hasNext() ? implicitColumns.next() : null;
    -          currentReader.setup(oContext, mutator);
    -          try {
    -            currentReader.allocate(mutator.fieldVectorMap());
    -          } catch (OutOfMemoryException e) {
    -            clearFieldVectorMap();
    -            throw UserException.memoryError(e).build(logger);
    +          if (isNewSchema) {
    +            // current reader presents a new schema in mutator even though it has 0 row.
    +            // This could happen when data sources have a non-trivial schema with 0 row.
    +            container.buildSchema(SelectionVectorMode.NONE);
    +            schema = container.getSchema();
    +            if (readers.hasNext()) {
    --- End diff --
    
    I'm neither clear nor convinced that using internal state (such as NO_READER) )would make the code clearer. But if you have such idea implemented in your patch, why don't we review such idea later on?
    
    To some degree, everything we do is "temporary"; as we continue to improve Drill, nothing is implied to be "permanent". 


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] drill pull request #906: DRILL-5546: Handle schema change exception failure ...

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/906#discussion_r136446075
  
    --- Diff: exec/vector/src/main/java/org/apache/drill/exec/vector/UntypedNullHolder.java ---
    @@ -0,0 +1,50 @@
    +/*
    + * 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;
    +
    +import org.apache.drill.common.types.TypeProtos;
    +import org.apache.drill.common.types.Types;
    +import org.apache.drill.exec.expr.holders.ValueHolder;
    +
    +public class UntypedNullHolder implements ValueHolder {
    +  public static final TypeProtos.MajorType TYPE = Types.optional(TypeProtos.MinorType.NULL);
    +
    +  public TypeProtos.MajorType getType() {return TYPE;}
    +
    +  public static final int WIDTH = 0;
    +
    +  public int isSet = 1;
    +
    +  @Deprecated
    +  public int hashCode(){
    +    throw new UnsupportedOperationException();
    +  }
    +
    +  /*
    +   * Reason for deprecation is that ValueHolders are potential scalar replacements
    +   * and hence we don't want any methods to be invoked on them.
    +   */
    +  @Deprecated
    +  public String toString(){
    +    throw new UnsupportedOperationException();
    --- End diff --
    
    Hmmm... Not entirely convinced, but let's let it go for now.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] drill pull request #906: DRILL-5546: Handle schema change exception failure ...

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

    https://github.com/apache/drill/pull/906#discussion_r135924402
  
    --- Diff: exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/project/ProjectRecordBatch.java ---
    @@ -768,4 +765,73 @@ else if (exprHasPrefix && refHasPrefix) {
           }
         }
       }
    +
    +  /**
    +   * handle FAST NONE specially when Project for query output. This happens when input returns a
    +   * FAST NONE directly ( input does not return any batch with schema/data).
    +   *
    --- End diff --
    
    Done.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] drill pull request #906: DRILL-5546: Handle schema change exception failure ...

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

    https://github.com/apache/drill/pull/906#discussion_r136188197
  
    --- Diff: exec/vector/src/main/java/org/apache/drill/exec/vector/UntypedNullVector.java ---
    @@ -0,0 +1,267 @@
    +/*
    + * 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;
    +
    +
    +import com.google.common.base.Preconditions;
    +import io.netty.buffer.DrillBuf;
    +import org.apache.drill.exec.memory.BufferAllocator;
    +import org.apache.drill.exec.proto.UserBitShared.SerializedField;
    +import org.apache.drill.exec.record.MaterializedField;
    +import org.apache.drill.exec.record.TransferPair;
    +import org.apache.drill.exec.vector.complex.reader.FieldReader;
    +
    +/** UntypedNullVector is to represent a value vector with {@link org.apache.drill.common.types.MinorType.NULL}
    + *  All values in the vector represent two semantic implications: 1) the value is unknown, 2) the type is unknown.
    + *  Because of this, we only have to keep track of the number of values in value vector,
    + *  and there is no allocated buffer to back up this value vector. Therefore, the majority of
    + *  methods in this class is either no-op, or throws {@link UnsupportedOperationException}.
    + *
    + */
    +public final class UntypedNullVector extends BaseDataValueVector implements FixedWidthVector {
    +  private static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(UntypedNullVector.class);
    +
    +  /**
    +   * Width of each fixed-width value.
    +   */
    +  public static final int VALUE_WIDTH = 0;
    +
    +  private final Accessor accessor = new Accessor();
    +  private final Mutator mutator = new Mutator();
    +  private int valueCount ;
    +
    +  public UntypedNullVector(MaterializedField field, BufferAllocator allocator) {
    +    super(field, allocator);
    +    valueCount = 0;
    +  }
    +
    +  @Override
    +  public FieldReader getReader() { throw new UnsupportedOperationException(); }
    +
    +  @Override
    +  public int getBufferSizeFor(final int valueCount) {
    +    return 0;
    +  }
    +
    +  @Override
    +  public int getValueCapacity(){
    +    return Character.MAX_VALUE;
    --- End diff --
    
    Done.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] drill pull request #906: DRILL-5546: Handle schema change exception failure ...

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/906#discussion_r135580061
  
    --- Diff: exec/vector/src/main/java/org/apache/drill/exec/vector/UntypedNullVector.java ---
    @@ -0,0 +1,267 @@
    +/*
    + * 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;
    +
    +
    +import com.google.common.base.Preconditions;
    +import io.netty.buffer.DrillBuf;
    +import org.apache.drill.exec.memory.BufferAllocator;
    +import org.apache.drill.exec.proto.UserBitShared.SerializedField;
    +import org.apache.drill.exec.record.MaterializedField;
    +import org.apache.drill.exec.record.TransferPair;
    +import org.apache.drill.exec.vector.complex.reader.FieldReader;
    +
    +/** UntypedNullVector is to represent a value vector with {@link org.apache.drill.common.types.MinorType.NULL}
    + *  All values in the vector represent two semantic implications: 1) the value is unknown, 2) the type is unknown.
    + *  Because of this, we only have to keep track of the number of values in value vector,
    + *  and there is no allocated buffer to back up this value vector. Therefore, the majority of
    + *  methods in this class is either no-op, or throws {@link UnsupportedOperationException}.
    + *
    + */
    +public final class UntypedNullVector extends BaseDataValueVector implements FixedWidthVector {
    +  private static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(UntypedNullVector.class);
    +
    +  /**
    +   * Width of each fixed-width value.
    +   */
    +  public static final int VALUE_WIDTH = 0;
    +
    +  private final Accessor accessor = new Accessor();
    +  private final Mutator mutator = new Mutator();
    +  private int valueCount ;
    +
    +  public UntypedNullVector(MaterializedField field, BufferAllocator allocator) {
    +    super(field, allocator);
    +    valueCount = 0;
    +  }
    +
    +  @Override
    +  public FieldReader getReader() { throw new UnsupportedOperationException(); }
    +
    +  @Override
    +  public int getBufferSizeFor(final int valueCount) {
    +    return 0;
    +  }
    +
    +  @Override
    +  public int getValueCapacity(){
    +    return Character.MAX_VALUE;
    +  }
    +
    +  @Override
    +  public Accessor getAccessor() { return accessor; }
    +
    +  @Override
    +  public Mutator getMutator() { return mutator; }
    +
    +  @Override
    +  public void setInitialCapacity(final int valueCount) {
    +  }
    +
    +  @Override
    +  public void allocateNew() {
    +  }
    +
    +  @Override
    +  public boolean allocateNewSafe() {
    +    return true;
    +  }
    +
    +  @Override
    +  public void allocateNew(final int valueCount) {
    +  }
    +
    +  @Override
    +  public void reset() {
    +  }
    +
    +  /**
    +   * {@inheritDoc}
    +   */
    +  @Override
    +  public void zeroVector() {
    +  }
    +
    +  @Override
    +  public void load(SerializedField metadata, DrillBuf buffer) {
    +    Preconditions.checkArgument(this.field.getPath().equals(metadata.getNamePart().getName()),
    +        "The field %s doesn't match the provided metadata %s.", this.field, metadata);
    +    final int actualLength = metadata.getBufferLength();
    +    final int valueCount = metadata.getValueCount();
    +    final int expectedLength = valueCount * VALUE_WIDTH;
    +    assert actualLength == expectedLength : String.format("Expected to load %d bytes but actually loaded %d bytes", expectedLength, actualLength);
    +
    +    this.valueCount = valueCount;
    +  }
    +
    +  @Override
    +  public TransferPair getTransferPair(BufferAllocator allocator){
    +    return new TransferImpl(getField(), allocator);
    +  }
    +
    +  @Override
    +  public TransferPair getTransferPair(String ref, BufferAllocator allocator){
    +    return new TransferImpl(getField().withPath(ref), allocator);
    +  }
    +
    +  @Override
    +  public TransferPair makeTransferPair(ValueVector to) {
    +    return new TransferImpl((UntypedNullVector) to);
    +  }
    +
    +  public void transferTo(UntypedNullVector target){
    +  }
    +
    +  public void splitAndTransferTo(int startIndex, int length, UntypedNullVector target) {
    +  }
    +
    +  @Override
    +  public int getPayloadByteCount() {
    +    return 0;
    +  }
    +
    +  private class TransferImpl implements TransferPair{
    +    private UntypedNullVector to;
    +
    +    public TransferImpl(MaterializedField field, BufferAllocator allocator){
    +      to = new UntypedNullVector(field, allocator);
    +    }
    +
    +    public TransferImpl(UntypedNullVector to) {
    +      this.to = to;
    +    }
    +
    +    @Override
    +    public UntypedNullVector getTo(){
    +      return to;
    +    }
    +
    +    @Override
    +    public void transfer(){
    +      transferTo(to);
    +    }
    +
    +    @Override
    +    public void splitAndTransfer(int startIndex, int length) {
    +      checkBounds(startIndex);
    +      checkBounds(startIndex + length - 1);
    +      splitAndTransferTo(startIndex, length, to);
    +    }
    +
    +    @Override
    +    public void copyValueSafe(int fromIndex, int toIndex) {
    +      checkBounds(fromIndex);
    +      to.copyFromSafe(fromIndex, toIndex, UntypedNullVector.this);
    +    }
    +  }
    +
    +  public void copyFrom(int fromIndex, int thisIndex, UntypedNullVector from){
    +  }
    +
    +  public void copyFromSafe(int fromIndex, int thisIndex, UntypedNullVector from){
    +  }
    +
    +  private void checkBounds(int index) {
    +    if (index < 0 || index >= valueCount) {
    +      throw new IndexOutOfBoundsException(String.format(
    +          "index: %d, expected: range(0, %d-1))", index, valueCount));
    +    }
    +  }
    +  @Override
    +  public void copyEntry(int toIndex, ValueVector from, int fromIndex) {
    +    ((UntypedNullVector) from).data.getBytes(fromIndex * 4, data, toIndex * 4, 4);
    +  }
    +
    +  public final class Accessor extends BaseAccessor {
    +    @Override
    +    public int getValueCount() {
    +      return valueCount;
    +    }
    +
    +    @Override
    +    public boolean isNull(int index){
    +      checkBounds(index);
    +      return true;
    +    }
    +
    +    public int isSet(int index) {
    +      checkBounds(index);
    +      return 0;
    +    }
    +
    +    @Override
    +    public Object getObject(int index) {
    +      throw new UnsupportedOperationException();
    +    }
    +
    +    public void get(int index, UntypedNullHolder holder) {
    +      checkBounds(index);
    +    }
    +
    +  }
    +
    +  /**
    +   * UntypedNullVector.Mutator throws Exception for most of its mutate operations, except for the ones that set
    +   * value counts.
    +   *
    +   */
    +   public final class Mutator extends BaseMutator {
    +
    +    private Mutator() {};
    --- End diff --
    
    Drop trailing ;


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] drill pull request #906: DRILL-5546: Handle schema change exception failure ...

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

    https://github.com/apache/drill/pull/906#discussion_r135924412
  
    --- Diff: exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/project/ProjectRecordBatch.java ---
    @@ -768,4 +765,73 @@ else if (exprHasPrefix && refHasPrefix) {
           }
         }
       }
    +
    +  /**
    +   * handle FAST NONE specially when Project for query output. This happens when input returns a
    +   * FAST NONE directly ( input does not return any batch with schema/data).
    +   *
    +   * Project operator has to return a batch with schema derived using the following 3 rules:
    +   *  Case 1:  *  ==>  expand into an empty list of columns.
    +   *  Case 2:  regular column reference ==> treat as nullable-int column
    +   *  Case 3:  expressions => Call ExpressionTreeMaterialization over an empty vector contain.
    --- End diff --
    
    Done. 


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] drill pull request #906: DRILL-5546: Handle schema change exception failure ...

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/906#discussion_r134296622
  
    --- Diff: exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/ScanBatch.java ---
    @@ -152,97 +157,75 @@ public void kill(boolean sendUpstream) {
         }
       }
     
    -  private void releaseAssets() {
    -    container.zeroVectors();
    -  }
    -
    -  private void clearFieldVectorMap() {
    -    for (final ValueVector v : mutator.fieldVectorMap().values()) {
    -      v.clear();
    -    }
    -  }
    -
       @Override
       public IterOutcome next() {
         if (done) {
           return IterOutcome.NONE;
         }
         oContext.getStats().startProcessing();
         try {
    -      try {
    -        injector.injectChecked(context.getExecutionControls(), "next-allocate", OutOfMemoryException.class);
    -
    -        currentReader.allocate(mutator.fieldVectorMap());
    -      } catch (OutOfMemoryException e) {
    -        clearFieldVectorMap();
    -        throw UserException.memoryError(e).build(logger);
    -      }
    -      while ((recordCount = currentReader.next()) == 0) {
    +      while (true) {
             try {
    -          if (!readers.hasNext()) {
    -            // We're on the last reader, and it has no (more) rows.
    -            currentReader.close();
    -            releaseAssets();
    -            done = true;  // have any future call to next() return NONE
    -
    -            if (mutator.isNewSchema()) {
    -              // This last reader has a new schema (e.g., we have a zero-row
    -              // file or other source).  (Note that some sources have a non-
    -              // null/non-trivial schema even when there are no rows.)
    +          injector.injectChecked(context.getExecutionControls(), "next-allocate", OutOfMemoryException.class);
    +          currentReader.allocate(mutator.fieldVectorMap());
    +        } catch (OutOfMemoryException e) {
    +          clearFieldVectorMap();
    +          throw UserException.memoryError(e).build(logger);
    +        }
     
    -              container.buildSchema(SelectionVectorMode.NONE);
    -              schema = container.getSchema();
    +        recordCount = currentReader.next();
    +        Preconditions.checkArgument(recordCount >= 0,
    +            "recordCount from RecordReader.next() should not be negative");
     
    -              return IterOutcome.OK_NEW_SCHEMA;
    -            }
    -            return IterOutcome.NONE;
    -          }
    -          // At this point, the reader that hit its end is not the last reader.
    +        boolean isNewRegularSchema = mutator.isNewSchema();
    --- End diff --
    
    `isNewRegularSchema` --> `isNewTableSchema`? This describes the table portion of the schema, as contrasted with the implicit part mentioned below.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] drill pull request #906: DRILL-5546: Handle schema change exception failure ...

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/906#discussion_r134301372
  
    --- Diff: exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/ScanBatch.java ---
    @@ -152,97 +157,75 @@ public void kill(boolean sendUpstream) {
         }
       }
     
    -  private void releaseAssets() {
    -    container.zeroVectors();
    -  }
    -
    -  private void clearFieldVectorMap() {
    -    for (final ValueVector v : mutator.fieldVectorMap().values()) {
    -      v.clear();
    -    }
    -  }
    -
       @Override
       public IterOutcome next() {
         if (done) {
           return IterOutcome.NONE;
         }
         oContext.getStats().startProcessing();
         try {
    -      try {
    -        injector.injectChecked(context.getExecutionControls(), "next-allocate", OutOfMemoryException.class);
    -
    -        currentReader.allocate(mutator.fieldVectorMap());
    -      } catch (OutOfMemoryException e) {
    -        clearFieldVectorMap();
    -        throw UserException.memoryError(e).build(logger);
    -      }
    -      while ((recordCount = currentReader.next()) == 0) {
    +      while (true) {
             try {
    -          if (!readers.hasNext()) {
    -            // We're on the last reader, and it has no (more) rows.
    -            currentReader.close();
    -            releaseAssets();
    -            done = true;  // have any future call to next() return NONE
    -
    -            if (mutator.isNewSchema()) {
    -              // This last reader has a new schema (e.g., we have a zero-row
    -              // file or other source).  (Note that some sources have a non-
    -              // null/non-trivial schema even when there are no rows.)
    +          injector.injectChecked(context.getExecutionControls(), "next-allocate", OutOfMemoryException.class);
    +          currentReader.allocate(mutator.fieldVectorMap());
    +        } catch (OutOfMemoryException e) {
    +          clearFieldVectorMap();
    +          throw UserException.memoryError(e).build(logger);
    +        }
     
    -              container.buildSchema(SelectionVectorMode.NONE);
    -              schema = container.getSchema();
    +        recordCount = currentReader.next();
    +        Preconditions.checkArgument(recordCount >= 0,
    +            "recordCount from RecordReader.next() should not be negative");
     
    -              return IterOutcome.OK_NEW_SCHEMA;
    -            }
    -            return IterOutcome.NONE;
    -          }
    -          // At this point, the reader that hit its end is not the last reader.
    +        boolean isNewRegularSchema = mutator.isNewSchema();
    +        // We should skip the reader, when recordCount = 0 && ! isNewRegularSchema.
    +        // Add/set implicit column vectors, only when reader gets > 0 row, or
    +        // when reader gets 0 row but with a schema with new field added
    +        if (recordCount > 0 || isNewRegularSchema) {
    +          addImplicitVectors();
    +          populateImplicitVectors();
    +        }
     
    -          // If all the files we have read so far are just empty, the schema is not useful
    -          if (! hasReadNonEmptyFile) {
    -            container.clear();
    -            clearFieldVectorMap();
    -            mutator.clear();
    -          }
    +        boolean isNewImplicitSchema = mutator.isNewSchema();
    +        for (VectorWrapper<?> w : container) {
    +          w.getValueVector().getMutator().setValueCount(recordCount);
    +        }
    +        final boolean isNewSchema = isNewRegularSchema || isNewImplicitSchema;
    +        oContext.getStats().batchReceived(0, recordCount, isNewSchema);
     
    +        if (recordCount == 0) {
               currentReader.close();
    -          currentReader = readers.next();
    -          implicitValues = implicitColumns.hasNext() ? implicitColumns.next() : null;
    -          currentReader.setup(oContext, mutator);
    -          try {
    -            currentReader.allocate(mutator.fieldVectorMap());
    -          } catch (OutOfMemoryException e) {
    -            clearFieldVectorMap();
    -            throw UserException.memoryError(e).build(logger);
    +          if (isNewSchema) {
    +            // current reader presents a new schema in mutator even though it has 0 row.
    --- End diff --
    
    Thanks for the comments here and in the PR. I wonder, can the PR description be moved into a class Javadoc comment so that it is available for future readers? Also, makes it easier to review since the description is close to the code.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] drill pull request #906: DRILL-5546: Handle schema change exception failure ...

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/906#discussion_r134635590
  
    --- Diff: exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/project/ProjectRecordBatch.java ---
    @@ -768,4 +765,73 @@ else if (exprHasPrefix && refHasPrefix) {
           }
         }
       }
    +
    +  /**
    +   * handle FAST NONE specially when Project for query output. This happens when input returns a
    +   * FAST NONE directly ( input does not return any batch with schema/data).
    +   *
    +   * Project operator has to return a batch with schema derived using the following 3 rules:
    +   *  Case 1:  *  ==>  expand into an empty list of columns.
    +   *  Case 2:  regular column reference ==> treat as nullable-int column
    +   *  Case 3:  expressions => Call ExpressionTreeMaterialization over an empty vector contain.
    +   *           Once the expression is materialized without error, use the output type of materialized
    +   *           expression.
    +   * The batch is constructed with the above rules, and recordCount = 0.
    +   * Returned with OK_NEW_SCHEMA to down-stream operator.
    +   */
    +  @Override
    +  protected IterOutcome handleFastNone() {
    +    if (! popConfig.isOutputProj()) {
    +      return super.handleFastNone();
    +    }
    +
    +    allocationVectors = new ArrayList<>();
    +    final List<NamedExpression> exprs = getExpressionList();
    +    final ErrorCollector collector = new ErrorCollectorImpl();
    +    VectorContainer fakedIncomingVC = new VectorContainer();
    +
    +    for (NamedExpression namedExpression : exprs) {
    +      if (namedExpression.getExpr() instanceof SchemaPath) {
    +        final NameSegment expr = ((SchemaPath) namedExpression.getExpr()).getRootSegment();
    +        if (expr.getPath().contains(StarColumnHelper.STAR_COLUMN)) {
    +          continue; // * would expand into an empty list.
    +        } else {
    +          final TypeProtos.MajorType majorType = TypeProtos.MajorType.newBuilder()
    +              .setMinorType(MinorType.INT)
    +              .setMode(TypeProtos.DataMode.OPTIONAL)
    +              .build();
    +
    +          MaterializedField outputField = MaterializedField.create(namedExpression.getRef().getRootSegment().getPath(), majorType);
    +          final ValueVector vv = container.addOrGet(outputField, callBack);
    +          allocationVectors.add(vv);
    +        }
    +        continue;
    +      }
    +
    +      final LogicalExpression materializedExpr = ExpressionTreeMaterializer.materialize(namedExpression.getExpr(),
    +          fakedIncomingVC,
    +          collector,
    +          context.getFunctionRegistry(),
    +          true,
    +          unionTypeEnabled);
    +
    +      if (collector.hasErrors()) {
    +        throw new IllegalArgumentException(String.format("Failure while trying to materialize expressions : %s.  Errors:\n %s.",
    +            namedExpression.getExpr(),
    +            collector.toErrorString()));
    +      }
    --- End diff --
    
    The above block (call the expression tree materializer, and handle errors) already appears twice the code. Can we make use of a handy feature of Java to factor out the code into a method that is called three times, rather than just copy/pasting the code? Remember: [DRY](https://en.wikipedia.org/wiki/Don%27t_repeat_yourself).


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] drill pull request #906: DRILL-5546: Handle schema change exception failure ...

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

    https://github.com/apache/drill/pull/906#discussion_r136187789
  
    --- Diff: exec/vector/src/main/java/org/apache/drill/exec/vector/UntypedNullHolder.java ---
    @@ -0,0 +1,50 @@
    +/*
    + * 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;
    +
    +import org.apache.drill.common.types.TypeProtos;
    +import org.apache.drill.common.types.Types;
    +import org.apache.drill.exec.expr.holders.ValueHolder;
    +
    +public class UntypedNullHolder implements ValueHolder {
    +  public static final TypeProtos.MajorType TYPE = Types.optional(TypeProtos.MinorType.NULL);
    +
    +  public TypeProtos.MajorType getType() {return TYPE;}
    +
    +  public static final int WIDTH = 0;
    +
    +  public int isSet = 1;
    +
    +  @Deprecated
    +  public int hashCode(){
    +    throw new UnsupportedOperationException();
    +  }
    --- End diff --
    
    See response for `toString()`.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] drill pull request #906: DRILL-5546: Handle schema change exception failure ...

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/906#discussion_r135368986
  
    --- Diff: exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/union/UnionAllRecordBatch.java ---
    @@ -130,562 +145,248 @@ public IterOutcome innerNext() {
       }
     
       @Override
    -  public WritableBatch getWritableBatch() {
    -    return WritableBatch.get(this);
    +  public int getRecordCount() {
    +    return recordCount;
       }
     
    -  private void setValueCount(int count) {
    -    for (ValueVector v : allocationVectors) {
    -      ValueVector.Mutator m = v.getMutator();
    -      m.setValueCount(count);
    -    }
    -  }
     
    -  private boolean doAlloc() {
    -    for (ValueVector v : allocationVectors) {
    -      try {
    -        AllocationHelper.allocateNew(v, current.getRecordCount());
    -      } catch (OutOfMemoryException ex) {
    -        return false;
    -      }
    +  @SuppressWarnings("resource")
    +  private IterOutcome doWork(RecordBatch inputBatch, boolean newSchema) throws ClassTransformationException, IOException, SchemaChangeException {
    +    if (inputBatch.getSchema().getFieldCount() != container.getSchema().getFieldCount()) {
    +      // wrong.
         }
    -    return true;
    -  }
     
    -  @SuppressWarnings("resource")
    -  private IterOutcome doWork() throws ClassTransformationException, IOException, SchemaChangeException {
    -    if (allocationVectors != null) {
    -      for (ValueVector v : allocationVectors) {
    -        v.clear();
    -      }
    +    if (newSchema) {
    +      createUnionAller(inputBatch);
         }
     
    -    allocationVectors = Lists.newArrayList();
    -    transfers.clear();
    +    container.zeroVectors();
     
    -    // If both sides of Union-All are empty
    -    if(unionAllInput.isBothSideEmpty()) {
    -      for(int i = 0; i < outputFields.size(); ++i) {
    -        final String colName = outputFields.get(i).getPath();
    -        final MajorType majorType = MajorType.newBuilder()
    -            .setMinorType(MinorType.INT)
    -            .setMode(DataMode.OPTIONAL)
    -            .build();
    -
    -        MaterializedField outputField = MaterializedField.create(colName, majorType);
    -        ValueVector vv = container.addOrGet(outputField, callBack);
    -        allocationVectors.add(vv);
    -      }
    +    for (final ValueVector v : this.allocationVectors) {
    +      AllocationHelper.allocateNew(v, inputBatch.getRecordCount());
    +    }
    --- End diff --
    
    This is a common operation. Maybe move it into `VectorAccessibleUtilities` or some such. The difference here is that `allocationVectors` (no need for `this.`) is a list of vectors, not an iterator over vector wrappers.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] drill pull request #906: DRILL-5546: Handle schema change exception failure ...

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/906#discussion_r135369170
  
    --- Diff: exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/union/UnionAllRecordBatch.java ---
    @@ -130,562 +145,248 @@ public IterOutcome innerNext() {
       }
     
       @Override
    -  public WritableBatch getWritableBatch() {
    -    return WritableBatch.get(this);
    +  public int getRecordCount() {
    +    return recordCount;
       }
     
    -  private void setValueCount(int count) {
    -    for (ValueVector v : allocationVectors) {
    -      ValueVector.Mutator m = v.getMutator();
    -      m.setValueCount(count);
    -    }
    -  }
     
    -  private boolean doAlloc() {
    -    for (ValueVector v : allocationVectors) {
    -      try {
    -        AllocationHelper.allocateNew(v, current.getRecordCount());
    -      } catch (OutOfMemoryException ex) {
    -        return false;
    -      }
    +  @SuppressWarnings("resource")
    +  private IterOutcome doWork(RecordBatch inputBatch, boolean newSchema) throws ClassTransformationException, IOException, SchemaChangeException {
    +    if (inputBatch.getSchema().getFieldCount() != container.getSchema().getFieldCount()) {
    +      // wrong.
         }
    -    return true;
    -  }
     
    -  @SuppressWarnings("resource")
    -  private IterOutcome doWork() throws ClassTransformationException, IOException, SchemaChangeException {
    -    if (allocationVectors != null) {
    -      for (ValueVector v : allocationVectors) {
    -        v.clear();
    -      }
    +    if (newSchema) {
    +      createUnionAller(inputBatch);
         }
     
    -    allocationVectors = Lists.newArrayList();
    -    transfers.clear();
    +    container.zeroVectors();
     
    -    // If both sides of Union-All are empty
    -    if(unionAllInput.isBothSideEmpty()) {
    -      for(int i = 0; i < outputFields.size(); ++i) {
    -        final String colName = outputFields.get(i).getPath();
    -        final MajorType majorType = MajorType.newBuilder()
    -            .setMinorType(MinorType.INT)
    -            .setMode(DataMode.OPTIONAL)
    -            .build();
    -
    -        MaterializedField outputField = MaterializedField.create(colName, majorType);
    -        ValueVector vv = container.addOrGet(outputField, callBack);
    -        allocationVectors.add(vv);
    -      }
    +    for (final ValueVector v : this.allocationVectors) {
    +      AllocationHelper.allocateNew(v, inputBatch.getRecordCount());
    +    }
     
    -      container.buildSchema(BatchSchema.SelectionVectorMode.NONE);
    +    recordCount = unionall.unionRecords(0, inputBatch.getRecordCount(), 0);
    +    for (final ValueVector v : allocationVectors) {
    +      final ValueVector.Mutator m = v.getMutator();
    +      m.setValueCount(recordCount);
    +    }
    --- End diff --
    
    Same comment as above. Setting value count is common. Here, we have a list of vectors, not vector wrappers.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] drill pull request #906: DRILL-5546: Handle schema change exception failure ...

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/906#discussion_r135368764
  
    --- Diff: exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/union/UnionAllRecordBatch.java ---
    @@ -130,562 +145,248 @@ public IterOutcome innerNext() {
       }
     
       @Override
    -  public WritableBatch getWritableBatch() {
    -    return WritableBatch.get(this);
    +  public int getRecordCount() {
    +    return recordCount;
       }
     
    -  private void setValueCount(int count) {
    -    for (ValueVector v : allocationVectors) {
    -      ValueVector.Mutator m = v.getMutator();
    -      m.setValueCount(count);
    -    }
    -  }
     
    -  private boolean doAlloc() {
    -    for (ValueVector v : allocationVectors) {
    -      try {
    -        AllocationHelper.allocateNew(v, current.getRecordCount());
    -      } catch (OutOfMemoryException ex) {
    -        return false;
    -      }
    +  @SuppressWarnings("resource")
    +  private IterOutcome doWork(RecordBatch inputBatch, boolean newSchema) throws ClassTransformationException, IOException, SchemaChangeException {
    +    if (inputBatch.getSchema().getFieldCount() != container.getSchema().getFieldCount()) {
    +      // wrong.
    --- End diff --
    
    If it is wrong, do we want to do something? Throw an `IllegalStateException`?


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] drill pull request #906: DRILL-5546: Handle schema change exception failure ...

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

    https://github.com/apache/drill/pull/906#discussion_r136188791
  
    --- Diff: exec/vector/src/main/java/org/apache/drill/exec/vector/UntypedNullVector.java ---
    @@ -0,0 +1,267 @@
    +/*
    + * 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;
    +
    +
    +import com.google.common.base.Preconditions;
    +import io.netty.buffer.DrillBuf;
    +import org.apache.drill.exec.memory.BufferAllocator;
    +import org.apache.drill.exec.proto.UserBitShared.SerializedField;
    +import org.apache.drill.exec.record.MaterializedField;
    +import org.apache.drill.exec.record.TransferPair;
    +import org.apache.drill.exec.vector.complex.reader.FieldReader;
    +
    +/** UntypedNullVector is to represent a value vector with {@link org.apache.drill.common.types.MinorType.NULL}
    + *  All values in the vector represent two semantic implications: 1) the value is unknown, 2) the type is unknown.
    + *  Because of this, we only have to keep track of the number of values in value vector,
    + *  and there is no allocated buffer to back up this value vector. Therefore, the majority of
    + *  methods in this class is either no-op, or throws {@link UnsupportedOperationException}.
    + *
    + */
    +public final class UntypedNullVector extends BaseDataValueVector implements FixedWidthVector {
    +  private static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(UntypedNullVector.class);
    +
    +  /**
    +   * Width of each fixed-width value.
    +   */
    +  public static final int VALUE_WIDTH = 0;
    +
    +  private final Accessor accessor = new Accessor();
    +  private final Mutator mutator = new Mutator();
    +  private int valueCount ;
    +
    +  public UntypedNullVector(MaterializedField field, BufferAllocator allocator) {
    +    super(field, allocator);
    +    valueCount = 0;
    +  }
    +
    +  @Override
    +  public FieldReader getReader() { throw new UnsupportedOperationException(); }
    +
    +  @Override
    +  public int getBufferSizeFor(final int valueCount) {
    +    return 0;
    +  }
    +
    +  @Override
    +  public int getValueCapacity(){
    +    return Character.MAX_VALUE;
    +  }
    +
    +  @Override
    +  public Accessor getAccessor() { return accessor; }
    +
    +  @Override
    +  public Mutator getMutator() { return mutator; }
    +
    +  @Override
    +  public void setInitialCapacity(final int valueCount) {
    +  }
    +
    +  @Override
    +  public void allocateNew() {
    +  }
    +
    +  @Override
    +  public boolean allocateNewSafe() {
    +    return true;
    +  }
    +
    +  @Override
    +  public void allocateNew(final int valueCount) {
    +  }
    +
    +  @Override
    +  public void reset() {
    +  }
    +
    +  /**
    +   * {@inheritDoc}
    +   */
    +  @Override
    +  public void zeroVector() {
    +  }
    +
    +  @Override
    +  public void load(SerializedField metadata, DrillBuf buffer) {
    +    Preconditions.checkArgument(this.field.getPath().equals(metadata.getNamePart().getName()),
    +        "The field %s doesn't match the provided metadata %s.", this.field, metadata);
    +    final int actualLength = metadata.getBufferLength();
    +    final int valueCount = metadata.getValueCount();
    +    final int expectedLength = valueCount * VALUE_WIDTH;
    +    assert actualLength == expectedLength : String.format("Expected to load %d bytes but actually loaded %d bytes", expectedLength, actualLength);
    +
    +    this.valueCount = valueCount;
    +  }
    +
    +  @Override
    +  public TransferPair getTransferPair(BufferAllocator allocator){
    +    return new TransferImpl(getField(), allocator);
    +  }
    +
    +  @Override
    +  public TransferPair getTransferPair(String ref, BufferAllocator allocator){
    +    return new TransferImpl(getField().withPath(ref), allocator);
    +  }
    +
    +  @Override
    +  public TransferPair makeTransferPair(ValueVector to) {
    +    return new TransferImpl((UntypedNullVector) to);
    +  }
    +
    +  public void transferTo(UntypedNullVector target){
    +  }
    +
    +  public void splitAndTransferTo(int startIndex, int length, UntypedNullVector target) {
    +  }
    +
    +  @Override
    +  public int getPayloadByteCount() {
    +    return 0;
    +  }
    +
    +  private class TransferImpl implements TransferPair{
    +    private UntypedNullVector to;
    +
    +    public TransferImpl(MaterializedField field, BufferAllocator allocator){
    +      to = new UntypedNullVector(field, allocator);
    +    }
    +
    +    public TransferImpl(UntypedNullVector to) {
    +      this.to = to;
    +    }
    +
    +    @Override
    +    public UntypedNullVector getTo(){
    +      return to;
    +    }
    +
    +    @Override
    +    public void transfer(){
    +      transferTo(to);
    +    }
    +
    +    @Override
    +    public void splitAndTransfer(int startIndex, int length) {
    +      checkBounds(startIndex);
    +      checkBounds(startIndex + length - 1);
    +      splitAndTransferTo(startIndex, length, to);
    +    }
    +
    +    @Override
    +    public void copyValueSafe(int fromIndex, int toIndex) {
    +      checkBounds(fromIndex);
    +      to.copyFromSafe(fromIndex, toIndex, UntypedNullVector.this);
    +    }
    +  }
    +
    +  public void copyFrom(int fromIndex, int thisIndex, UntypedNullVector from){
    +  }
    +
    +  public void copyFromSafe(int fromIndex, int thisIndex, UntypedNullVector from){
    +  }
    +
    +  private void checkBounds(int index) {
    +    if (index < 0 || index >= valueCount) {
    +      throw new IndexOutOfBoundsException(String.format(
    +          "index: %d, expected: range(0, %d-1))", index, valueCount));
    +    }
    +  }
    +  @Override
    +  public void copyEntry(int toIndex, ValueVector from, int fromIndex) {
    +    ((UntypedNullVector) from).data.getBytes(fromIndex * 4, data, toIndex * 4, 4);
    +  }
    +
    +  public final class Accessor extends BaseAccessor {
    +    @Override
    +    public int getValueCount() {
    +      return valueCount;
    +    }
    +
    +    @Override
    +    public boolean isNull(int index){
    +      checkBounds(index);
    +      return true;
    +    }
    +
    +    public int isSet(int index) {
    +      checkBounds(index);
    +      return 0;
    +    }
    +
    +    @Override
    +    public Object getObject(int index) {
    +      throw new UnsupportedOperationException();
    --- End diff --
    
    makes sense.  Changed to `return null`.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] drill pull request #906: DRILL-5546: Handle schema change exception failure ...

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

    https://github.com/apache/drill/pull/906#discussion_r135976739
  
    --- Diff: exec/java-exec/src/test/java/org/apache/drill/exec/physical/unit/TestEmptyBatchMiniPlan.java ---
    @@ -0,0 +1,453 @@
    +/*
    + * 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
    + * <p/>
    + * http://www.apache.org/licenses/LICENSE-2.0
    + * <p/>
    + * 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.physical.unit;
    +
    +import com.google.common.collect.Lists;
    +import org.apache.calcite.rel.RelFieldCollation;
    +import org.apache.calcite.rel.core.JoinRelType;
    +import org.apache.drill.common.expression.SchemaPath;
    +import org.apache.drill.common.types.TypeProtos;
    +import org.apache.drill.common.util.FileUtils;
    +import org.apache.drill.exec.physical.base.PhysicalOperator;
    +import org.apache.drill.exec.physical.config.ExternalSort;
    +import org.apache.drill.exec.physical.config.Filter;
    +import org.apache.drill.exec.physical.config.FlattenPOP;
    +import org.apache.drill.exec.physical.config.HashAggregate;
    +import org.apache.drill.exec.physical.config.HashJoinPOP;
    +import org.apache.drill.exec.physical.config.Limit;
    +import org.apache.drill.exec.physical.config.MergeJoinPOP;
    +import org.apache.drill.exec.physical.config.Project;
    +import org.apache.drill.exec.physical.config.StreamingAggregate;
    +import org.apache.drill.exec.physical.config.UnionAll;
    +import org.apache.drill.exec.planner.physical.AggPrelBase;
    +import org.apache.drill.exec.record.BatchSchema;
    +import org.apache.drill.exec.record.RecordBatch;
    +import org.apache.drill.exec.store.dfs.DrillFileSystem;
    +import org.apache.drill.test.rowSet.SchemaBuilder;
    +import org.apache.hadoop.conf.Configuration;
    +import org.apache.hadoop.fs.FileSystem;
    +import org.junit.BeforeClass;
    +import org.junit.Ignore;
    +import org.junit.Test;
    +
    +import java.util.Collections;
    +import java.util.List;
    +
    +public class TestEmptyBatchMiniPlan extends MiniPlanUnitTestBase{
    +  protected static DrillFileSystem fs;
    +
    +  @BeforeClass
    +  public static void initFS() throws Exception {
    +    Configuration conf = new Configuration();
    +    conf.set(FileSystem.FS_DEFAULT_NAME_KEY, FileSystem.DEFAULT_FS);
    +    fs = new DrillFileSystem(conf);
    +  }
    +
    +  @Test
    +  public void testEmptyJsonInput() throws Exception {
    +    RecordBatch scanBatch = createEmptyBatchFromJson();
    +
    +    new MiniPlanTestBuilder()
    +        .root(scanBatch)
    +        .expectNullBatch(true)
    +        .go();
    +  }
    +
    +  @Test
    +  public void testProjectEmpty() throws Exception {
    +    final PhysicalOperator project = new Project(parseExprs("x+5", "x"), null);
    +    testSingleInputEmptyBatchHandling(project);
    +  }
    +
    +  @Test
    +  public void testFilterEmpty() throws Exception {
    +    final PhysicalOperator filter = new Filter(null, parseExpr("a=5"), 1.0f);
    +    testSingleInputEmptyBatchHandling(filter);
    +  }
    +
    +  @Test
    +  public void testHashAggEmpty() throws Exception {
    +    final PhysicalOperator hashAgg = new HashAggregate(null, AggPrelBase.OperatorPhase.PHASE_1of1, parseExprs("a", "a"), parseExprs("sum(b)", "b_sum"), 1.0f);
    +    testSingleInputEmptyBatchHandling(hashAgg);
    +  }
    +
    +  @Test
    +  public void testStreamingAggEmpty() throws Exception {
    +    final PhysicalOperator hashAgg = new StreamingAggregate(null, parseExprs("a", "a"), parseExprs("sum(b)", "b_sum"), 1.0f);
    +    testSingleInputEmptyBatchHandling(hashAgg);
    +  }
    +
    +  @Test
    +  public void testSortEmpty() throws Exception {
    +    final PhysicalOperator sort = new ExternalSort(null,
    +        Lists.newArrayList(ordering("b", RelFieldCollation.Direction.ASCENDING, RelFieldCollation.NullDirection.FIRST)), false);
    +    testSingleInputEmptyBatchHandling(sort);
    +  }
    +
    +  @Test
    +  public void testLimitEmpty() throws Exception {
    +    final PhysicalOperator limit = new Limit(null, 10, 5);
    +    testSingleInputEmptyBatchHandling(limit);
    +  }
    +
    +  @Test
    +  public void testFlattenEmpty() throws Exception {
    +    final PhysicalOperator flatten = new FlattenPOP(null, SchemaPath.getSimplePath("col1"));
    +    testSingleInputEmptyBatchHandling(flatten);
    +  }
    +
    +  @Test
    +  public void testUnionEmptyBoth() throws Exception {
    +    final PhysicalOperator unionAll = new UnionAll(Collections.EMPTY_LIST); // Children list is provided through RecordBatch
    +    testTwoInputEmptyBatchHandling(unionAll);
    +  }
    +
    +  @Test
    +  public void testHashJoinEmptyBoth() throws Exception {
    +   final PhysicalOperator join = new HashJoinPOP(null, null, Lists.newArrayList(joinCond("a", "EQUALS", "b")), JoinRelType.INNER);
    +    testTwoInputEmptyBatchHandling(join);
    +  }
    +
    +  @Test
    +  public void testLeftHashJoinEmptyBoth() throws Exception {
    +    final PhysicalOperator join = new HashJoinPOP(null, null, Lists.newArrayList(joinCond("a", "EQUALS", "b")), JoinRelType.LEFT);
    +    testTwoInputEmptyBatchHandling(join);
    +  }
    +
    +  @Test
    +  public void testRightHashJoinEmptyBoth() throws Exception {
    +    final PhysicalOperator join = new HashJoinPOP(null, null, Lists.newArrayList(joinCond("a", "EQUALS", "b")), JoinRelType.RIGHT);
    +    testTwoInputEmptyBatchHandling(join);
    +  }
    +
    +  @Test
    +  public void testFullHashJoinEmptyBoth() throws Exception {
    +    final PhysicalOperator join = new HashJoinPOP(null, null, Lists.newArrayList(joinCond("a", "EQUALS", "b")), JoinRelType.FULL);
    +    testTwoInputEmptyBatchHandling(join);
    +  }
    +
    +  @Test
    +  public void testMergeJoinEmptyBoth() throws Exception {
    +    final PhysicalOperator join = new MergeJoinPOP(null, null, Lists.newArrayList(joinCond("a", "EQUALS", "b")), JoinRelType.INNER);
    +    testTwoInputEmptyBatchHandling(join);
    +  }
    +
    +  @Test
    +  public void testLeftMergeJoinEmptyBoth() throws Exception {
    +    final PhysicalOperator join = new MergeJoinPOP(null, null, Lists.newArrayList(joinCond("a", "EQUALS", "b")), JoinRelType.LEFT);
    +    testTwoInputEmptyBatchHandling(join);
    +  }
    +
    +  @Test
    +  public void testRightMergeJoinEmptyBoth() throws Exception {
    +    final PhysicalOperator join = new MergeJoinPOP(null, null, Lists.newArrayList(joinCond("a", "EQUALS", "b")), JoinRelType.RIGHT);
    +    testTwoInputEmptyBatchHandling(join);
    +  }
    +
    +  @Test
    +  @Ignore("Full Merge join is not supported.")
    +  public void testFullMergeJoinEmptyBoth() throws Exception {
    +    final PhysicalOperator join = new MergeJoinPOP(null, null, Lists.newArrayList(joinCond("a", "EQUALS", "b")), JoinRelType.FULL);
    +    testTwoInputEmptyBatchHandling(join);
    +  }
    +
    +  @Test
    +  public void testUnionLeftEmtpy() throws Exception {
    +    final PhysicalOperator unionAll = new UnionAll(Collections.EMPTY_LIST); // Children list is provided through RecordBatch
    +
    +    RecordBatch left = createEmptyBatchFromJson();
    +
    +    String file = FileUtils.getResourceAsFile("/tpchmulti/region/01.parquet").toURI().toString();
    +
    +    RecordBatch scanBatch = new ParquetScanBuilder()
    +        .fileSystem(fs)
    +        .columnsToRead("R_REGIONKEY")
    +        .inputPaths(Lists.newArrayList(file))
    +        .build();
    +
    +    RecordBatch projectBatch = new PopBuilder()
    +        .physicalOperator(new Project(parseExprs("R_REGIONKEY+10", "regionkey"), null))
    +        .addInput(scanBatch)
    +        .build();
    +
    +    RecordBatch unionBatch = new PopBuilder()
    +        .physicalOperator(unionAll)
    +        .addInput(left)
    +        .addInput(projectBatch)
    +        .build();
    +
    +    BatchSchema expectedSchema = new SchemaBuilder()
    +        .add("regionkey", TypeProtos.MinorType.BIGINT)
    +        .build();
    +
    +    new MiniPlanTestBuilder()
    +        .root(unionBatch)
    +        .expectSchema(expectedSchema)
    +        .baselineValues(10L)
    +        .baselineValues(11L)
    +        .go();
    +  }
    +
    +
    +  @Test
    +  public void testHashJoinLeftEmpty() throws Exception {
    +    RecordBatch left = createEmptyBatchFromJson();
    +
    +    List<String> rightJsonBatches = Lists.newArrayList(
    +        "[{\"a\": 50, \"b\" : 10 }]");
    +
    +    RecordBatch rightScan = new JsonScanBuilder()
    +        .jsonBatches(rightJsonBatches)
    +        .columnsToRead("a", "b")
    +        .build();
    +
    +    RecordBatch joinBatch = new PopBuilder()
    +        .physicalOperator(new HashJoinPOP(null, null, Lists.newArrayList(joinCond("a", "EQUALS", "a2")), JoinRelType.INNER))
    +        .addInput(left)
    +        .addInput(rightScan)
    +        .build();
    +
    +    BatchSchema expectedSchema = new SchemaBuilder()
    +        .addNullable("a", TypeProtos.MinorType.BIGINT)
    +        .addNullable("b", TypeProtos.MinorType.BIGINT)
    +        .withSVMode(BatchSchema.SelectionVectorMode.NONE)
    +        .build();
    +
    +    new MiniPlanTestBuilder()
    +        .root(joinBatch)
    +        .expectSchema(expectedSchema)
    +        .expectZeroRow(true)
    +        .go();
    +  }
    +
    +  @Test
    +  public void testHashJoinRightEmpty() throws Exception {
    +    List<String> leftJsonBatches = Lists.newArrayList(
    +        "[{\"a\": 50, \"b\" : 10 }]");
    +
    +    RecordBatch leftScan = new JsonScanBuilder()
    +        .jsonBatches(leftJsonBatches)
    +        .columnsToRead("a", "b")
    +        .build();
    +
    +    RecordBatch right = createEmptyBatchFromJson();
    +
    +    RecordBatch joinBatch = new PopBuilder()
    +        .physicalOperator(new HashJoinPOP(null, null, Lists.newArrayList(joinCond("a", "EQUALS", "a2")), JoinRelType.INNER))
    +        .addInput(leftScan)
    +        .addInput(right)
    +        .build();
    +
    +    BatchSchema expectedSchema = new SchemaBuilder()
    +        .addNullable("a", TypeProtos.MinorType.BIGINT)
    +        .addNullable("b", TypeProtos.MinorType.BIGINT)
    +        .withSVMode(BatchSchema.SelectionVectorMode.NONE)
    +        .build();
    +
    +    new MiniPlanTestBuilder()
    +        .root(joinBatch)
    +        .expectSchema(expectedSchema)
    +        .expectZeroRow(true)
    +        .go();
    +  }
    +
    +
    +  @Test
    +  public void testLeftHashJoinLeftEmpty() throws Exception {
    +    RecordBatch left = createEmptyBatchFromJson();
    +
    +    List<String> rightJsonBatches = Lists.newArrayList(
    +        "[{\"a\": 50, \"b\" : 10 }]");
    +
    +    RecordBatch rightScan = new JsonScanBuilder()
    +        .jsonBatches(rightJsonBatches)
    +        .columnsToRead("a", "b")
    +        .build();
    +
    +    RecordBatch joinBatch = new PopBuilder()
    +        .physicalOperator(new HashJoinPOP(null, null, Lists.newArrayList(joinCond("a", "EQUALS", "a2")), JoinRelType.LEFT))
    +        .addInput(left)
    +        .addInput(rightScan)
    +        .build();
    +
    +    BatchSchema expectedSchema = new SchemaBuilder()
    +        .addNullable("a", TypeProtos.MinorType.BIGINT)
    +        .addNullable("b", TypeProtos.MinorType.BIGINT)
    +        .withSVMode(BatchSchema.SelectionVectorMode.NONE)
    +        .build();
    +
    +    new MiniPlanTestBuilder()
    +        .root(joinBatch)
    +        .expectSchema(expectedSchema)
    +        .expectZeroRow(true)
    +        .go();
    +  }
    +
    +  @Test
    +  public void testLeftHashJoinRightEmpty() throws Exception {
    +    List<String> leftJsonBatches = Lists.newArrayList(
    +        "[{\"a\": 50, \"b\" : 10 }]");
    +
    +    RecordBatch leftScan = new JsonScanBuilder()
    +        .jsonBatches(leftJsonBatches)
    +        .columnsToRead("a", "b")
    +        .build();
    +
    +    RecordBatch right = createEmptyBatchFromJson();
    +
    +    RecordBatch joinBatch = new PopBuilder()
    +        .physicalOperator(new HashJoinPOP(null, null, Lists.newArrayList(joinCond("a", "EQUALS", "a2")), JoinRelType.LEFT))
    +        .addInput(leftScan)
    +        .addInput(right)
    +        .build();
    +
    +    BatchSchema expectedSchema = new SchemaBuilder()
    +        .addNullable("a", TypeProtos.MinorType.BIGINT)
    +        .addNullable("b", TypeProtos.MinorType.BIGINT)
    +        .withSVMode(BatchSchema.SelectionVectorMode.NONE)
    +        .build();
    +
    +    new MiniPlanTestBuilder()
    +        .root(joinBatch)
    +        .expectSchema(expectedSchema)
    +        .baselineValues(50L, 10L)
    +        .go();
    +  }
    +
    +  @Test
    +  public void testUnionFilterAll() throws Exception {
    +    List<String> leftJsonBatches = Lists.newArrayList(
    +        "[{\"a\": 5, \"b\" : 1 }]");
    +
    +    List<String> rightJsonBatches = Lists.newArrayList(
    +        "[{\"a\": 50, \"b\" : 10 }]");
    +
    +    RecordBatch leftScan = new JsonScanBuilder()
    +        .jsonBatches(leftJsonBatches)
    +        .columnsToRead("a", "b")
    +        .build();
    +
    +    RecordBatch leftFilter = new PopBuilder()
    +        .physicalOperator(new Filter(null, parseExpr("a < 0"), 1.0f))
    +        .addInput(leftScan)
    +        .build();
    +
    +    RecordBatch rightScan = new JsonScanBuilder()
    +        .jsonBatches(rightJsonBatches)
    +        .columnsToRead("a", "b")
    +        .build();
    +
    +    RecordBatch rightFilter = new PopBuilder()
    +        .physicalOperator(new Filter(null, parseExpr("a < 0"), 1.0f))
    +        .addInput(rightScan)
    +        .build();
    +
    +    RecordBatch batch = new PopBuilder()
    +        .physicalOperator(new UnionAll(Collections.EMPTY_LIST)) // Children list is provided through RecordBatch
    +        .addInput(leftFilter)
    +        .addInput(rightFilter)
    +        .build();
    +
    +    BatchSchema expectedSchema = new SchemaBuilder()
    +        .addNullable("a", TypeProtos.MinorType.BIGINT)
    +        .addNullable("b", TypeProtos.MinorType.BIGINT)
    +        .withSVMode(BatchSchema.SelectionVectorMode.NONE)
    +        .build();
    +
    +    new MiniPlanTestBuilder()
    +        .root(batch)
    +        .expectSchema(expectedSchema)
    +        .expectZeroRow(true)
    +        .go();
    +  }
    +
    +  @Test
    +  public void testOutputProjectEmpty() throws Exception {
    --- End diff --
    
    Add unit case for *, foo.*, bar.*, map and array reference.
    
    I did not cover the case of error in expression tree materializer. If you have an example to produce error in tree materializer, maybe you can share it here. On the other hand, as explained above, the code path is an existing code block. 


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] drill pull request #906: DRILL-5546: Handle schema change exception failure ...

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/906#discussion_r135578936
  
    --- Diff: exec/vector/src/main/java/org/apache/drill/exec/vector/UntypedNullHolder.java ---
    @@ -0,0 +1,50 @@
    +/*
    + * 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;
    +
    +import org.apache.drill.common.types.TypeProtos;
    +import org.apache.drill.common.types.Types;
    +import org.apache.drill.exec.expr.holders.ValueHolder;
    +
    +public class UntypedNullHolder implements ValueHolder {
    +  public static final TypeProtos.MajorType TYPE = Types.optional(TypeProtos.MinorType.NULL);
    +
    +  public TypeProtos.MajorType getType() {return TYPE;}
    +
    +  public static final int WIDTH = 0;
    +
    +  public int isSet = 1;
    +
    +  @Deprecated
    +  public int hashCode(){
    +    throw new UnsupportedOperationException();
    +  }
    --- End diff --
    
    See note for `toString()`. This has the same issues.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] drill pull request #906: DRILL-5546: Handle schema change exception failure ...

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

    https://github.com/apache/drill/pull/906#discussion_r135351183
  
    --- Diff: exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/ScanBatch.java ---
    @@ -152,97 +157,75 @@ public void kill(boolean sendUpstream) {
         }
       }
     
    -  private void releaseAssets() {
    -    container.zeroVectors();
    -  }
    -
    -  private void clearFieldVectorMap() {
    -    for (final ValueVector v : mutator.fieldVectorMap().values()) {
    -      v.clear();
    -    }
    -  }
    -
       @Override
       public IterOutcome next() {
         if (done) {
           return IterOutcome.NONE;
         }
         oContext.getStats().startProcessing();
         try {
    -      try {
    -        injector.injectChecked(context.getExecutionControls(), "next-allocate", OutOfMemoryException.class);
    -
    -        currentReader.allocate(mutator.fieldVectorMap());
    -      } catch (OutOfMemoryException e) {
    -        clearFieldVectorMap();
    -        throw UserException.memoryError(e).build(logger);
    -      }
    -      while ((recordCount = currentReader.next()) == 0) {
    +      while (true) {
    --- End diff --
    
    It's not entirely true that the original logic allocates a batch once per call to next().  There are two places where allocate were called. The first one happens outside of the while loop [1], while the second happens inside the loop for the case where we want to skip certain record readers [2].
    
    The new logic essentially combine the two into one, by putting the first one inside the loop as well. In that sense, it's not doing a completely job than the previous logic.
    
    Regarding where the buffers are allocated if there are two passes,  each value vector releases its own buffer by calling clear(), before it allocate new buffer.  That's why the previous/new logic did not run into memory leak problem. 
     
    
    1. https://github.com/apache/drill/blob/master/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/ScanBatch.java#L175
    2. https://github.com/apache/drill/blob/master/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/ScanBatch.java#L214


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] drill pull request #906: DRILL-5546: Handle schema change exception failure ...

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/906#discussion_r135366734
  
    --- Diff: exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/ScanBatch.java ---
    @@ -152,97 +157,75 @@ public void kill(boolean sendUpstream) {
         }
       }
     
    -  private void releaseAssets() {
    -    container.zeroVectors();
    -  }
    -
    -  private void clearFieldVectorMap() {
    -    for (final ValueVector v : mutator.fieldVectorMap().values()) {
    -      v.clear();
    -    }
    -  }
    -
       @Override
       public IterOutcome next() {
         if (done) {
           return IterOutcome.NONE;
         }
         oContext.getStats().startProcessing();
         try {
    -      try {
    -        injector.injectChecked(context.getExecutionControls(), "next-allocate", OutOfMemoryException.class);
    -
    -        currentReader.allocate(mutator.fieldVectorMap());
    -      } catch (OutOfMemoryException e) {
    -        clearFieldVectorMap();
    -        throw UserException.memoryError(e).build(logger);
    -      }
    -      while ((recordCount = currentReader.next()) == 0) {
    +      while (true) {
    --- End diff --
    
    Thanks for the explanation!


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] drill pull request #906: DRILL-5546: Handle schema change exception failure ...

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/906#discussion_r134297885
  
    --- Diff: exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/ScanBatch.java ---
    @@ -152,97 +157,75 @@ public void kill(boolean sendUpstream) {
         }
       }
     
    -  private void releaseAssets() {
    -    container.zeroVectors();
    -  }
    -
    -  private void clearFieldVectorMap() {
    -    for (final ValueVector v : mutator.fieldVectorMap().values()) {
    -      v.clear();
    -    }
    -  }
    -
       @Override
       public IterOutcome next() {
         if (done) {
           return IterOutcome.NONE;
         }
         oContext.getStats().startProcessing();
         try {
    -      try {
    -        injector.injectChecked(context.getExecutionControls(), "next-allocate", OutOfMemoryException.class);
    -
    -        currentReader.allocate(mutator.fieldVectorMap());
    -      } catch (OutOfMemoryException e) {
    -        clearFieldVectorMap();
    -        throw UserException.memoryError(e).build(logger);
    -      }
    -      while ((recordCount = currentReader.next()) == 0) {
    +      while (true) {
             try {
    -          if (!readers.hasNext()) {
    -            // We're on the last reader, and it has no (more) rows.
    -            currentReader.close();
    -            releaseAssets();
    -            done = true;  // have any future call to next() return NONE
    -
    -            if (mutator.isNewSchema()) {
    -              // This last reader has a new schema (e.g., we have a zero-row
    -              // file or other source).  (Note that some sources have a non-
    -              // null/non-trivial schema even when there are no rows.)
    +          injector.injectChecked(context.getExecutionControls(), "next-allocate", OutOfMemoryException.class);
    +          currentReader.allocate(mutator.fieldVectorMap());
    +        } catch (OutOfMemoryException e) {
    +          clearFieldVectorMap();
    +          throw UserException.memoryError(e).build(logger);
    +        }
     
    -              container.buildSchema(SelectionVectorMode.NONE);
    -              schema = container.getSchema();
    +        recordCount = currentReader.next();
    +        Preconditions.checkArgument(recordCount >= 0,
    +            "recordCount from RecordReader.next() should not be negative");
     
    -              return IterOutcome.OK_NEW_SCHEMA;
    -            }
    -            return IterOutcome.NONE;
    -          }
    -          // At this point, the reader that hit its end is not the last reader.
    +        boolean isNewRegularSchema = mutator.isNewSchema();
    +        // We should skip the reader, when recordCount = 0 && ! isNewRegularSchema.
    +        // Add/set implicit column vectors, only when reader gets > 0 row, or
    +        // when reader gets 0 row but with a schema with new field added
    +        if (recordCount > 0 || isNewRegularSchema) {
    +          addImplicitVectors();
    +          populateImplicitVectors();
    +        }
     
    -          // If all the files we have read so far are just empty, the schema is not useful
    -          if (! hasReadNonEmptyFile) {
    -            container.clear();
    -            clearFieldVectorMap();
    -            mutator.clear();
    -          }
    +        boolean isNewImplicitSchema = mutator.isNewSchema();
    +        for (VectorWrapper<?> w : container) {
    +          w.getValueVector().getMutator().setValueCount(recordCount);
    +        }
    +        final boolean isNewSchema = isNewRegularSchema || isNewImplicitSchema;
    +        oContext.getStats().batchReceived(0, recordCount, isNewSchema);
     
    +        if (recordCount == 0) {
               currentReader.close();
    -          currentReader = readers.next();
    -          implicitValues = implicitColumns.hasNext() ? implicitColumns.next() : null;
    -          currentReader.setup(oContext, mutator);
    -          try {
    -            currentReader.allocate(mutator.fieldVectorMap());
    -          } catch (OutOfMemoryException e) {
    -            clearFieldVectorMap();
    -            throw UserException.memoryError(e).build(logger);
    +          if (isNewSchema) {
    +            // current reader presents a new schema in mutator even though it has 0 row.
    +            // This could happen when data sources have a non-trivial schema with 0 row.
    +            container.buildSchema(SelectionVectorMode.NONE);
    +            schema = container.getSchema();
    +            if (readers.hasNext()) {
    +              advanceNextReader();
    +            } else {
    +              done = true;  // indicates the follow-up next() call will return IterOutcome.NONE.
    +            }
    +            return IterOutcome.OK_NEW_SCHEMA;
    +          } else { // not a new schema
    +            if (readers.hasNext()) {
    +              advanceNextReader();
    +              continue; // skip reader returning 0 row and having same schema.
    --- End diff --
    
    Per comment above: we have a payload of allocated vectors. Which code releases that memory before we allocate again?


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] drill pull request #906: DRILL-5546: Handle schema change exception failure ...

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/906#discussion_r135577968
  
    --- Diff: exec/java-exec/src/test/java/org/apache/drill/TestEmptyBatchSql.java ---
    @@ -0,0 +1,124 @@
    +/*
    + * 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;
    +
    +import com.google.common.collect.Lists;
    +import org.apache.commons.lang3.tuple.Pair;
    +import org.apache.drill.common.expression.SchemaPath;
    +import org.apache.drill.common.types.TypeProtos;
    +import org.apache.drill.common.util.FileUtils;
    +import org.junit.Test;
    +
    +import java.util.List;
    +
    +public class TestEmptyBatchSql extends  BaseTestQuery {
    --- End diff --
    
    Nice set of tests for JSON. Thanks!


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] drill pull request #906: DRILL-5546: Handle schema change exception failure ...

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

    https://github.com/apache/drill/pull/906#discussion_r135424127
  
    --- Diff: exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/ScanBatch.java ---
    @@ -152,97 +157,75 @@ public void kill(boolean sendUpstream) {
         }
       }
     
    -  private void releaseAssets() {
    -    container.zeroVectors();
    -  }
    -
    -  private void clearFieldVectorMap() {
    -    for (final ValueVector v : mutator.fieldVectorMap().values()) {
    -      v.clear();
    -    }
    -  }
    -
       @Override
       public IterOutcome next() {
         if (done) {
           return IterOutcome.NONE;
         }
         oContext.getStats().startProcessing();
         try {
    -      try {
    -        injector.injectChecked(context.getExecutionControls(), "next-allocate", OutOfMemoryException.class);
    -
    -        currentReader.allocate(mutator.fieldVectorMap());
    -      } catch (OutOfMemoryException e) {
    -        clearFieldVectorMap();
    -        throw UserException.memoryError(e).build(logger);
    -      }
    -      while ((recordCount = currentReader.next()) == 0) {
    +      while (true) {
             try {
    -          if (!readers.hasNext()) {
    -            // We're on the last reader, and it has no (more) rows.
    -            currentReader.close();
    -            releaseAssets();
    -            done = true;  // have any future call to next() return NONE
    -
    -            if (mutator.isNewSchema()) {
    -              // This last reader has a new schema (e.g., we have a zero-row
    -              // file or other source).  (Note that some sources have a non-
    -              // null/non-trivial schema even when there are no rows.)
    +          injector.injectChecked(context.getExecutionControls(), "next-allocate", OutOfMemoryException.class);
    +          currentReader.allocate(mutator.fieldVectorMap());
    +        } catch (OutOfMemoryException e) {
    +          clearFieldVectorMap();
    +          throw UserException.memoryError(e).build(logger);
    +        }
     
    -              container.buildSchema(SelectionVectorMode.NONE);
    -              schema = container.getSchema();
    +        recordCount = currentReader.next();
    +        Preconditions.checkArgument(recordCount >= 0,
    +            "recordCount from RecordReader.next() should not be negative");
     
    -              return IterOutcome.OK_NEW_SCHEMA;
    -            }
    -            return IterOutcome.NONE;
    -          }
    -          // At this point, the reader that hit its end is not the last reader.
    +        boolean isNewRegularSchema = mutator.isNewSchema();
    +        // We should skip the reader, when recordCount = 0 && ! isNewRegularSchema.
    +        // Add/set implicit column vectors, only when reader gets > 0 row, or
    +        // when reader gets 0 row but with a schema with new field added
    +        if (recordCount > 0 || isNewRegularSchema) {
    +          addImplicitVectors();
    +          populateImplicitVectors();
    +        }
     
    -          // If all the files we have read so far are just empty, the schema is not useful
    -          if (! hasReadNonEmptyFile) {
    -            container.clear();
    -            clearFieldVectorMap();
    -            mutator.clear();
    -          }
    +        boolean isNewImplicitSchema = mutator.isNewSchema();
    +        for (VectorWrapper<?> w : container) {
    +          w.getValueVector().getMutator().setValueCount(recordCount);
    +        }
    +        final boolean isNewSchema = isNewRegularSchema || isNewImplicitSchema;
    +        oContext.getStats().batchReceived(0, recordCount, isNewSchema);
     
    +        if (recordCount == 0) {
               currentReader.close();
    -          currentReader = readers.next();
    -          implicitValues = implicitColumns.hasNext() ? implicitColumns.next() : null;
    -          currentReader.setup(oContext, mutator);
    -          try {
    -            currentReader.allocate(mutator.fieldVectorMap());
    -          } catch (OutOfMemoryException e) {
    -            clearFieldVectorMap();
    -            throw UserException.memoryError(e).build(logger);
    +          if (isNewSchema) {
    +            // current reader presents a new schema in mutator even though it has 0 row.
    +            // This could happen when data sources have a non-trivial schema with 0 row.
    +            container.buildSchema(SelectionVectorMode.NONE);
    +            schema = container.getSchema();
    +            if (readers.hasNext()) {
    +              advanceNextReader();
    +            } else {
    +              done = true;  // indicates the follow-up next() call will return IterOutcome.NONE.
    +            }
    +            return IterOutcome.OK_NEW_SCHEMA;
    +          } else { // not a new schema
    +            if (readers.hasNext()) {
    +              advanceNextReader();
    +              continue; // skip reader returning 0 row and having same schema.
    --- End diff --
    
    As explained above, both the original code and new code rely on value vector free existing buffer before allocate again. 


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] drill pull request #906: DRILL-5546: Handle schema change exception failure ...

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

    https://github.com/apache/drill/pull/906#discussion_r135974928
  
    --- Diff: exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/project/ProjectRecordBatch.java ---
    @@ -768,4 +765,73 @@ else if (exprHasPrefix && refHasPrefix) {
           }
         }
       }
    +
    +  /**
    +   * handle FAST NONE specially when Project for query output. This happens when input returns a
    +   * FAST NONE directly ( input does not return any batch with schema/data).
    +   *
    +   * Project operator has to return a batch with schema derived using the following 3 rules:
    +   *  Case 1:  *  ==>  expand into an empty list of columns.
    +   *  Case 2:  regular column reference ==> treat as nullable-int column
    +   *  Case 3:  expressions => Call ExpressionTreeMaterialization over an empty vector contain.
    +   *           Once the expression is materialized without error, use the output type of materialized
    +   *           expression.
    +   * The batch is constructed with the above rules, and recordCount = 0.
    +   * Returned with OK_NEW_SCHEMA to down-stream operator.
    +   */
    +  @Override
    +  protected IterOutcome handleFastNone() {
    +    if (! popConfig.isOutputProj()) {
    +      return super.handleFastNone();
    +    }
    +
    +    allocationVectors = new ArrayList<>();
    +    final List<NamedExpression> exprs = getExpressionList();
    +    final ErrorCollector collector = new ErrorCollectorImpl();
    +    VectorContainer fakedIncomingVC = new VectorContainer();
    +
    +    for (NamedExpression namedExpression : exprs) {
    +      if (namedExpression.getExpr() instanceof SchemaPath) {
    +        final NameSegment expr = ((SchemaPath) namedExpression.getExpr()).getRootSegment();
    +        if (expr.getPath().contains(StarColumnHelper.STAR_COLUMN)) {
    +          continue; // * would expand into an empty list.
    +        } else {
    +          final TypeProtos.MajorType majorType = TypeProtos.MajorType.newBuilder()
    +              .setMinorType(MinorType.INT)
    +              .setMode(TypeProtos.DataMode.OPTIONAL)
    +              .build();
    +
    +          MaterializedField outputField = MaterializedField.create(namedExpression.getRef().getRootSegment().getPath(), majorType);
    +          final ValueVector vv = container.addOrGet(outputField, callBack);
    +          allocationVectors.add(vv);
    +        }
    +        continue;
    +      }
    +
    +      final LogicalExpression materializedExpr = ExpressionTreeMaterializer.materialize(namedExpression.getExpr(),
    +          fakedIncomingVC,
    +          collector,
    +          context.getFunctionRegistry(),
    +          true,
    +          unionTypeEnabled);
    +
    +      if (collector.hasErrors()) {
    +        throw new IllegalArgumentException(String.format("Failure while trying to materialize expressions : %s.  Errors:\n %s.",
    +            namedExpression.getExpr(),
    +            collector.toErrorString()));
    +      }
    --- End diff --
    
    This code has been removed in revised patch.
    



---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] drill pull request #906: DRILL-5546: Handle schema change exception failure ...

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/906#discussion_r136439631
  
    --- Diff: exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/ScanBatch.java ---
    @@ -329,9 +326,11 @@ public TypedFieldId getValueVectorId(SchemaPath path) {
     
       @VisibleForTesting
       public static class Mutator implements OutputMutator {
    -    /** Whether schema has changed since last inquiry (via #isNewSchema}).  Is
    -     *  true before first inquiry. */
    -    private boolean schemaChanged = true;
    +    /** Flag keeping track whether top-level schema has changed since last inquiry (via #isNewSchema}).
    +     * It's initialized to false, or reset to false after #isNewSchema or after #clear, until a new value vector
    +     * or a value vector with different type is added to fieldVectorMap.
    +     **/
    +    private boolean schemaChanged;
    --- End diff --
    
    Thanks for the explanation.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] drill pull request #906: DRILL-5546: Handle schema change exception failure ...

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

    https://github.com/apache/drill/pull/906#discussion_r135979350
  
    --- Diff: exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/union/UnionAllRecordBatch.java ---
    @@ -130,562 +145,248 @@ public IterOutcome innerNext() {
       }
     
       @Override
    -  public WritableBatch getWritableBatch() {
    -    return WritableBatch.get(this);
    +  public int getRecordCount() {
    +    return recordCount;
       }
     
    -  private void setValueCount(int count) {
    -    for (ValueVector v : allocationVectors) {
    -      ValueVector.Mutator m = v.getMutator();
    -      m.setValueCount(count);
    -    }
    -  }
     
    -  private boolean doAlloc() {
    -    for (ValueVector v : allocationVectors) {
    -      try {
    -        AllocationHelper.allocateNew(v, current.getRecordCount());
    -      } catch (OutOfMemoryException ex) {
    -        return false;
    -      }
    +  @SuppressWarnings("resource")
    +  private IterOutcome doWork(RecordBatch inputBatch, boolean newSchema) throws ClassTransformationException, IOException, SchemaChangeException {
    +    if (inputBatch.getSchema().getFieldCount() != container.getSchema().getFieldCount()) {
    +      // wrong.
    --- End diff --
    
    The revised patch will throw  IllegalArgumentException. 


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] drill pull request #906: DRILL-5546: Handle schema change exception failure ...

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

    https://github.com/apache/drill/pull/906#discussion_r136157172
  
    --- Diff: exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/union/UnionAllRecordBatch.java ---
    @@ -130,562 +145,248 @@ public IterOutcome innerNext() {
       }
     
       @Override
    -  public WritableBatch getWritableBatch() {
    -    return WritableBatch.get(this);
    +  public int getRecordCount() {
    +    return recordCount;
       }
     
    -  private void setValueCount(int count) {
    -    for (ValueVector v : allocationVectors) {
    -      ValueVector.Mutator m = v.getMutator();
    -      m.setValueCount(count);
    -    }
    -  }
     
    -  private boolean doAlloc() {
    -    for (ValueVector v : allocationVectors) {
    -      try {
    -        AllocationHelper.allocateNew(v, current.getRecordCount());
    -      } catch (OutOfMemoryException ex) {
    -        return false;
    -      }
    +  @SuppressWarnings("resource")
    +  private IterOutcome doWork(RecordBatch inputBatch, boolean newSchema) throws ClassTransformationException, IOException, SchemaChangeException {
    +    if (inputBatch.getSchema().getFieldCount() != container.getSchema().getFieldCount()) {
    +      // wrong.
         }
    -    return true;
    -  }
     
    -  @SuppressWarnings("resource")
    -  private IterOutcome doWork() throws ClassTransformationException, IOException, SchemaChangeException {
    -    if (allocationVectors != null) {
    -      for (ValueVector v : allocationVectors) {
    -        v.clear();
    -      }
    +    if (newSchema) {
    +      createUnionAller(inputBatch);
         }
     
    -    allocationVectors = Lists.newArrayList();
    -    transfers.clear();
    +    container.zeroVectors();
     
    -    // If both sides of Union-All are empty
    -    if(unionAllInput.isBothSideEmpty()) {
    -      for(int i = 0; i < outputFields.size(); ++i) {
    -        final String colName = outputFields.get(i).getPath();
    -        final MajorType majorType = MajorType.newBuilder()
    -            .setMinorType(MinorType.INT)
    -            .setMode(DataMode.OPTIONAL)
    -            .build();
    -
    -        MaterializedField outputField = MaterializedField.create(colName, majorType);
    -        ValueVector vv = container.addOrGet(outputField, callBack);
    -        allocationVectors.add(vv);
    -      }
    +    for (final ValueVector v : this.allocationVectors) {
    +      AllocationHelper.allocateNew(v, inputBatch.getRecordCount());
    +    }
     
    -      container.buildSchema(BatchSchema.SelectionVectorMode.NONE);
    +    recordCount = unionall.unionRecords(0, inputBatch.getRecordCount(), 0);
    +    for (final ValueVector v : allocationVectors) {
    +      final ValueVector.Mutator m = v.getMutator();
    +      m.setValueCount(recordCount);
    +    }
    +
    +    if (callBack.getSchemaChangedAndReset()) {
           return IterOutcome.OK_NEW_SCHEMA;
    +    } else {
    +      return IterOutcome.OK;
         }
    +  }
    +
    +  private void createUnionAller(RecordBatch inputBatch) throws ClassTransformationException, IOException, SchemaChangeException {
    +    transfers.clear();
    +    allocationVectors.clear();;
     
         final ClassGenerator<UnionAller> cg = CodeGenerator.getRoot(UnionAller.TEMPLATE_DEFINITION, context.getFunctionRegistry(), context.getOptions());
         cg.getCodeGenerator().plainJavaCapable(true);
         // Uncomment out this line to debug the generated code.
    -//    cg.getCodeGenerator().saveCodeForDebugging(true);
    +    //    cg.getCodeGenerator().saveCodeForDebugging(true);
    +
         int index = 0;
    -    for(VectorWrapper<?> vw : current) {
    -       ValueVector vvIn = vw.getValueVector();
    -      // get the original input column names
    -      SchemaPath inputPath = SchemaPath.getSimplePath(vvIn.getField().getPath());
    -      // get the renamed column names
    -      SchemaPath outputPath = SchemaPath.getSimplePath(outputFields.get(index).getPath());
    +    for(VectorWrapper<?> vw : inputBatch) {
    +      ValueVector vvIn = vw.getValueVector();
    +      ValueVector vvOut = container.getValueVector(index).getValueVector();
     
           final ErrorCollector collector = new ErrorCollectorImpl();
           // According to input data names, Minortypes, Datamodes, choose to
           // transfer directly,
           // rename columns or
           // cast data types (Minortype or DataMode)
    -      if (hasSameTypeAndMode(outputFields.get(index), vw.getValueVector().getField())) {
    +      if (hasSameTypeAndMode(container.getSchema().getColumn(index), vvIn.getField())
    +          && vvIn.getField().getType().getMinorType() != TypeProtos.MinorType.MAP // Per DRILL-5521, existing bug for map transfer
    +          ) {
             // Transfer column
    -
    -        MajorType outputFieldType = outputFields.get(index).getType();
    -        MaterializedField outputField = MaterializedField.create(outputPath.getAsUnescapedPath(), outputFieldType);
    -
    -        /*
    -          todo: Fix if condition when DRILL-4824 is merged
    -          If condition should be changed to:
    -          `if (outputFields.get(index).getPath().equals(inputPath.getAsUnescapedPath())) {`
    -          DRILL-5419 has changed condition to correct one but this caused regression (DRILL-5521).
    -          Root cause is missing indication of child column in map types when it is null.
    -          DRILL-4824 is re-working json reader implementation, including map types and will fix this problem.
    -          Reverting condition to previous one to avoid regression till DRILL-4824 is merged.
    -          Unit test - TestJsonReader.testKvgenWithUnionAll().
    -         */
    -        if (outputFields.get(index).getPath().equals(inputPath)) {
    -          ValueVector vvOut = container.addOrGet(outputField);
    -          TransferPair tp = vvIn.makeTransferPair(vvOut);
    -          transfers.add(tp);
    +        TransferPair tp = vvIn.makeTransferPair(vvOut);
    +        transfers.add(tp);
             // Copy data in order to rename the column
    -        } else {
    -          final LogicalExpression expr = ExpressionTreeMaterializer.materialize(inputPath, current, collector, context.getFunctionRegistry() );
    -          if (collector.hasErrors()) {
    -            throw new SchemaChangeException(String.format("Failure while trying to materialize incoming schema.  Errors:\n %s.", collector.toErrorString()));
    -          }
    -
    -          ValueVector vv = container.addOrGet(outputField, callBack);
    -          allocationVectors.add(vv);
    -          TypedFieldId fid = container.getValueVectorId(SchemaPath.getSimplePath(outputField.getPath()));
    -          ValueVectorWriteExpression write = new ValueVectorWriteExpression(fid, expr, true);
    -          cg.addExpr(write);
    -        }
    -      // Cast is necessary
    +      } else if (vvIn.getField().getType().getMinorType() == TypeProtos.MinorType.NULL) {
    --- End diff --
    
    The comment is for the `else` branch, and should be moved a couple lines down. For rename a column, I do not think we are doing a copy; transfer should be good enough. 



---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] drill pull request #906: DRILL-5546: Handle schema change exception failure ...

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/906#discussion_r135369535
  
    --- Diff: exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/union/UnionAllRecordBatch.java ---
    @@ -130,562 +145,248 @@ public IterOutcome innerNext() {
       }
     
       @Override
    -  public WritableBatch getWritableBatch() {
    -    return WritableBatch.get(this);
    +  public int getRecordCount() {
    +    return recordCount;
       }
     
    -  private void setValueCount(int count) {
    -    for (ValueVector v : allocationVectors) {
    -      ValueVector.Mutator m = v.getMutator();
    -      m.setValueCount(count);
    -    }
    -  }
     
    -  private boolean doAlloc() {
    -    for (ValueVector v : allocationVectors) {
    -      try {
    -        AllocationHelper.allocateNew(v, current.getRecordCount());
    -      } catch (OutOfMemoryException ex) {
    -        return false;
    -      }
    +  @SuppressWarnings("resource")
    +  private IterOutcome doWork(RecordBatch inputBatch, boolean newSchema) throws ClassTransformationException, IOException, SchemaChangeException {
    +    if (inputBatch.getSchema().getFieldCount() != container.getSchema().getFieldCount()) {
    +      // wrong.
         }
    -    return true;
    -  }
     
    -  @SuppressWarnings("resource")
    -  private IterOutcome doWork() throws ClassTransformationException, IOException, SchemaChangeException {
    -    if (allocationVectors != null) {
    -      for (ValueVector v : allocationVectors) {
    -        v.clear();
    -      }
    +    if (newSchema) {
    +      createUnionAller(inputBatch);
         }
     
    -    allocationVectors = Lists.newArrayList();
    -    transfers.clear();
    +    container.zeroVectors();
     
    -    // If both sides of Union-All are empty
    -    if(unionAllInput.isBothSideEmpty()) {
    -      for(int i = 0; i < outputFields.size(); ++i) {
    -        final String colName = outputFields.get(i).getPath();
    -        final MajorType majorType = MajorType.newBuilder()
    -            .setMinorType(MinorType.INT)
    -            .setMode(DataMode.OPTIONAL)
    -            .build();
    -
    -        MaterializedField outputField = MaterializedField.create(colName, majorType);
    -        ValueVector vv = container.addOrGet(outputField, callBack);
    -        allocationVectors.add(vv);
    -      }
    +    for (final ValueVector v : this.allocationVectors) {
    +      AllocationHelper.allocateNew(v, inputBatch.getRecordCount());
    +    }
     
    -      container.buildSchema(BatchSchema.SelectionVectorMode.NONE);
    +    recordCount = unionall.unionRecords(0, inputBatch.getRecordCount(), 0);
    +    for (final ValueVector v : allocationVectors) {
    +      final ValueVector.Mutator m = v.getMutator();
    +      m.setValueCount(recordCount);
    +    }
    +
    +    if (callBack.getSchemaChangedAndReset()) {
           return IterOutcome.OK_NEW_SCHEMA;
    +    } else {
    +      return IterOutcome.OK;
         }
    +  }
    +
    +  private void createUnionAller(RecordBatch inputBatch) throws ClassTransformationException, IOException, SchemaChangeException {
    +    transfers.clear();
    +    allocationVectors.clear();;
    --- End diff --
    
    One semi-colon should be enough... :-)


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] drill pull request #906: DRILL-5546: Handle schema change exception failure ...

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

    https://github.com/apache/drill/pull/906#discussion_r136188895
  
    --- Diff: exec/vector/src/main/java/org/apache/drill/exec/vector/UntypedNullVector.java ---
    @@ -0,0 +1,267 @@
    +/*
    + * 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;
    +
    +
    +import com.google.common.base.Preconditions;
    +import io.netty.buffer.DrillBuf;
    +import org.apache.drill.exec.memory.BufferAllocator;
    +import org.apache.drill.exec.proto.UserBitShared.SerializedField;
    +import org.apache.drill.exec.record.MaterializedField;
    +import org.apache.drill.exec.record.TransferPair;
    +import org.apache.drill.exec.vector.complex.reader.FieldReader;
    +
    +/** UntypedNullVector is to represent a value vector with {@link org.apache.drill.common.types.MinorType.NULL}
    + *  All values in the vector represent two semantic implications: 1) the value is unknown, 2) the type is unknown.
    + *  Because of this, we only have to keep track of the number of values in value vector,
    + *  and there is no allocated buffer to back up this value vector. Therefore, the majority of
    + *  methods in this class is either no-op, or throws {@link UnsupportedOperationException}.
    + *
    + */
    +public final class UntypedNullVector extends BaseDataValueVector implements FixedWidthVector {
    +  private static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(UntypedNullVector.class);
    +
    +  /**
    +   * Width of each fixed-width value.
    +   */
    +  public static final int VALUE_WIDTH = 0;
    +
    +  private final Accessor accessor = new Accessor();
    +  private final Mutator mutator = new Mutator();
    +  private int valueCount ;
    +
    +  public UntypedNullVector(MaterializedField field, BufferAllocator allocator) {
    +    super(field, allocator);
    +    valueCount = 0;
    +  }
    +
    +  @Override
    +  public FieldReader getReader() { throw new UnsupportedOperationException(); }
    +
    +  @Override
    +  public int getBufferSizeFor(final int valueCount) {
    +    return 0;
    +  }
    +
    +  @Override
    +  public int getValueCapacity(){
    +    return Character.MAX_VALUE;
    +  }
    +
    +  @Override
    +  public Accessor getAccessor() { return accessor; }
    +
    +  @Override
    +  public Mutator getMutator() { return mutator; }
    +
    +  @Override
    +  public void setInitialCapacity(final int valueCount) {
    +  }
    +
    +  @Override
    +  public void allocateNew() {
    +  }
    +
    +  @Override
    +  public boolean allocateNewSafe() {
    +    return true;
    +  }
    +
    +  @Override
    +  public void allocateNew(final int valueCount) {
    +  }
    +
    +  @Override
    +  public void reset() {
    +  }
    +
    +  /**
    +   * {@inheritDoc}
    +   */
    +  @Override
    +  public void zeroVector() {
    +  }
    +
    +  @Override
    +  public void load(SerializedField metadata, DrillBuf buffer) {
    +    Preconditions.checkArgument(this.field.getPath().equals(metadata.getNamePart().getName()),
    +        "The field %s doesn't match the provided metadata %s.", this.field, metadata);
    +    final int actualLength = metadata.getBufferLength();
    +    final int valueCount = metadata.getValueCount();
    +    final int expectedLength = valueCount * VALUE_WIDTH;
    +    assert actualLength == expectedLength : String.format("Expected to load %d bytes but actually loaded %d bytes", expectedLength, actualLength);
    +
    +    this.valueCount = valueCount;
    +  }
    +
    +  @Override
    +  public TransferPair getTransferPair(BufferAllocator allocator){
    +    return new TransferImpl(getField(), allocator);
    +  }
    +
    +  @Override
    +  public TransferPair getTransferPair(String ref, BufferAllocator allocator){
    +    return new TransferImpl(getField().withPath(ref), allocator);
    +  }
    +
    +  @Override
    +  public TransferPair makeTransferPair(ValueVector to) {
    +    return new TransferImpl((UntypedNullVector) to);
    +  }
    +
    +  public void transferTo(UntypedNullVector target){
    +  }
    +
    +  public void splitAndTransferTo(int startIndex, int length, UntypedNullVector target) {
    +  }
    +
    +  @Override
    +  public int getPayloadByteCount() {
    +    return 0;
    +  }
    +
    +  private class TransferImpl implements TransferPair{
    +    private UntypedNullVector to;
    +
    +    public TransferImpl(MaterializedField field, BufferAllocator allocator){
    +      to = new UntypedNullVector(field, allocator);
    +    }
    +
    +    public TransferImpl(UntypedNullVector to) {
    +      this.to = to;
    +    }
    +
    +    @Override
    +    public UntypedNullVector getTo(){
    +      return to;
    +    }
    +
    +    @Override
    +    public void transfer(){
    +      transferTo(to);
    +    }
    +
    +    @Override
    +    public void splitAndTransfer(int startIndex, int length) {
    +      checkBounds(startIndex);
    +      checkBounds(startIndex + length - 1);
    +      splitAndTransferTo(startIndex, length, to);
    +    }
    +
    +    @Override
    +    public void copyValueSafe(int fromIndex, int toIndex) {
    +      checkBounds(fromIndex);
    +      to.copyFromSafe(fromIndex, toIndex, UntypedNullVector.this);
    +    }
    +  }
    +
    +  public void copyFrom(int fromIndex, int thisIndex, UntypedNullVector from){
    +  }
    +
    +  public void copyFromSafe(int fromIndex, int thisIndex, UntypedNullVector from){
    +  }
    +
    +  private void checkBounds(int index) {
    +    if (index < 0 || index >= valueCount) {
    +      throw new IndexOutOfBoundsException(String.format(
    +          "index: %d, expected: range(0, %d-1))", index, valueCount));
    +    }
    +  }
    +  @Override
    +  public void copyEntry(int toIndex, ValueVector from, int fromIndex) {
    +    ((UntypedNullVector) from).data.getBytes(fromIndex * 4, data, toIndex * 4, 4);
    +  }
    +
    +  public final class Accessor extends BaseAccessor {
    +    @Override
    +    public int getValueCount() {
    +      return valueCount;
    +    }
    +
    +    @Override
    +    public boolean isNull(int index){
    +      checkBounds(index);
    +      return true;
    +    }
    +
    +    public int isSet(int index) {
    +      checkBounds(index);
    +      return 0;
    +    }
    +
    +    @Override
    +    public Object getObject(int index) {
    +      throw new UnsupportedOperationException();
    +    }
    +
    +    public void get(int index, UntypedNullHolder holder) {
    +      checkBounds(index);
    +    }
    +
    +  }
    +
    +  /**
    +   * UntypedNullVector.Mutator throws Exception for most of its mutate operations, except for the ones that set
    +   * value counts.
    +   *
    +   */
    +   public final class Mutator extends BaseMutator {
    +
    +    private Mutator() {};
    --- End diff --
    
    Done.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] drill pull request #906: DRILL-5546: Handle schema change exception failure ...

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/906#discussion_r134300698
  
    --- Diff: exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/ScanBatch.java ---
    @@ -152,97 +157,75 @@ public void kill(boolean sendUpstream) {
         }
       }
     
    -  private void releaseAssets() {
    -    container.zeroVectors();
    -  }
    -
    -  private void clearFieldVectorMap() {
    -    for (final ValueVector v : mutator.fieldVectorMap().values()) {
    -      v.clear();
    -    }
    -  }
    -
       @Override
       public IterOutcome next() {
         if (done) {
           return IterOutcome.NONE;
         }
         oContext.getStats().startProcessing();
         try {
    -      try {
    -        injector.injectChecked(context.getExecutionControls(), "next-allocate", OutOfMemoryException.class);
    -
    -        currentReader.allocate(mutator.fieldVectorMap());
    -      } catch (OutOfMemoryException e) {
    -        clearFieldVectorMap();
    -        throw UserException.memoryError(e).build(logger);
    -      }
    -      while ((recordCount = currentReader.next()) == 0) {
    +      while (true) {
             try {
    -          if (!readers.hasNext()) {
    -            // We're on the last reader, and it has no (more) rows.
    -            currentReader.close();
    -            releaseAssets();
    -            done = true;  // have any future call to next() return NONE
    -
    -            if (mutator.isNewSchema()) {
    -              // This last reader has a new schema (e.g., we have a zero-row
    -              // file or other source).  (Note that some sources have a non-
    -              // null/non-trivial schema even when there are no rows.)
    +          injector.injectChecked(context.getExecutionControls(), "next-allocate", OutOfMemoryException.class);
    --- End diff --
    
    The code in `ScanBatch` changed significantly -- looks like a very good improvement.
    
    However, I could not readily find unit tests that execute all the complex new code paths. Can you perhaps point out the tests? Otherwise, as a reviewer, I find myself acting as the unit tests; I must "mentally execute" the code paths for all scenarios I can imagine. This is slow and will lead to many, many comments as I try to think through each and every step. Plus, the situation is made harder by the fact that code is duplicated along various execution branches.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] drill pull request #906: DRILL-5546: Handle schema change exception failure ...

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/906#discussion_r134299504
  
    --- Diff: exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/ScanBatch.java ---
    @@ -329,9 +326,11 @@ public TypedFieldId getValueVectorId(SchemaPath path) {
     
       @VisibleForTesting
       public static class Mutator implements OutputMutator {
    -    /** Whether schema has changed since last inquiry (via #isNewSchema}).  Is
    -     *  true before first inquiry. */
    -    private boolean schemaChanged = true;
    +    /** Flag keeping track whether top-level schema has changed since last inquiry (via #isNewSchema}).
    +     * It's initialized to false, or reset to false after #isNewSchema or after #clear, until a new value vector
    +     * or a value vector with different type is added to fieldVectorMap.
    +     **/
    +    private boolean schemaChanged;
    --- End diff --
    
    Using a flag is very messy. The new version uses a counter. An observer simply remembers the previous count and compares it against the current count. Allows multiple observers without negotiating over which is responsible for resetting the flag.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] drill pull request #906: DRILL-5546: Handle schema change exception failure ...

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/906#discussion_r135371013
  
    --- Diff: exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/union/UnionAllRecordBatch.java ---
    @@ -39,88 +35,107 @@
     import org.apache.drill.exec.expr.ValueVectorWriteExpression;
     import org.apache.drill.exec.ops.FragmentContext;
     import org.apache.drill.exec.physical.config.UnionAll;
    -import org.apache.drill.exec.record.AbstractRecordBatch;
    +import org.apache.drill.exec.record.AbstractBinaryRecordBatch;
     import org.apache.drill.exec.record.BatchSchema;
     import org.apache.drill.exec.record.MaterializedField;
     import org.apache.drill.exec.record.RecordBatch;
     import org.apache.drill.exec.record.TransferPair;
     import org.apache.drill.exec.record.TypedFieldId;
     import org.apache.drill.exec.record.VectorWrapper;
    -import org.apache.drill.exec.record.WritableBatch;
    -import org.apache.drill.exec.record.selection.SelectionVector2;
    -import org.apache.drill.exec.record.selection.SelectionVector4;
     import org.apache.drill.exec.resolver.TypeCastRules;
     import org.apache.drill.exec.vector.AllocationHelper;
     import org.apache.drill.exec.vector.FixedWidthVector;
     import org.apache.drill.exec.vector.SchemaChangeCallBack;
     import org.apache.drill.exec.vector.ValueVector;
     
    -import com.google.common.collect.Lists;
    +import java.io.IOException;
    +import java.util.Iterator;
    +import java.util.List;
    +import java.util.NoSuchElementException;
    +import java.util.Stack;
     
    -public class UnionAllRecordBatch extends AbstractRecordBatch<UnionAll> {
    +public class UnionAllRecordBatch extends AbstractBinaryRecordBatch<UnionAll> {
    --- End diff --
    
    Very nice cleanup here! Thanks! The new code is much easier to follow.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] drill pull request #906: DRILL-5546: Handle schema change exception failure ...

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

    https://github.com/apache/drill/pull/906#discussion_r136187915
  
    --- Diff: exec/vector/src/main/java/org/apache/drill/exec/vector/UntypedNullVector.java ---
    @@ -0,0 +1,267 @@
    +/*
    + * 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;
    +
    +
    +import com.google.common.base.Preconditions;
    +import io.netty.buffer.DrillBuf;
    +import org.apache.drill.exec.memory.BufferAllocator;
    +import org.apache.drill.exec.proto.UserBitShared.SerializedField;
    +import org.apache.drill.exec.record.MaterializedField;
    +import org.apache.drill.exec.record.TransferPair;
    +import org.apache.drill.exec.vector.complex.reader.FieldReader;
    +
    +/** UntypedNullVector is to represent a value vector with {@link org.apache.drill.common.types.MinorType.NULL}
    + *  All values in the vector represent two semantic implications: 1) the value is unknown, 2) the type is unknown.
    + *  Because of this, we only have to keep track of the number of values in value vector,
    + *  and there is no allocated buffer to back up this value vector. Therefore, the majority of
    + *  methods in this class is either no-op, or throws {@link UnsupportedOperationException}.
    + *
    + */
    +public final class UntypedNullVector extends BaseDataValueVector implements FixedWidthVector {
    +  private static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(UntypedNullVector.class);
    +
    +  /**
    +   * Width of each fixed-width value.
    +   */
    +  public static final int VALUE_WIDTH = 0;
    +
    +  private final Accessor accessor = new Accessor();
    +  private final Mutator mutator = new Mutator();
    +  private int valueCount ;
    --- End diff --
    
    done.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] drill pull request #906: DRILL-5546: Handle schema change exception failure ...

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

    https://github.com/apache/drill/pull/906#discussion_r136143881
  
    --- Diff: exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/union/UnionAllRecordBatch.java ---
    @@ -39,88 +35,107 @@
     import org.apache.drill.exec.expr.ValueVectorWriteExpression;
     import org.apache.drill.exec.ops.FragmentContext;
     import org.apache.drill.exec.physical.config.UnionAll;
    -import org.apache.drill.exec.record.AbstractRecordBatch;
    +import org.apache.drill.exec.record.AbstractBinaryRecordBatch;
     import org.apache.drill.exec.record.BatchSchema;
     import org.apache.drill.exec.record.MaterializedField;
     import org.apache.drill.exec.record.RecordBatch;
     import org.apache.drill.exec.record.TransferPair;
     import org.apache.drill.exec.record.TypedFieldId;
     import org.apache.drill.exec.record.VectorWrapper;
    -import org.apache.drill.exec.record.WritableBatch;
    -import org.apache.drill.exec.record.selection.SelectionVector2;
    -import org.apache.drill.exec.record.selection.SelectionVector4;
     import org.apache.drill.exec.resolver.TypeCastRules;
     import org.apache.drill.exec.vector.AllocationHelper;
     import org.apache.drill.exec.vector.FixedWidthVector;
     import org.apache.drill.exec.vector.SchemaChangeCallBack;
     import org.apache.drill.exec.vector.ValueVector;
     
    -import com.google.common.collect.Lists;
    +import java.io.IOException;
    +import java.util.Iterator;
    +import java.util.List;
    +import java.util.NoSuchElementException;
    +import java.util.Stack;
     
    -public class UnionAllRecordBatch extends AbstractRecordBatch<UnionAll> {
    +public class UnionAllRecordBatch extends AbstractBinaryRecordBatch<UnionAll> {
       private static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(UnionAllRecordBatch.class);
     
    -  private List<MaterializedField> outputFields;
    +  private SchemaChangeCallBack callBack = new SchemaChangeCallBack();
       private UnionAller unionall;
    -  private UnionAllInput unionAllInput;
    -  private RecordBatch current;
    -
       private final List<TransferPair> transfers = Lists.newArrayList();
    -  private List<ValueVector> allocationVectors;
    -  protected SchemaChangeCallBack callBack = new SchemaChangeCallBack();
    +  private List<ValueVector> allocationVectors = Lists.newArrayList();
       private int recordCount = 0;
    -  private boolean schemaAvailable = false;
    +  private UnionInputIterator unionInputIterator;
     
       public UnionAllRecordBatch(UnionAll config, List<RecordBatch> children, FragmentContext context) throws OutOfMemoryException {
    -    super(config, context, false);
    -    assert (children.size() == 2) : "The number of the operands of Union must be 2";
    -    unionAllInput = new UnionAllInput(this, children.get(0), children.get(1));
    -  }
    -
    -  @Override
    -  public int getRecordCount() {
    -    return recordCount;
    +    super(config, context, true, children.get(0), children.get(1));
       }
     
       @Override
       protected void killIncoming(boolean sendUpstream) {
    -    unionAllInput.getLeftRecordBatch().kill(sendUpstream);
    -    unionAllInput.getRightRecordBatch().kill(sendUpstream);
    +    left.kill(sendUpstream);
    +    right.kill(sendUpstream);
       }
     
    -  @Override
    -  public SelectionVector2 getSelectionVector2() {
    -    throw new UnsupportedOperationException("UnionAllRecordBatch does not support selection vector");
    -  }
    +  protected void buildSchema() throws SchemaChangeException {
    +    if (! prefetchFirstBatchFromBothSides()) {
    +      return;
    +    }
     
    -  @Override
    -  public SelectionVector4 getSelectionVector4() {
    -    throw new UnsupportedOperationException("UnionAllRecordBatch does not support selection vector");
    +    unionInputIterator = new UnionInputIterator(leftUpstream, left, rightUpstream, right);
    +
    +    if (leftUpstream == IterOutcome.NONE && rightUpstream == IterOutcome.OK_NEW_SCHEMA) {
    +      inferOutputFieldsOneSide(right.getSchema());
    +    } else if (rightUpstream == IterOutcome.NONE && leftUpstream == IterOutcome.OK_NEW_SCHEMA) {
    +      inferOutputFieldsOneSide((left.getSchema()));
    +    } else if (leftUpstream == IterOutcome.OK_NEW_SCHEMA && rightUpstream == IterOutcome.OK_NEW_SCHEMA) {
    +      inferOutputFieldsBothSide(left.getSchema(), right.getSchema());
    +    }
    +
    +    container.buildSchema(BatchSchema.SelectionVectorMode.NONE);
    +
    +    for (VectorWrapper vv: container) {
    +      vv.getValueVector().allocateNew();
    +      vv.getValueVector().getMutator().setValueCount(0);
    +    }
       }
     
       @Override
       public IterOutcome innerNext() {
         try {
    -      IterOutcome upstream = unionAllInput.nextBatch();
    -      logger.debug("Upstream of Union-All: {}", upstream);
    +      if (!unionInputIterator.hasNext()) {
    +        return IterOutcome.NONE;
    +      }
    +
    +      Pair<IterOutcome, RecordBatch> nextBatch = unionInputIterator.next();
    +
    +      IterOutcome upstream = nextBatch.left;
    +      RecordBatch incoming = nextBatch.right;
    +
    +      // skip batches with same schema as the previous one yet having 0 row.
    +      if (upstream == IterOutcome.OK && incoming.getRecordCount() == 0) {
    +        do {
    +          for (final VectorWrapper<?> w : incoming) {
    +            w.clear();
    +          }
    +          if (!unionInputIterator.hasNext()) {
    +            return IterOutcome.NONE;
    +          }
    +          nextBatch = unionInputIterator.next();
    +          upstream = nextBatch.left;
    +          incoming = nextBatch.right;
    +        } while ((upstream == IterOutcome.OK) &&
    +                incoming.getRecordCount() == 0);
    +      }
    +
    --- End diff --
    
    The  loop does not have to handle `STOP` or `OOM` as they are handled in the `switch` statement.  Previously, the loop was intended to handle the same schema yet having row cases.
    
    The revised patch adopted your suggestion, by putting everything in the loop. 



---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] drill pull request #906: DRILL-5546: Handle schema change exception failure ...

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

    https://github.com/apache/drill/pull/906#discussion_r135925091
  
    --- Diff: exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/project/ProjectRecordBatch.java ---
    @@ -768,4 +765,73 @@ else if (exprHasPrefix && refHasPrefix) {
           }
         }
       }
    +
    +  /**
    +   * handle FAST NONE specially when Project for query output. This happens when input returns a
    +   * FAST NONE directly ( input does not return any batch with schema/data).
    +   *
    +   * Project operator has to return a batch with schema derived using the following 3 rules:
    +   *  Case 1:  *  ==>  expand into an empty list of columns.
    +   *  Case 2:  regular column reference ==> treat as nullable-int column
    +   *  Case 3:  expressions => Call ExpressionTreeMaterialization over an empty vector contain.
    +   *           Once the expression is materialized without error, use the output type of materialized
    +   *           expression.
    +   * The batch is constructed with the above rules, and recordCount = 0.
    +   * Returned with OK_NEW_SCHEMA to down-stream operator.
    --- End diff --
    
    It could be Screen/Writer. 


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] drill pull request #906: DRILL-5546: Handle schema change exception failure ...

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/906#discussion_r134627196
  
    --- Diff: exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/project/ProjectRecordBatch.java ---
    @@ -768,4 +765,73 @@ else if (exprHasPrefix && refHasPrefix) {
           }
         }
       }
    +
    +  /**
    +   * handle FAST NONE specially when Project for query output. This happens when input returns a
    +   * FAST NONE directly ( input does not return any batch with schema/data).
    +   *
    +   * Project operator has to return a batch with schema derived using the following 3 rules:
    +   *  Case 1:  *  ==>  expand into an empty list of columns.
    +   *  Case 2:  regular column reference ==> treat as nullable-int column
    +   *  Case 3:  expressions => Call ExpressionTreeMaterialization over an empty vector contain.
    --- End diff --
    
    Again, this is HTML, so:
    
    \<ul>
    \<li>Case 1: ...\</li>
    ...
    \</ul>


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] drill pull request #906: DRILL-5546: Handle schema change exception failure ...

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

    https://github.com/apache/drill/pull/906#discussion_r135708467
  
    --- Diff: exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/ScanBatch.java ---
    @@ -152,97 +157,75 @@ public void kill(boolean sendUpstream) {
         }
       }
     
    -  private void releaseAssets() {
    -    container.zeroVectors();
    -  }
    -
    -  private void clearFieldVectorMap() {
    -    for (final ValueVector v : mutator.fieldVectorMap().values()) {
    -      v.clear();
    -    }
    -  }
    -
       @Override
       public IterOutcome next() {
         if (done) {
           return IterOutcome.NONE;
         }
         oContext.getStats().startProcessing();
         try {
    -      try {
    -        injector.injectChecked(context.getExecutionControls(), "next-allocate", OutOfMemoryException.class);
    -
    -        currentReader.allocate(mutator.fieldVectorMap());
    -      } catch (OutOfMemoryException e) {
    -        clearFieldVectorMap();
    -        throw UserException.memoryError(e).build(logger);
    -      }
    -      while ((recordCount = currentReader.next()) == 0) {
    +      while (true) {
             try {
    -          if (!readers.hasNext()) {
    -            // We're on the last reader, and it has no (more) rows.
    -            currentReader.close();
    -            releaseAssets();
    -            done = true;  // have any future call to next() return NONE
    -
    -            if (mutator.isNewSchema()) {
    -              // This last reader has a new schema (e.g., we have a zero-row
    -              // file or other source).  (Note that some sources have a non-
    -              // null/non-trivial schema even when there are no rows.)
    +          injector.injectChecked(context.getExecutionControls(), "next-allocate", OutOfMemoryException.class);
    +          currentReader.allocate(mutator.fieldVectorMap());
    +        } catch (OutOfMemoryException e) {
    +          clearFieldVectorMap();
    +          throw UserException.memoryError(e).build(logger);
    +        }
     
    -              container.buildSchema(SelectionVectorMode.NONE);
    -              schema = container.getSchema();
    +        recordCount = currentReader.next();
    +        Preconditions.checkArgument(recordCount >= 0,
    +            "recordCount from RecordReader.next() should not be negative");
     
    -              return IterOutcome.OK_NEW_SCHEMA;
    -            }
    -            return IterOutcome.NONE;
    -          }
    -          // At this point, the reader that hit its end is not the last reader.
    +        boolean isNewRegularSchema = mutator.isNewSchema();
    +        // We should skip the reader, when recordCount = 0 && ! isNewRegularSchema.
    +        // Add/set implicit column vectors, only when reader gets > 0 row, or
    +        // when reader gets 0 row but with a schema with new field added
    +        if (recordCount > 0 || isNewRegularSchema) {
    +          addImplicitVectors();
    +          populateImplicitVectors();
    +        }
     
    -          // If all the files we have read so far are just empty, the schema is not useful
    -          if (! hasReadNonEmptyFile) {
    -            container.clear();
    -            clearFieldVectorMap();
    -            mutator.clear();
    -          }
    +        boolean isNewImplicitSchema = mutator.isNewSchema();
    +        for (VectorWrapper<?> w : container) {
    +          w.getValueVector().getMutator().setValueCount(recordCount);
    +        }
    +        final boolean isNewSchema = isNewRegularSchema || isNewImplicitSchema;
    +        oContext.getStats().batchReceived(0, recordCount, isNewSchema);
     
    +        if (recordCount == 0) {
               currentReader.close();
    -          currentReader = readers.next();
    -          implicitValues = implicitColumns.hasNext() ? implicitColumns.next() : null;
    -          currentReader.setup(oContext, mutator);
    -          try {
    -            currentReader.allocate(mutator.fieldVectorMap());
    -          } catch (OutOfMemoryException e) {
    -            clearFieldVectorMap();
    -            throw UserException.memoryError(e).build(logger);
    +          if (isNewSchema) {
    +            // current reader presents a new schema in mutator even though it has 0 row.
    +            // This could happen when data sources have a non-trivial schema with 0 row.
    +            container.buildSchema(SelectionVectorMode.NONE);
    +            schema = container.getSchema();
    +            if (readers.hasNext()) {
    +              advanceNextReader();
    +            } else {
    +              done = true;  // indicates the follow-up next() call will return IterOutcome.NONE.
    +            }
    +            return IterOutcome.OK_NEW_SCHEMA;
    +          } else { // not a new schema
    +            if (readers.hasNext()) {
    +              advanceNextReader();
    +              continue; // skip reader returning 0 row and having same schema.
    +                        // Skip to next loop iteration with next available reader.
    +            } else {
    +              releaseAssets(); // All data has been read. Release resource.
    +              return IterOutcome.NONE;
    +            }
    +          }
    +        } else { // recordCount > 0
    +          if (isNewSchema) {
    --- End diff --
    
    Removed one copy of shared code to handle two different cases in revised patch, though I did not use the idea of state. Hopefully, the new code looks cleaner. 


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] drill pull request #906: DRILL-5546: Handle schema change exception failure ...

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/906#discussion_r134626985
  
    --- Diff: exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/project/ProjectRecordBatch.java ---
    @@ -768,4 +765,73 @@ else if (exprHasPrefix && refHasPrefix) {
           }
         }
       }
    +
    +  /**
    +   * handle FAST NONE specially when Project for query output. This happens when input returns a
    --- End diff --
    
    "handle" --> "Handle"
    
    For future reference, can we rename "FAST NONE" to "empty result set"? "FAST NONE" is an implementation detail that does not capture why we are returning NONE. But "empty result set" captures the idea that a result (not just a batch) has no rows, which is what we are really trying to handle here.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] drill pull request #906: DRILL-5546: Handle schema change exception failure ...

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

    https://github.com/apache/drill/pull/906#discussion_r136181920
  
    --- Diff: exec/java-exec/src/main/java/org/apache/drill/exec/planner/logical/DrillScanRel.java ---
    @@ -160,12 +161,7 @@ public RelOptCost computeSelfCost(final RelOptPlanner planner, RelMetadataQuery
         final ScanStats stats = groupScan.getScanStats(settings);
         int columnCount = getRowType().getFieldCount();
         double ioCost = 0;
    -    boolean isStarQuery = Iterables.tryFind(getRowType().getFieldNames(), new Predicate<String>() {
    -      @Override
    -      public boolean apply(String input) {
    -        return Preconditions.checkNotNull(input).equals("*");
    -      }
    -    }).isPresent();
    +    boolean isStarQuery = AbstractRecordReader.isStarQuery(columns);
    --- End diff --
    
    Move to `org.apache.drill.exec.util.Utilities`. 


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] drill pull request #906: DRILL-5546: Handle schema change exception failure ...

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

    https://github.com/apache/drill/pull/906#discussion_r136187598
  
    --- Diff: exec/vector/src/main/java/org/apache/drill/exec/vector/UntypedNullHolder.java ---
    @@ -0,0 +1,50 @@
    +/*
    + * 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;
    +
    +import org.apache.drill.common.types.TypeProtos;
    +import org.apache.drill.common.types.Types;
    +import org.apache.drill.exec.expr.holders.ValueHolder;
    +
    +public class UntypedNullHolder implements ValueHolder {
    +  public static final TypeProtos.MajorType TYPE = Types.optional(TypeProtos.MinorType.NULL);
    +
    +  public TypeProtos.MajorType getType() {return TYPE;}
    +
    +  public static final int WIDTH = 0;
    +
    +  public int isSet = 1;
    +
    +  @Deprecated
    +  public int hashCode(){
    +    throw new UnsupportedOperationException();
    +  }
    +
    +  /*
    +   * Reason for deprecation is that ValueHolders are potential scalar replacements
    +   * and hence we don't want any methods to be invoked on them.
    +   */
    +  @Deprecated
    +  public String toString(){
    +    throw new UnsupportedOperationException();
    --- End diff --
    
    This applies to all the value holder class. See [1].
    
    My understanding is it's for the benefit of potential scalar replacements in byte-code fixup during execution. We may argue we do not have to leverage scalar replacement with JDK8. But before we know for sure, it's better to keep it inline with the other value holder classes. 
    
    
    1. https://github.com/apache/drill/blob/master/exec/vector/src/main/codegen/templates/ValueHolders.java#L97-L109


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] drill pull request #906: DRILL-5546: Handle schema change exception failure ...

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/906#discussion_r134628276
  
    --- Diff: exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/project/ProjectRecordBatch.java ---
    @@ -768,4 +765,73 @@ else if (exprHasPrefix && refHasPrefix) {
           }
         }
       }
    +
    +  /**
    +   * handle FAST NONE specially when Project for query output. This happens when input returns a
    +   * FAST NONE directly ( input does not return any batch with schema/data).
    +   *
    +   * Project operator has to return a batch with schema derived using the following 3 rules:
    +   *  Case 1:  *  ==>  expand into an empty list of columns.
    +   *  Case 2:  regular column reference ==> treat as nullable-int column
    +   *  Case 3:  expressions => Call ExpressionTreeMaterialization over an empty vector contain.
    +   *           Once the expression is materialized without error, use the output type of materialized
    +   *           expression.
    +   * The batch is constructed with the above rules, and recordCount = 0.
    +   * Returned with OK_NEW_SCHEMA to down-stream operator.
    +   */
    +  @Override
    +  protected IterOutcome handleFastNone() {
    +    if (! popConfig.isOutputProj()) {
    +      return super.handleFastNone();
    +    }
    +
    +    allocationVectors = new ArrayList<>();
    +    final List<NamedExpression> exprs = getExpressionList();
    +    final ErrorCollector collector = new ErrorCollectorImpl();
    +    VectorContainer fakedIncomingVC = new VectorContainer();
    +
    +    for (NamedExpression namedExpression : exprs) {
    +      if (namedExpression.getExpr() instanceof SchemaPath) {
    +        final NameSegment expr = ((SchemaPath) namedExpression.getExpr()).getRootSegment();
    +        if (expr.getPath().contains(StarColumnHelper.STAR_COLUMN)) {
    --- End diff --
    
    `contains`? This handles `foo.*`, `*bar*` and so on? Is this expected, or is only `*` allowed?


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] drill pull request #906: DRILL-5546: Handle schema change exception failure ...

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/906#discussion_r135371136
  
    --- Diff: exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/union/UnionAllRecordBatch.java ---
    @@ -130,562 +145,248 @@ public IterOutcome innerNext() {
       }
     
       @Override
    -  public WritableBatch getWritableBatch() {
    -    return WritableBatch.get(this);
    +  public int getRecordCount() {
    +    return recordCount;
       }
     
    -  private void setValueCount(int count) {
    -    for (ValueVector v : allocationVectors) {
    -      ValueVector.Mutator m = v.getMutator();
    -      m.setValueCount(count);
    -    }
    -  }
     
    -  private boolean doAlloc() {
    -    for (ValueVector v : allocationVectors) {
    -      try {
    -        AllocationHelper.allocateNew(v, current.getRecordCount());
    -      } catch (OutOfMemoryException ex) {
    -        return false;
    -      }
    +  @SuppressWarnings("resource")
    +  private IterOutcome doWork(RecordBatch inputBatch, boolean newSchema) throws ClassTransformationException, IOException, SchemaChangeException {
    +    if (inputBatch.getSchema().getFieldCount() != container.getSchema().getFieldCount()) {
    +      // wrong.
         }
    -    return true;
    -  }
     
    -  @SuppressWarnings("resource")
    -  private IterOutcome doWork() throws ClassTransformationException, IOException, SchemaChangeException {
    -    if (allocationVectors != null) {
    -      for (ValueVector v : allocationVectors) {
    -        v.clear();
    -      }
    +    if (newSchema) {
    +      createUnionAller(inputBatch);
         }
     
    -    allocationVectors = Lists.newArrayList();
    -    transfers.clear();
    +    container.zeroVectors();
     
    -    // If both sides of Union-All are empty
    -    if(unionAllInput.isBothSideEmpty()) {
    -      for(int i = 0; i < outputFields.size(); ++i) {
    -        final String colName = outputFields.get(i).getPath();
    -        final MajorType majorType = MajorType.newBuilder()
    -            .setMinorType(MinorType.INT)
    -            .setMode(DataMode.OPTIONAL)
    -            .build();
    -
    -        MaterializedField outputField = MaterializedField.create(colName, majorType);
    -        ValueVector vv = container.addOrGet(outputField, callBack);
    -        allocationVectors.add(vv);
    -      }
    +    for (final ValueVector v : this.allocationVectors) {
    +      AllocationHelper.allocateNew(v, inputBatch.getRecordCount());
    +    }
     
    -      container.buildSchema(BatchSchema.SelectionVectorMode.NONE);
    +    recordCount = unionall.unionRecords(0, inputBatch.getRecordCount(), 0);
    +    for (final ValueVector v : allocationVectors) {
    +      final ValueVector.Mutator m = v.getMutator();
    +      m.setValueCount(recordCount);
    +    }
    +
    +    if (callBack.getSchemaChangedAndReset()) {
           return IterOutcome.OK_NEW_SCHEMA;
    +    } else {
    +      return IterOutcome.OK;
         }
    +  }
    +
    +  private void createUnionAller(RecordBatch inputBatch) throws ClassTransformationException, IOException, SchemaChangeException {
    +    transfers.clear();
    +    allocationVectors.clear();;
     
         final ClassGenerator<UnionAller> cg = CodeGenerator.getRoot(UnionAller.TEMPLATE_DEFINITION, context.getFunctionRegistry(), context.getOptions());
         cg.getCodeGenerator().plainJavaCapable(true);
         // Uncomment out this line to debug the generated code.
    -//    cg.getCodeGenerator().saveCodeForDebugging(true);
    +    //    cg.getCodeGenerator().saveCodeForDebugging(true);
    +
         int index = 0;
    -    for(VectorWrapper<?> vw : current) {
    -       ValueVector vvIn = vw.getValueVector();
    -      // get the original input column names
    -      SchemaPath inputPath = SchemaPath.getSimplePath(vvIn.getField().getPath());
    -      // get the renamed column names
    -      SchemaPath outputPath = SchemaPath.getSimplePath(outputFields.get(index).getPath());
    +    for(VectorWrapper<?> vw : inputBatch) {
    +      ValueVector vvIn = vw.getValueVector();
    +      ValueVector vvOut = container.getValueVector(index).getValueVector();
     
           final ErrorCollector collector = new ErrorCollectorImpl();
           // According to input data names, Minortypes, Datamodes, choose to
           // transfer directly,
           // rename columns or
           // cast data types (Minortype or DataMode)
    -      if (hasSameTypeAndMode(outputFields.get(index), vw.getValueVector().getField())) {
    +      if (hasSameTypeAndMode(container.getSchema().getColumn(index), vvIn.getField())
    +          && vvIn.getField().getType().getMinorType() != TypeProtos.MinorType.MAP // Per DRILL-5521, existing bug for map transfer
    +          ) {
             // Transfer column
    -
    -        MajorType outputFieldType = outputFields.get(index).getType();
    -        MaterializedField outputField = MaterializedField.create(outputPath.getAsUnescapedPath(), outputFieldType);
    -
    -        /*
    -          todo: Fix if condition when DRILL-4824 is merged
    -          If condition should be changed to:
    -          `if (outputFields.get(index).getPath().equals(inputPath.getAsUnescapedPath())) {`
    -          DRILL-5419 has changed condition to correct one but this caused regression (DRILL-5521).
    -          Root cause is missing indication of child column in map types when it is null.
    -          DRILL-4824 is re-working json reader implementation, including map types and will fix this problem.
    -          Reverting condition to previous one to avoid regression till DRILL-4824 is merged.
    -          Unit test - TestJsonReader.testKvgenWithUnionAll().
    -         */
    -        if (outputFields.get(index).getPath().equals(inputPath)) {
    -          ValueVector vvOut = container.addOrGet(outputField);
    -          TransferPair tp = vvIn.makeTransferPair(vvOut);
    -          transfers.add(tp);
    +        TransferPair tp = vvIn.makeTransferPair(vvOut);
    +        transfers.add(tp);
             // Copy data in order to rename the column
    -        } else {
    -          final LogicalExpression expr = ExpressionTreeMaterializer.materialize(inputPath, current, collector, context.getFunctionRegistry() );
    -          if (collector.hasErrors()) {
    -            throw new SchemaChangeException(String.format("Failure while trying to materialize incoming schema.  Errors:\n %s.", collector.toErrorString()));
    -          }
    -
    -          ValueVector vv = container.addOrGet(outputField, callBack);
    -          allocationVectors.add(vv);
    -          TypedFieldId fid = container.getValueVectorId(SchemaPath.getSimplePath(outputField.getPath()));
    -          ValueVectorWriteExpression write = new ValueVectorWriteExpression(fid, expr, true);
    -          cg.addExpr(write);
    -        }
    -      // Cast is necessary
    +      } else if (vvIn.getField().getType().getMinorType() == TypeProtos.MinorType.NULL) {
    +        continue;
           } else {
    -        LogicalExpression expr = ExpressionTreeMaterializer.materialize(inputPath, current, collector, context.getFunctionRegistry());
    +        SchemaPath inputPath = SchemaPath.getSimplePath(vvIn.getField().getPath());
    +        MaterializedField inField = vvIn.getField();
    +        MaterializedField outputField = vvOut.getField();
    +
    +        LogicalExpression expr = ExpressionTreeMaterializer.materialize(inputPath, inputBatch, collector, context.getFunctionRegistry());
    +
             if (collector.hasErrors()) {
               throw new SchemaChangeException(String.format("Failure while trying to materialize incoming schema.  Errors:\n %s.", collector.toErrorString()));
             }
     
             // If the inputs' DataMode is required and the outputs' DataMode is not required
             // cast to the one with the least restriction
    -        if(vvIn.getField().getType().getMode() == DataMode.REQUIRED
    -            && outputFields.get(index).getType().getMode() != DataMode.REQUIRED) {
    -          expr = ExpressionTreeMaterializer.convertToNullableType(expr, vvIn.getField().getType().getMinorType(), context.getFunctionRegistry(), collector);
    +        if(inField.getType().getMode() == TypeProtos.DataMode.REQUIRED
    +            && outputField.getType().getMode() != TypeProtos.DataMode.REQUIRED) {
    +          expr = ExpressionTreeMaterializer.convertToNullableType(expr, inField.getType().getMinorType(), context.getFunctionRegistry(), collector);
               if (collector.hasErrors()) {
                 throw new SchemaChangeException(String.format("Failure while trying to materialize incoming schema.  Errors:\n %s.", collector.toErrorString()));
               }
             }
     
             // If two inputs' MinorTypes are different,
             // Insert a cast before the Union operation
    -        if(vvIn.getField().getType().getMinorType() != outputFields.get(index).getType().getMinorType()) {
    -          expr = ExpressionTreeMaterializer.addCastExpression(expr, outputFields.get(index).getType(), context.getFunctionRegistry(), collector);
    +        if(inField.getType().getMinorType() != outputField.getType().getMinorType()) {
    +          expr = ExpressionTreeMaterializer.addCastExpression(expr, outputField.getType(), context.getFunctionRegistry(), collector);
               if (collector.hasErrors()) {
                 throw new SchemaChangeException(String.format("Failure while trying to materialize incoming schema.  Errors:\n %s.", collector.toErrorString()));
               }
             }
     
    -        final MaterializedField outputField = MaterializedField.create(outputPath.getAsUnescapedPath(), expr.getMajorType());
    -        ValueVector vector = container.addOrGet(outputField, callBack);
    -        allocationVectors.add(vector);
             TypedFieldId fid = container.getValueVectorId(SchemaPath.getSimplePath(outputField.getPath()));
     
    -        boolean useSetSafe = !(vector instanceof FixedWidthVector);
    +        boolean useSetSafe = !(vvOut instanceof FixedWidthVector);
             ValueVectorWriteExpression write = new ValueVectorWriteExpression(fid, expr, useSetSafe);
             cg.addExpr(write);
    +
    +        allocationVectors.add(vvOut);
           }
           ++index;
         }
     
         unionall = context.getImplementationClass(cg.getCodeGenerator());
    -    unionall.setup(context, current, this, transfers);
    -
    -    if(!schemaAvailable) {
    -      container.buildSchema(BatchSchema.SelectionVectorMode.NONE);
    -      schemaAvailable = true;
    -    }
    -
    -    if(!doAlloc()) {
    -      return IterOutcome.OUT_OF_MEMORY;
    -    }
    -
    -    recordCount = unionall.unionRecords(0, current.getRecordCount(), 0);
    -    setValueCount(recordCount);
    -    return IterOutcome.OK;
    -  }
    -
    -  public static boolean hasSameTypeAndMode(MaterializedField leftField, MaterializedField rightField) {
    -    return (leftField.getType().getMinorType() == rightField.getType().getMinorType())
    -        && (leftField.getType().getMode() == rightField.getType().getMode());
    -  }
    -
    -  // This method is used by inner class to point the reference `current` to the correct record batch
    -  private void setCurrentRecordBatch(RecordBatch target) {
    -    this.current = target;
    -  }
    -
    -  // This method is used by inner class to clear the current record batch
    -  private void clearCurrentRecordBatch() {
    -    for(VectorWrapper<?> v: current) {
    -      v.clear();
    -    }
    +    unionall.setup(context, inputBatch, this, transfers);
       }
     
    -  public static class UnionAllInput {
    -    private UnionAllRecordBatch unionAllRecordBatch;
    -    private List<MaterializedField> outputFields;
    -    private OneSideInput leftSide;
    -    private OneSideInput rightSide;
    -    private IterOutcome upstream = IterOutcome.NOT_YET;
    -    private boolean leftIsFinish = false;
    -    private boolean rightIsFinish = false;
    -
    -    // These two schemas are obtained from the first record batches of the left and right inputs
    -    // They are used to check if the schema is changed between recordbatches
    -    private BatchSchema leftSchema;
    -    private BatchSchema rightSchema;
    -    private boolean bothEmpty = false;
    -
    -    public UnionAllInput(UnionAllRecordBatch unionAllRecordBatch, RecordBatch left, RecordBatch right) {
    -      this.unionAllRecordBatch = unionAllRecordBatch;
    -      leftSide = new OneSideInput(left);
    -      rightSide = new OneSideInput(right);
    -    }
    -
    -    private void setBothSideEmpty(boolean bothEmpty) {
    -      this.bothEmpty = bothEmpty;
    -    }
    -
    -    private boolean isBothSideEmpty() {
    -      return bothEmpty;
    -    }
    -
    -    public IterOutcome nextBatch() throws SchemaChangeException {
    -      if(upstream == RecordBatch.IterOutcome.NOT_YET) {
    -        IterOutcome iterLeft = leftSide.nextBatch();
    -        switch(iterLeft) {
    -          case OK_NEW_SCHEMA:
    -            /*
    -             * If the first few record batches are all empty,
    -             * there is no way to tell whether these empty batches are coming from empty files.
    -             * It is incorrect to infer output types when either side could be coming from empty.
    -             *
    -             * Thus, while-loop is necessary to skip those empty batches.
    -             */
    -            whileLoop:
    -            while(leftSide.getRecordBatch().getRecordCount() == 0) {
    -              iterLeft = leftSide.nextBatch();
    -
    -              switch(iterLeft) {
    -                case STOP:
    -                case OUT_OF_MEMORY:
    -                  return iterLeft;
    -
    -                case NONE:
    -                  // Special Case: The left side was an empty input.
    -                  leftIsFinish = true;
    -                  break whileLoop;
    -
    -                case NOT_YET:
    -                case OK_NEW_SCHEMA:
    -                case OK:
    -                  continue whileLoop;
    -
    -                default:
    -                  throw new IllegalStateException(
    -                      String.format("Unexpected state %s.", iterLeft));
    -              }
    -            }
    -
    -            break;
    -          case STOP:
    -          case OUT_OF_MEMORY:
    -            return iterLeft;
    -
    -          default:
    -            throw new IllegalStateException(
    -                String.format("Unexpected state %s.", iterLeft));
    -        }
    -
    -        IterOutcome iterRight = rightSide.nextBatch();
    -        switch(iterRight) {
    -          case OK_NEW_SCHEMA:
    -            // Unless there is no record batch on the left side of the inputs,
    -            // always start processing from the left side.
    -            if(leftIsFinish) {
    -              unionAllRecordBatch.setCurrentRecordBatch(rightSide.getRecordBatch());
    -            } else {
    -              unionAllRecordBatch.setCurrentRecordBatch(leftSide.getRecordBatch());
    -            }
    -            // If the record count of the first batch from right input is zero,
    -            // there are two possibilities:
    -            // 1. The right side is an empty input (e.g., file).
    -            // 2. There will be more records carried by later batches.
    -
    -            /*
    -             * If the first few record batches are all empty,
    -             * there is no way to tell whether these empty batches are coming from empty files.
    -             * It is incorrect to infer output types when either side could be coming from empty.
    -             *
    -             * Thus, while-loop is necessary to skip those empty batches.
    -             */
    -            whileLoop:
    -            while(rightSide.getRecordBatch().getRecordCount() == 0) {
    -              iterRight = rightSide.nextBatch();
    -              switch(iterRight) {
    -                case STOP:
    -                case OUT_OF_MEMORY:
    -                  return iterRight;
    -
    -                case NONE:
    -                  // Special Case: The right side was an empty input.
    -                  rightIsFinish = true;
    -                  break whileLoop;
    -
    -                case NOT_YET:
    -                case OK_NEW_SCHEMA:
    -                case OK:
    -                  continue whileLoop;
    -
    -                default:
    -                  throw new IllegalStateException(
    -                      String.format("Unexpected state %s.", iterRight));
    -              }
    -            }
    -
    -            if(leftIsFinish && rightIsFinish) {
    -              setBothSideEmpty(true);
    -            }
    -
    -            inferOutputFields();
    -            break;
    -
    -          case STOP:
    -          case OUT_OF_MEMORY:
    -            return iterRight;
    -
    -          default:
    -            throw new IllegalStateException(
    -                String.format("Unexpected state %s.", iterRight));
    -        }
    -
    -
    -
    -        upstream = IterOutcome.OK_NEW_SCHEMA;
    -        return upstream;
    -      } else {
    -        if(isBothSideEmpty()) {
    -          return IterOutcome.NONE;
    -        }
    -
    -        unionAllRecordBatch.clearCurrentRecordBatch();
    -
    -        if(leftIsFinish && rightIsFinish) {
    -          upstream = IterOutcome.NONE;
    -          return upstream;
    -        } else if(leftIsFinish) {
    -          IterOutcome iterOutcome = rightSide.nextBatch();
    -
    -          switch(iterOutcome) {
    -            case NONE:
    -              rightIsFinish = true;
    -              // fall through
    -            case STOP:
    -            case OUT_OF_MEMORY:
    -              upstream = iterOutcome;
    -              return upstream;
    -
    -            case OK_NEW_SCHEMA:
    -              if(!rightSide.getRecordBatch().getSchema().equals(rightSchema)) {
    -                throw new SchemaChangeException("Schema change detected in the right input of Union-All. This is not currently supported");
    -              }
    -              iterOutcome = IterOutcome.OK;
    -              // fall through
    -            case OK:
    -              unionAllRecordBatch.setCurrentRecordBatch(rightSide.getRecordBatch());
    -              upstream = iterOutcome;
    -              return upstream;
    -
    -            default:
    -              throw new IllegalStateException(String.format("Unknown state %s.", upstream));
    -          }
    -        } else if(rightIsFinish) {
    -          IterOutcome iterOutcome = leftSide.nextBatch();
    -          switch(iterOutcome) {
    -            case STOP:
    -            case OUT_OF_MEMORY:
    -            case NONE:
    -              upstream = iterOutcome;
    -              return upstream;
    -
    -            case OK:
    -              unionAllRecordBatch.setCurrentRecordBatch(leftSide.getRecordBatch());
    -              upstream = iterOutcome;
    -              return upstream;
    -
    -            default:
    -              throw new IllegalStateException(String.format("Unknown state %s.", iterOutcome));
    -          }
    -        } else {
    -          IterOutcome iterOutcome = leftSide.nextBatch();
    -
    -          switch(iterOutcome) {
    -            case STOP:
    -            case OUT_OF_MEMORY:
    -              upstream = iterOutcome;
    -              return upstream;
    -
    -            case OK_NEW_SCHEMA:
    -              if(!leftSide.getRecordBatch().getSchema().equals(leftSchema)) {
    -                throw new SchemaChangeException("Schema change detected in the left input of Union-All. This is not currently supported");
    -              }
    -
    -              iterOutcome = IterOutcome.OK;
    -              // fall through
    -            case OK:
    -              unionAllRecordBatch.setCurrentRecordBatch(leftSide.getRecordBatch());
    -              upstream = iterOutcome;
    -              return upstream;
    -
    -            case NONE:
    -              unionAllRecordBatch.setCurrentRecordBatch(rightSide.getRecordBatch());
    -              upstream = IterOutcome.OK;
    -              leftIsFinish = true;
    -              return upstream;
    -
    -            default:
    -              throw new IllegalStateException(String.format("Unknown state %s.", upstream));
    -          }
    -        }
    -      }
    -    }
     
    -    /**
    -     *
    -     * Summarize the inference in the four different situations:
    -     * First of all, the field names are always determined by the left side
    -     * (Even when the left side is from an empty file, we have the column names.)
    -     *
    -     * Cases:
    -     * 1. Left: non-empty; Right: non-empty
    -     *      types determined by both sides with implicit casting involved
    -     * 2. Left: empty; Right: non-empty
    -     *      type from the right
    -     * 3. Left: non-empty; Right: empty
    -     *      types from the left
    -     * 4. Left: empty; Right: empty
    -     *      types are nullable integer
    -     */
    -    private void inferOutputFields() {
    -      if(!leftIsFinish && !rightIsFinish) {
    -        // Both sides are non-empty
    -        inferOutputFieldsBothSide();
    -      } else if(!rightIsFinish) {
    -        // Left side is non-empty
    -        // While use left side's column names as output column names,
    -        // use right side's column types as output column types.
    -        inferOutputFieldsFromSingleSide(
    -            leftSide.getRecordBatch().getSchema(),
    -            rightSide.getRecordBatch().getSchema());
    +  // The output table's column names always follow the left table,
    +  // where the output type is chosen based on DRILL's implicit casting rules
    +  private void inferOutputFieldsBothSide(final BatchSchema leftSchema, final BatchSchema rightSchema) {
    +//    outputFields = Lists.newArrayList();
    +    final Iterator<MaterializedField> leftIter = leftSchema.iterator();
    +    final Iterator<MaterializedField> rightIter = rightSchema.iterator();
    +
    +    int index = 1;
    +    while (leftIter.hasNext() && rightIter.hasNext()) {
    +      MaterializedField leftField  = leftIter.next();
    +      MaterializedField rightField = rightIter.next();
    +
    +      if (hasSameTypeAndMode(leftField, rightField)) {
    --- End diff --
    
    The code here has the same "flavor" as the code in the code gen setup. Maybe add an explanation for why we need both flavors? What does the one do that the other does not do?


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] drill pull request #906: DRILL-5546: Handle schema change exception failure ...

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/906#discussion_r135575060
  
    --- Diff: exec/java-exec/src/main/java/org/apache/drill/exec/planner/physical/ProjectPrel.java ---
    @@ -35,18 +35,43 @@
     import org.apache.calcite.rex.RexNode;
     import org.apache.calcite.sql.SqlKind;
     
    +/**
    + * A physical Prel node for Project operator.
    + */
     public class ProjectPrel extends DrillProjectRelBase implements Prel{
       static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(ProjectPrel.class);
     
    +  private final boolean outputProj;
     
       public ProjectPrel(RelOptCluster cluster, RelTraitSet traits, RelNode child, List<RexNode> exps,
           RelDataType rowType) {
    +    this(cluster, traits, child, exps, rowType, false);
    +  }
    +
    +  /**
    +   * Constructor for ProjectPrel.
    +   * @param cluster
    +   * @param traits traits of ProjectPrel node
    +   * @param child  input
    +   * @param exps   list of RexNode, representing expressions of projection.
    +   * @param rowType output rowType of projection expression.
    +   * @param outputProj true if ProjectPrel is inserted by {@link org.apache.drill.exec.planner.physical.visitor.TopProjectVisitor}
    +   *                   Such top Project operator does the following processing, before the result was presented to Screen/Writer
    +   *                   1) ensure final output field names are preserved,
    +   *                   2) handle cases where input does not return any batch (a fast NONE) (see ProjectRecordBatch.handleFastNone() method)
    +   *                   3) handle cases where expressions in upstream operator were evaluated to NULL type
    --- End diff --
    
    This is Javadoc, so rather than a pre-formatted numeric list, consider \<ol>\<li>...\</li>\</ol>.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] drill pull request #906: DRILL-5546: Handle schema change exception failure ...

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

    https://github.com/apache/drill/pull/906#discussion_r136149852
  
    --- Diff: exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/union/UnionAllRecordBatch.java ---
    @@ -130,562 +145,248 @@ public IterOutcome innerNext() {
       }
     
       @Override
    -  public WritableBatch getWritableBatch() {
    -    return WritableBatch.get(this);
    +  public int getRecordCount() {
    +    return recordCount;
       }
     
    -  private void setValueCount(int count) {
    -    for (ValueVector v : allocationVectors) {
    -      ValueVector.Mutator m = v.getMutator();
    -      m.setValueCount(count);
    -    }
    -  }
     
    -  private boolean doAlloc() {
    -    for (ValueVector v : allocationVectors) {
    -      try {
    -        AllocationHelper.allocateNew(v, current.getRecordCount());
    -      } catch (OutOfMemoryException ex) {
    -        return false;
    -      }
    +  @SuppressWarnings("resource")
    +  private IterOutcome doWork(RecordBatch inputBatch, boolean newSchema) throws ClassTransformationException, IOException, SchemaChangeException {
    +    if (inputBatch.getSchema().getFieldCount() != container.getSchema().getFieldCount()) {
    +      // wrong.
         }
    -    return true;
    -  }
     
    -  @SuppressWarnings("resource")
    -  private IterOutcome doWork() throws ClassTransformationException, IOException, SchemaChangeException {
    -    if (allocationVectors != null) {
    -      for (ValueVector v : allocationVectors) {
    -        v.clear();
    -      }
    +    if (newSchema) {
    +      createUnionAller(inputBatch);
         }
     
    -    allocationVectors = Lists.newArrayList();
    -    transfers.clear();
    +    container.zeroVectors();
     
    -    // If both sides of Union-All are empty
    -    if(unionAllInput.isBothSideEmpty()) {
    -      for(int i = 0; i < outputFields.size(); ++i) {
    -        final String colName = outputFields.get(i).getPath();
    -        final MajorType majorType = MajorType.newBuilder()
    -            .setMinorType(MinorType.INT)
    -            .setMode(DataMode.OPTIONAL)
    -            .build();
    -
    -        MaterializedField outputField = MaterializedField.create(colName, majorType);
    -        ValueVector vv = container.addOrGet(outputField, callBack);
    -        allocationVectors.add(vv);
    -      }
    +    for (final ValueVector v : this.allocationVectors) {
    +      AllocationHelper.allocateNew(v, inputBatch.getRecordCount());
    +    }
    --- End diff --
    
    Sounds reasonable. I put two util method in `VectorUtil`, since `VectorAccessibleUtilities` probably should only take `VectorAccessible` related util method, as its comment stated.  


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] drill pull request #906: DRILL-5546: Handle schema change exception failure ...

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/906#discussion_r135371425
  
    --- Diff: exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/union/UnionAllRecordBatch.java ---
    @@ -130,562 +145,248 @@ public IterOutcome innerNext() {
       }
     
       @Override
    -  public WritableBatch getWritableBatch() {
    -    return WritableBatch.get(this);
    +  public int getRecordCount() {
    +    return recordCount;
       }
     
    -  private void setValueCount(int count) {
    -    for (ValueVector v : allocationVectors) {
    -      ValueVector.Mutator m = v.getMutator();
    -      m.setValueCount(count);
    -    }
    -  }
     
    -  private boolean doAlloc() {
    -    for (ValueVector v : allocationVectors) {
    -      try {
    -        AllocationHelper.allocateNew(v, current.getRecordCount());
    -      } catch (OutOfMemoryException ex) {
    -        return false;
    -      }
    +  @SuppressWarnings("resource")
    +  private IterOutcome doWork(RecordBatch inputBatch, boolean newSchema) throws ClassTransformationException, IOException, SchemaChangeException {
    +    if (inputBatch.getSchema().getFieldCount() != container.getSchema().getFieldCount()) {
    +      // wrong.
         }
    -    return true;
    -  }
     
    -  @SuppressWarnings("resource")
    -  private IterOutcome doWork() throws ClassTransformationException, IOException, SchemaChangeException {
    -    if (allocationVectors != null) {
    -      for (ValueVector v : allocationVectors) {
    -        v.clear();
    -      }
    +    if (newSchema) {
    +      createUnionAller(inputBatch);
         }
     
    -    allocationVectors = Lists.newArrayList();
    -    transfers.clear();
    +    container.zeroVectors();
     
    -    // If both sides of Union-All are empty
    -    if(unionAllInput.isBothSideEmpty()) {
    -      for(int i = 0; i < outputFields.size(); ++i) {
    -        final String colName = outputFields.get(i).getPath();
    -        final MajorType majorType = MajorType.newBuilder()
    -            .setMinorType(MinorType.INT)
    -            .setMode(DataMode.OPTIONAL)
    -            .build();
    -
    -        MaterializedField outputField = MaterializedField.create(colName, majorType);
    -        ValueVector vv = container.addOrGet(outputField, callBack);
    -        allocationVectors.add(vv);
    -      }
    +    for (final ValueVector v : this.allocationVectors) {
    +      AllocationHelper.allocateNew(v, inputBatch.getRecordCount());
    +    }
     
    -      container.buildSchema(BatchSchema.SelectionVectorMode.NONE);
    +    recordCount = unionall.unionRecords(0, inputBatch.getRecordCount(), 0);
    +    for (final ValueVector v : allocationVectors) {
    +      final ValueVector.Mutator m = v.getMutator();
    +      m.setValueCount(recordCount);
    +    }
    +
    +    if (callBack.getSchemaChangedAndReset()) {
           return IterOutcome.OK_NEW_SCHEMA;
    +    } else {
    +      return IterOutcome.OK;
         }
    +  }
    +
    +  private void createUnionAller(RecordBatch inputBatch) throws ClassTransformationException, IOException, SchemaChangeException {
    +    transfers.clear();
    +    allocationVectors.clear();;
     
         final ClassGenerator<UnionAller> cg = CodeGenerator.getRoot(UnionAller.TEMPLATE_DEFINITION, context.getFunctionRegistry(), context.getOptions());
         cg.getCodeGenerator().plainJavaCapable(true);
         // Uncomment out this line to debug the generated code.
    -//    cg.getCodeGenerator().saveCodeForDebugging(true);
    +    //    cg.getCodeGenerator().saveCodeForDebugging(true);
    +
         int index = 0;
    -    for(VectorWrapper<?> vw : current) {
    -       ValueVector vvIn = vw.getValueVector();
    -      // get the original input column names
    -      SchemaPath inputPath = SchemaPath.getSimplePath(vvIn.getField().getPath());
    -      // get the renamed column names
    -      SchemaPath outputPath = SchemaPath.getSimplePath(outputFields.get(index).getPath());
    +    for(VectorWrapper<?> vw : inputBatch) {
    +      ValueVector vvIn = vw.getValueVector();
    +      ValueVector vvOut = container.getValueVector(index).getValueVector();
     
           final ErrorCollector collector = new ErrorCollectorImpl();
           // According to input data names, Minortypes, Datamodes, choose to
           // transfer directly,
           // rename columns or
           // cast data types (Minortype or DataMode)
    -      if (hasSameTypeAndMode(outputFields.get(index), vw.getValueVector().getField())) {
    +      if (hasSameTypeAndMode(container.getSchema().getColumn(index), vvIn.getField())
    +          && vvIn.getField().getType().getMinorType() != TypeProtos.MinorType.MAP // Per DRILL-5521, existing bug for map transfer
    +          ) {
             // Transfer column
    -
    -        MajorType outputFieldType = outputFields.get(index).getType();
    -        MaterializedField outputField = MaterializedField.create(outputPath.getAsUnescapedPath(), outputFieldType);
    -
    -        /*
    -          todo: Fix if condition when DRILL-4824 is merged
    -          If condition should be changed to:
    -          `if (outputFields.get(index).getPath().equals(inputPath.getAsUnescapedPath())) {`
    -          DRILL-5419 has changed condition to correct one but this caused regression (DRILL-5521).
    -          Root cause is missing indication of child column in map types when it is null.
    -          DRILL-4824 is re-working json reader implementation, including map types and will fix this problem.
    -          Reverting condition to previous one to avoid regression till DRILL-4824 is merged.
    -          Unit test - TestJsonReader.testKvgenWithUnionAll().
    -         */
    -        if (outputFields.get(index).getPath().equals(inputPath)) {
    -          ValueVector vvOut = container.addOrGet(outputField);
    -          TransferPair tp = vvIn.makeTransferPair(vvOut);
    -          transfers.add(tp);
    +        TransferPair tp = vvIn.makeTransferPair(vvOut);
    +        transfers.add(tp);
             // Copy data in order to rename the column
    -        } else {
    -          final LogicalExpression expr = ExpressionTreeMaterializer.materialize(inputPath, current, collector, context.getFunctionRegistry() );
    -          if (collector.hasErrors()) {
    -            throw new SchemaChangeException(String.format("Failure while trying to materialize incoming schema.  Errors:\n %s.", collector.toErrorString()));
    -          }
    -
    -          ValueVector vv = container.addOrGet(outputField, callBack);
    -          allocationVectors.add(vv);
    -          TypedFieldId fid = container.getValueVectorId(SchemaPath.getSimplePath(outputField.getPath()));
    -          ValueVectorWriteExpression write = new ValueVectorWriteExpression(fid, expr, true);
    -          cg.addExpr(write);
    -        }
    -      // Cast is necessary
    +      } else if (vvIn.getField().getType().getMinorType() == TypeProtos.MinorType.NULL) {
    +        continue;
           } else {
    -        LogicalExpression expr = ExpressionTreeMaterializer.materialize(inputPath, current, collector, context.getFunctionRegistry());
    +        SchemaPath inputPath = SchemaPath.getSimplePath(vvIn.getField().getPath());
    +        MaterializedField inField = vvIn.getField();
    +        MaterializedField outputField = vvOut.getField();
    +
    +        LogicalExpression expr = ExpressionTreeMaterializer.materialize(inputPath, inputBatch, collector, context.getFunctionRegistry());
    +
             if (collector.hasErrors()) {
               throw new SchemaChangeException(String.format("Failure while trying to materialize incoming schema.  Errors:\n %s.", collector.toErrorString()));
             }
     
             // If the inputs' DataMode is required and the outputs' DataMode is not required
             // cast to the one with the least restriction
    -        if(vvIn.getField().getType().getMode() == DataMode.REQUIRED
    -            && outputFields.get(index).getType().getMode() != DataMode.REQUIRED) {
    -          expr = ExpressionTreeMaterializer.convertToNullableType(expr, vvIn.getField().getType().getMinorType(), context.getFunctionRegistry(), collector);
    +        if(inField.getType().getMode() == TypeProtos.DataMode.REQUIRED
    +            && outputField.getType().getMode() != TypeProtos.DataMode.REQUIRED) {
    +          expr = ExpressionTreeMaterializer.convertToNullableType(expr, inField.getType().getMinorType(), context.getFunctionRegistry(), collector);
               if (collector.hasErrors()) {
                 throw new SchemaChangeException(String.format("Failure while trying to materialize incoming schema.  Errors:\n %s.", collector.toErrorString()));
               }
             }
     
             // If two inputs' MinorTypes are different,
             // Insert a cast before the Union operation
    -        if(vvIn.getField().getType().getMinorType() != outputFields.get(index).getType().getMinorType()) {
    -          expr = ExpressionTreeMaterializer.addCastExpression(expr, outputFields.get(index).getType(), context.getFunctionRegistry(), collector);
    +        if(inField.getType().getMinorType() != outputField.getType().getMinorType()) {
    +          expr = ExpressionTreeMaterializer.addCastExpression(expr, outputField.getType(), context.getFunctionRegistry(), collector);
               if (collector.hasErrors()) {
                 throw new SchemaChangeException(String.format("Failure while trying to materialize incoming schema.  Errors:\n %s.", collector.toErrorString()));
               }
             }
     
    -        final MaterializedField outputField = MaterializedField.create(outputPath.getAsUnescapedPath(), expr.getMajorType());
    -        ValueVector vector = container.addOrGet(outputField, callBack);
    -        allocationVectors.add(vector);
             TypedFieldId fid = container.getValueVectorId(SchemaPath.getSimplePath(outputField.getPath()));
     
    -        boolean useSetSafe = !(vector instanceof FixedWidthVector);
    +        boolean useSetSafe = !(vvOut instanceof FixedWidthVector);
             ValueVectorWriteExpression write = new ValueVectorWriteExpression(fid, expr, useSetSafe);
             cg.addExpr(write);
    +
    +        allocationVectors.add(vvOut);
           }
           ++index;
         }
     
         unionall = context.getImplementationClass(cg.getCodeGenerator());
    -    unionall.setup(context, current, this, transfers);
    -
    -    if(!schemaAvailable) {
    -      container.buildSchema(BatchSchema.SelectionVectorMode.NONE);
    -      schemaAvailable = true;
    -    }
    -
    -    if(!doAlloc()) {
    -      return IterOutcome.OUT_OF_MEMORY;
    -    }
    -
    -    recordCount = unionall.unionRecords(0, current.getRecordCount(), 0);
    -    setValueCount(recordCount);
    -    return IterOutcome.OK;
    -  }
    -
    -  public static boolean hasSameTypeAndMode(MaterializedField leftField, MaterializedField rightField) {
    -    return (leftField.getType().getMinorType() == rightField.getType().getMinorType())
    -        && (leftField.getType().getMode() == rightField.getType().getMode());
    -  }
    -
    -  // This method is used by inner class to point the reference `current` to the correct record batch
    -  private void setCurrentRecordBatch(RecordBatch target) {
    -    this.current = target;
    -  }
    -
    -  // This method is used by inner class to clear the current record batch
    -  private void clearCurrentRecordBatch() {
    -    for(VectorWrapper<?> v: current) {
    -      v.clear();
    -    }
    +    unionall.setup(context, inputBatch, this, transfers);
       }
     
    -  public static class UnionAllInput {
    -    private UnionAllRecordBatch unionAllRecordBatch;
    -    private List<MaterializedField> outputFields;
    -    private OneSideInput leftSide;
    -    private OneSideInput rightSide;
    -    private IterOutcome upstream = IterOutcome.NOT_YET;
    -    private boolean leftIsFinish = false;
    -    private boolean rightIsFinish = false;
    -
    -    // These two schemas are obtained from the first record batches of the left and right inputs
    -    // They are used to check if the schema is changed between recordbatches
    -    private BatchSchema leftSchema;
    -    private BatchSchema rightSchema;
    -    private boolean bothEmpty = false;
    -
    -    public UnionAllInput(UnionAllRecordBatch unionAllRecordBatch, RecordBatch left, RecordBatch right) {
    -      this.unionAllRecordBatch = unionAllRecordBatch;
    -      leftSide = new OneSideInput(left);
    -      rightSide = new OneSideInput(right);
    -    }
    -
    -    private void setBothSideEmpty(boolean bothEmpty) {
    -      this.bothEmpty = bothEmpty;
    -    }
    -
    -    private boolean isBothSideEmpty() {
    -      return bothEmpty;
    -    }
    -
    -    public IterOutcome nextBatch() throws SchemaChangeException {
    -      if(upstream == RecordBatch.IterOutcome.NOT_YET) {
    -        IterOutcome iterLeft = leftSide.nextBatch();
    -        switch(iterLeft) {
    -          case OK_NEW_SCHEMA:
    -            /*
    -             * If the first few record batches are all empty,
    -             * there is no way to tell whether these empty batches are coming from empty files.
    -             * It is incorrect to infer output types when either side could be coming from empty.
    -             *
    -             * Thus, while-loop is necessary to skip those empty batches.
    -             */
    -            whileLoop:
    -            while(leftSide.getRecordBatch().getRecordCount() == 0) {
    -              iterLeft = leftSide.nextBatch();
    -
    -              switch(iterLeft) {
    -                case STOP:
    -                case OUT_OF_MEMORY:
    -                  return iterLeft;
    -
    -                case NONE:
    -                  // Special Case: The left side was an empty input.
    -                  leftIsFinish = true;
    -                  break whileLoop;
    -
    -                case NOT_YET:
    -                case OK_NEW_SCHEMA:
    -                case OK:
    -                  continue whileLoop;
    -
    -                default:
    -                  throw new IllegalStateException(
    -                      String.format("Unexpected state %s.", iterLeft));
    -              }
    -            }
    -
    -            break;
    -          case STOP:
    -          case OUT_OF_MEMORY:
    -            return iterLeft;
    -
    -          default:
    -            throw new IllegalStateException(
    -                String.format("Unexpected state %s.", iterLeft));
    -        }
    -
    -        IterOutcome iterRight = rightSide.nextBatch();
    -        switch(iterRight) {
    -          case OK_NEW_SCHEMA:
    -            // Unless there is no record batch on the left side of the inputs,
    -            // always start processing from the left side.
    -            if(leftIsFinish) {
    -              unionAllRecordBatch.setCurrentRecordBatch(rightSide.getRecordBatch());
    -            } else {
    -              unionAllRecordBatch.setCurrentRecordBatch(leftSide.getRecordBatch());
    -            }
    -            // If the record count of the first batch from right input is zero,
    -            // there are two possibilities:
    -            // 1. The right side is an empty input (e.g., file).
    -            // 2. There will be more records carried by later batches.
    -
    -            /*
    -             * If the first few record batches are all empty,
    -             * there is no way to tell whether these empty batches are coming from empty files.
    -             * It is incorrect to infer output types when either side could be coming from empty.
    -             *
    -             * Thus, while-loop is necessary to skip those empty batches.
    -             */
    -            whileLoop:
    -            while(rightSide.getRecordBatch().getRecordCount() == 0) {
    -              iterRight = rightSide.nextBatch();
    -              switch(iterRight) {
    -                case STOP:
    -                case OUT_OF_MEMORY:
    -                  return iterRight;
    -
    -                case NONE:
    -                  // Special Case: The right side was an empty input.
    -                  rightIsFinish = true;
    -                  break whileLoop;
    -
    -                case NOT_YET:
    -                case OK_NEW_SCHEMA:
    -                case OK:
    -                  continue whileLoop;
    -
    -                default:
    -                  throw new IllegalStateException(
    -                      String.format("Unexpected state %s.", iterRight));
    -              }
    -            }
    -
    -            if(leftIsFinish && rightIsFinish) {
    -              setBothSideEmpty(true);
    -            }
    -
    -            inferOutputFields();
    -            break;
    -
    -          case STOP:
    -          case OUT_OF_MEMORY:
    -            return iterRight;
    -
    -          default:
    -            throw new IllegalStateException(
    -                String.format("Unexpected state %s.", iterRight));
    -        }
    -
    -
    -
    -        upstream = IterOutcome.OK_NEW_SCHEMA;
    -        return upstream;
    -      } else {
    -        if(isBothSideEmpty()) {
    -          return IterOutcome.NONE;
    -        }
    -
    -        unionAllRecordBatch.clearCurrentRecordBatch();
    -
    -        if(leftIsFinish && rightIsFinish) {
    -          upstream = IterOutcome.NONE;
    -          return upstream;
    -        } else if(leftIsFinish) {
    -          IterOutcome iterOutcome = rightSide.nextBatch();
    -
    -          switch(iterOutcome) {
    -            case NONE:
    -              rightIsFinish = true;
    -              // fall through
    -            case STOP:
    -            case OUT_OF_MEMORY:
    -              upstream = iterOutcome;
    -              return upstream;
    -
    -            case OK_NEW_SCHEMA:
    -              if(!rightSide.getRecordBatch().getSchema().equals(rightSchema)) {
    -                throw new SchemaChangeException("Schema change detected in the right input of Union-All. This is not currently supported");
    -              }
    -              iterOutcome = IterOutcome.OK;
    -              // fall through
    -            case OK:
    -              unionAllRecordBatch.setCurrentRecordBatch(rightSide.getRecordBatch());
    -              upstream = iterOutcome;
    -              return upstream;
    -
    -            default:
    -              throw new IllegalStateException(String.format("Unknown state %s.", upstream));
    -          }
    -        } else if(rightIsFinish) {
    -          IterOutcome iterOutcome = leftSide.nextBatch();
    -          switch(iterOutcome) {
    -            case STOP:
    -            case OUT_OF_MEMORY:
    -            case NONE:
    -              upstream = iterOutcome;
    -              return upstream;
    -
    -            case OK:
    -              unionAllRecordBatch.setCurrentRecordBatch(leftSide.getRecordBatch());
    -              upstream = iterOutcome;
    -              return upstream;
    -
    -            default:
    -              throw new IllegalStateException(String.format("Unknown state %s.", iterOutcome));
    -          }
    -        } else {
    -          IterOutcome iterOutcome = leftSide.nextBatch();
    -
    -          switch(iterOutcome) {
    -            case STOP:
    -            case OUT_OF_MEMORY:
    -              upstream = iterOutcome;
    -              return upstream;
    -
    -            case OK_NEW_SCHEMA:
    -              if(!leftSide.getRecordBatch().getSchema().equals(leftSchema)) {
    -                throw new SchemaChangeException("Schema change detected in the left input of Union-All. This is not currently supported");
    -              }
    -
    -              iterOutcome = IterOutcome.OK;
    -              // fall through
    -            case OK:
    -              unionAllRecordBatch.setCurrentRecordBatch(leftSide.getRecordBatch());
    -              upstream = iterOutcome;
    -              return upstream;
    -
    -            case NONE:
    -              unionAllRecordBatch.setCurrentRecordBatch(rightSide.getRecordBatch());
    -              upstream = IterOutcome.OK;
    -              leftIsFinish = true;
    -              return upstream;
    -
    -            default:
    -              throw new IllegalStateException(String.format("Unknown state %s.", upstream));
    -          }
    -        }
    -      }
    -    }
     
    -    /**
    -     *
    -     * Summarize the inference in the four different situations:
    -     * First of all, the field names are always determined by the left side
    -     * (Even when the left side is from an empty file, we have the column names.)
    -     *
    -     * Cases:
    -     * 1. Left: non-empty; Right: non-empty
    -     *      types determined by both sides with implicit casting involved
    -     * 2. Left: empty; Right: non-empty
    -     *      type from the right
    -     * 3. Left: non-empty; Right: empty
    -     *      types from the left
    -     * 4. Left: empty; Right: empty
    -     *      types are nullable integer
    -     */
    -    private void inferOutputFields() {
    -      if(!leftIsFinish && !rightIsFinish) {
    -        // Both sides are non-empty
    -        inferOutputFieldsBothSide();
    -      } else if(!rightIsFinish) {
    -        // Left side is non-empty
    -        // While use left side's column names as output column names,
    -        // use right side's column types as output column types.
    -        inferOutputFieldsFromSingleSide(
    -            leftSide.getRecordBatch().getSchema(),
    -            rightSide.getRecordBatch().getSchema());
    +  // The output table's column names always follow the left table,
    +  // where the output type is chosen based on DRILL's implicit casting rules
    +  private void inferOutputFieldsBothSide(final BatchSchema leftSchema, final BatchSchema rightSchema) {
    +//    outputFields = Lists.newArrayList();
    +    final Iterator<MaterializedField> leftIter = leftSchema.iterator();
    +    final Iterator<MaterializedField> rightIter = rightSchema.iterator();
    +
    +    int index = 1;
    +    while (leftIter.hasNext() && rightIter.hasNext()) {
    +      MaterializedField leftField  = leftIter.next();
    +      MaterializedField rightField = rightIter.next();
    +
    +      if (hasSameTypeAndMode(leftField, rightField)) {
    +        TypeProtos.MajorType.Builder builder = TypeProtos.MajorType.newBuilder().setMinorType(leftField.getType().getMinorType()).setMode(leftField.getDataMode());
    +        builder = Types.calculateTypePrecisionAndScale(leftField.getType(), rightField.getType(), builder);
    +        container.addOrGet(MaterializedField.create(leftField.getPath(), builder.build()), callBack);
    +      } else if (Types.isUntypedNull(rightField.getType())) {
    +        container.addOrGet(leftField, callBack);
    +      } else if (Types.isUntypedNull(leftField.getType())) {
    +        container.addOrGet(MaterializedField.create(leftField.getPath(), rightField.getType()), callBack);
           } else {
    -        // Either right side is empty or both are empty
    -        // Using left side's schema is sufficient
    -        inferOutputFieldsFromSingleSide(
    -            leftSide.getRecordBatch().getSchema(),
    -            leftSide.getRecordBatch().getSchema());
    -      }
    -    }
    -
    -    // The output table's column names always follow the left table,
    -    // where the output type is chosen based on DRILL's implicit casting rules
    -    private void inferOutputFieldsBothSide() {
    -      outputFields = Lists.newArrayList();
    -      leftSchema = leftSide.getRecordBatch().getSchema();
    -      rightSchema = rightSide.getRecordBatch().getSchema();
    -      Iterator<MaterializedField> leftIter = leftSchema.iterator();
    -      Iterator<MaterializedField> rightIter = rightSchema.iterator();
    -
    -      int index = 1;
    -      while (leftIter.hasNext() && rightIter.hasNext()) {
    -        MaterializedField leftField  = leftIter.next();
    -        MaterializedField rightField = rightIter.next();
    -
    -        if (hasSameTypeAndMode(leftField, rightField)) {
    -          MajorType.Builder builder = MajorType.newBuilder().setMinorType(leftField.getType().getMinorType()).setMode(leftField.getDataMode());
    +        // If the output type is not the same,
    +        // cast the column of one of the table to a data type which is the Least Restrictive
    +        TypeProtos.MajorType.Builder builder = TypeProtos.MajorType.newBuilder();
    +        if (leftField.getType().getMinorType() == rightField.getType().getMinorType()) {
    +          builder.setMinorType(leftField.getType().getMinorType());
               builder = Types.calculateTypePrecisionAndScale(leftField.getType(), rightField.getType(), builder);
    -          outputFields.add(MaterializedField.create(leftField.getPath(), builder.build()));
             } else {
    -          // If the output type is not the same,
    -          // cast the column of one of the table to a data type which is the Least Restrictive
    -          MajorType.Builder builder = MajorType.newBuilder();
    -          if (leftField.getType().getMinorType() == rightField.getType().getMinorType()) {
    -            builder.setMinorType(leftField.getType().getMinorType());
    -            builder = Types.calculateTypePrecisionAndScale(leftField.getType(), rightField.getType(), builder);
    -          } else {
    -            List<MinorType> types = Lists.newLinkedList();
    -            types.add(leftField.getType().getMinorType());
    -            types.add(rightField.getType().getMinorType());
    -            MinorType outputMinorType = TypeCastRules.getLeastRestrictiveType(types);
    -            if (outputMinorType == null) {
    -              throw new DrillRuntimeException("Type mismatch between " + leftField.getType().getMinorType().toString() +
    -                  " on the left side and " + rightField.getType().getMinorType().toString() +
    -                  " on the right side in column " + index + " of UNION ALL");
    -            }
    -            builder.setMinorType(outputMinorType);
    +          List<TypeProtos.MinorType> types = Lists.newLinkedList();
    +          types.add(leftField.getType().getMinorType());
    +          types.add(rightField.getType().getMinorType());
    +          TypeProtos.MinorType outputMinorType = TypeCastRules.getLeastRestrictiveType(types);
    +          if (outputMinorType == null) {
    +            throw new DrillRuntimeException("Type mismatch between " + leftField.getType().getMinorType().toString() +
    +                " on the left side and " + rightField.getType().getMinorType().toString() +
    +                " on the right side in column " + index + " of UNION ALL");
               }
    -
    -          // The output data mode should be as flexible as the more flexible one from the two input tables
    -          List<DataMode> dataModes = Lists.newLinkedList();
    -          dataModes.add(leftField.getType().getMode());
    -          dataModes.add(rightField.getType().getMode());
    -          builder.setMode(TypeCastRules.getLeastRestrictiveDataMode(dataModes));
    -
    -          outputFields.add(MaterializedField.create(leftField.getPath(), builder.build()));
    +          builder.setMinorType(outputMinorType);
             }
    -        ++index;
    -      }
    -
    -      assert !leftIter.hasNext() && ! rightIter.hasNext() : "Mis-match of column count should have been detected when validating sqlNode at planning";
    -    }
     
    -    private void inferOutputFieldsFromSingleSide(final BatchSchema schemaForNames, final BatchSchema schemaForTypes) {
    -      outputFields = Lists.newArrayList();
    +        // The output data mode should be as flexible as the more flexible one from the two input tables
    +        List<TypeProtos.DataMode> dataModes = Lists.newLinkedList();
    +        dataModes.add(leftField.getType().getMode());
    +        dataModes.add(rightField.getType().getMode());
    +        builder.setMode(TypeCastRules.getLeastRestrictiveDataMode(dataModes));
     
    -      final List<String> outputColumnNames = Lists.newArrayList();
    -      final Iterator<MaterializedField> iterForNames = schemaForNames.iterator();
    -      while(iterForNames.hasNext()) {
    -        outputColumnNames.add(iterForNames.next().getPath());
    -      }
    -
    -      final Iterator<MaterializedField> iterForTypes = schemaForTypes.iterator();
    -      for(int i = 0; iterForTypes.hasNext(); ++i) {
    -        MaterializedField field = iterForTypes.next();
    -        outputFields.add(MaterializedField.create(outputColumnNames.get(i), field.getType()));
    +        container.addOrGet(MaterializedField.create(leftField.getPath(), builder.build()), callBack);
           }
    +      ++index;
         }
     
    -    public List<MaterializedField> getOutputFields() {
    -      if(outputFields == null) {
    -        throw new NullPointerException("Output fields have not been inferred");
    -      }
    -
    -      return outputFields;
    -    }
    +    assert !leftIter.hasNext() && ! rightIter.hasNext() : "Mis-match of column count should have been detected when validating sqlNode at planning";
    --- End diff --
    
    True? Even in the case of a `SELECT *`? Any other cases where the set of columns isn't known at plan time?


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] drill pull request #906: DRILL-5546: Handle schema change exception failure ...

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/906#discussion_r135369394
  
    --- Diff: exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/union/UnionAllRecordBatch.java ---
    @@ -39,88 +35,107 @@
     import org.apache.drill.exec.expr.ValueVectorWriteExpression;
     import org.apache.drill.exec.ops.FragmentContext;
     import org.apache.drill.exec.physical.config.UnionAll;
    -import org.apache.drill.exec.record.AbstractRecordBatch;
    +import org.apache.drill.exec.record.AbstractBinaryRecordBatch;
     import org.apache.drill.exec.record.BatchSchema;
     import org.apache.drill.exec.record.MaterializedField;
     import org.apache.drill.exec.record.RecordBatch;
     import org.apache.drill.exec.record.TransferPair;
     import org.apache.drill.exec.record.TypedFieldId;
     import org.apache.drill.exec.record.VectorWrapper;
    -import org.apache.drill.exec.record.WritableBatch;
    -import org.apache.drill.exec.record.selection.SelectionVector2;
    -import org.apache.drill.exec.record.selection.SelectionVector4;
     import org.apache.drill.exec.resolver.TypeCastRules;
     import org.apache.drill.exec.vector.AllocationHelper;
     import org.apache.drill.exec.vector.FixedWidthVector;
     import org.apache.drill.exec.vector.SchemaChangeCallBack;
     import org.apache.drill.exec.vector.ValueVector;
     
    -import com.google.common.collect.Lists;
    +import java.io.IOException;
    +import java.util.Iterator;
    +import java.util.List;
    +import java.util.NoSuchElementException;
    +import java.util.Stack;
     
    -public class UnionAllRecordBatch extends AbstractRecordBatch<UnionAll> {
    +public class UnionAllRecordBatch extends AbstractBinaryRecordBatch<UnionAll> {
       private static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(UnionAllRecordBatch.class);
     
    -  private List<MaterializedField> outputFields;
    +  private SchemaChangeCallBack callBack = new SchemaChangeCallBack();
       private UnionAller unionall;
    -  private UnionAllInput unionAllInput;
    -  private RecordBatch current;
    -
       private final List<TransferPair> transfers = Lists.newArrayList();
    -  private List<ValueVector> allocationVectors;
    -  protected SchemaChangeCallBack callBack = new SchemaChangeCallBack();
    +  private List<ValueVector> allocationVectors = Lists.newArrayList();
       private int recordCount = 0;
    -  private boolean schemaAvailable = false;
    +  private UnionInputIterator unionInputIterator;
     
       public UnionAllRecordBatch(UnionAll config, List<RecordBatch> children, FragmentContext context) throws OutOfMemoryException {
    -    super(config, context, false);
    -    assert (children.size() == 2) : "The number of the operands of Union must be 2";
    -    unionAllInput = new UnionAllInput(this, children.get(0), children.get(1));
    -  }
    -
    -  @Override
    -  public int getRecordCount() {
    -    return recordCount;
    +    super(config, context, true, children.get(0), children.get(1));
       }
     
       @Override
       protected void killIncoming(boolean sendUpstream) {
    -    unionAllInput.getLeftRecordBatch().kill(sendUpstream);
    -    unionAllInput.getRightRecordBatch().kill(sendUpstream);
    +    left.kill(sendUpstream);
    +    right.kill(sendUpstream);
       }
     
    -  @Override
    -  public SelectionVector2 getSelectionVector2() {
    -    throw new UnsupportedOperationException("UnionAllRecordBatch does not support selection vector");
    -  }
    +  protected void buildSchema() throws SchemaChangeException {
    +    if (! prefetchFirstBatchFromBothSides()) {
    +      return;
    +    }
     
    -  @Override
    -  public SelectionVector4 getSelectionVector4() {
    -    throw new UnsupportedOperationException("UnionAllRecordBatch does not support selection vector");
    +    unionInputIterator = new UnionInputIterator(leftUpstream, left, rightUpstream, right);
    +
    +    if (leftUpstream == IterOutcome.NONE && rightUpstream == IterOutcome.OK_NEW_SCHEMA) {
    +      inferOutputFieldsOneSide(right.getSchema());
    +    } else if (rightUpstream == IterOutcome.NONE && leftUpstream == IterOutcome.OK_NEW_SCHEMA) {
    +      inferOutputFieldsOneSide((left.getSchema()));
    +    } else if (leftUpstream == IterOutcome.OK_NEW_SCHEMA && rightUpstream == IterOutcome.OK_NEW_SCHEMA) {
    +      inferOutputFieldsBothSide(left.getSchema(), right.getSchema());
    +    }
    +
    +    container.buildSchema(BatchSchema.SelectionVectorMode.NONE);
    +
    +    for (VectorWrapper vv: container) {
    +      vv.getValueVector().allocateNew();
    +      vv.getValueVector().getMutator().setValueCount(0);
    +    }
    --- End diff --
    
    ```
    VectorAccessibleUtilities.allocateVectors(container, 0)
    VectorAccessibleUtilities.setValueCount(container,0)
    ```
    
    Hides a bit of the implementation details to use common functions rather than copying the code each time...


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] drill pull request #906: DRILL-5546: Handle schema change exception failure ...

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

    https://github.com/apache/drill/pull/906#discussion_r136184973
  
    --- Diff: exec/java-exec/src/main/java/org/apache/drill/exec/record/AbstractBinaryRecordBatch.java ---
    @@ -0,0 +1,75 @@
    +/**
    + * 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.record;
    +
    +import org.apache.drill.exec.exception.OutOfMemoryException;
    +import org.apache.drill.exec.ops.FragmentContext;
    +import org.apache.drill.exec.physical.base.PhysicalOperator;
    +
    +public abstract class AbstractBinaryRecordBatch<T extends PhysicalOperator> extends  AbstractRecordBatch<T> {
    +  protected final RecordBatch left;
    +  protected final RecordBatch right;
    +
    +  // state (IterOutcome) of the left input
    +  protected IterOutcome leftUpstream = IterOutcome.NONE;
    +
    +  // state (IterOutcome) of the right input
    +  protected IterOutcome rightUpstream = IterOutcome.NONE;
    --- End diff --
    
    Better not wrap the two sides in a wrapper class, since the sub-class has to differentiate the two sides. (for instance in HashJoin, left is probe while right is build)



---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] drill pull request #906: DRILL-5546: Handle schema change exception failure ...

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/906#discussion_r134298612
  
    --- Diff: exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/ScanBatch.java ---
    @@ -152,97 +157,75 @@ public void kill(boolean sendUpstream) {
         }
       }
     
    -  private void releaseAssets() {
    -    container.zeroVectors();
    -  }
    -
    -  private void clearFieldVectorMap() {
    -    for (final ValueVector v : mutator.fieldVectorMap().values()) {
    -      v.clear();
    -    }
    -  }
    -
       @Override
       public IterOutcome next() {
         if (done) {
           return IterOutcome.NONE;
         }
         oContext.getStats().startProcessing();
         try {
    -      try {
    -        injector.injectChecked(context.getExecutionControls(), "next-allocate", OutOfMemoryException.class);
    -
    -        currentReader.allocate(mutator.fieldVectorMap());
    -      } catch (OutOfMemoryException e) {
    -        clearFieldVectorMap();
    -        throw UserException.memoryError(e).build(logger);
    -      }
    -      while ((recordCount = currentReader.next()) == 0) {
    +      while (true) {
             try {
    -          if (!readers.hasNext()) {
    -            // We're on the last reader, and it has no (more) rows.
    -            currentReader.close();
    -            releaseAssets();
    -            done = true;  // have any future call to next() return NONE
    -
    -            if (mutator.isNewSchema()) {
    -              // This last reader has a new schema (e.g., we have a zero-row
    -              // file or other source).  (Note that some sources have a non-
    -              // null/non-trivial schema even when there are no rows.)
    +          injector.injectChecked(context.getExecutionControls(), "next-allocate", OutOfMemoryException.class);
    +          currentReader.allocate(mutator.fieldVectorMap());
    +        } catch (OutOfMemoryException e) {
    +          clearFieldVectorMap();
    +          throw UserException.memoryError(e).build(logger);
    +        }
     
    -              container.buildSchema(SelectionVectorMode.NONE);
    -              schema = container.getSchema();
    +        recordCount = currentReader.next();
    +        Preconditions.checkArgument(recordCount >= 0,
    +            "recordCount from RecordReader.next() should not be negative");
     
    -              return IterOutcome.OK_NEW_SCHEMA;
    -            }
    -            return IterOutcome.NONE;
    -          }
    -          // At this point, the reader that hit its end is not the last reader.
    +        boolean isNewRegularSchema = mutator.isNewSchema();
    +        // We should skip the reader, when recordCount = 0 && ! isNewRegularSchema.
    +        // Add/set implicit column vectors, only when reader gets > 0 row, or
    +        // when reader gets 0 row but with a schema with new field added
    +        if (recordCount > 0 || isNewRegularSchema) {
    +          addImplicitVectors();
    +          populateImplicitVectors();
    +        }
     
    -          // If all the files we have read so far are just empty, the schema is not useful
    -          if (! hasReadNonEmptyFile) {
    -            container.clear();
    -            clearFieldVectorMap();
    -            mutator.clear();
    -          }
    +        boolean isNewImplicitSchema = mutator.isNewSchema();
    +        for (VectorWrapper<?> w : container) {
    +          w.getValueVector().getMutator().setValueCount(recordCount);
    +        }
    +        final boolean isNewSchema = isNewRegularSchema || isNewImplicitSchema;
    +        oContext.getStats().batchReceived(0, recordCount, isNewSchema);
     
    +        if (recordCount == 0) {
               currentReader.close();
    -          currentReader = readers.next();
    -          implicitValues = implicitColumns.hasNext() ? implicitColumns.next() : null;
    -          currentReader.setup(oContext, mutator);
    -          try {
    -            currentReader.allocate(mutator.fieldVectorMap());
    -          } catch (OutOfMemoryException e) {
    -            clearFieldVectorMap();
    -            throw UserException.memoryError(e).build(logger);
    +          if (isNewSchema) {
    +            // current reader presents a new schema in mutator even though it has 0 row.
    +            // This could happen when data sources have a non-trivial schema with 0 row.
    +            container.buildSchema(SelectionVectorMode.NONE);
    +            schema = container.getSchema();
    +            if (readers.hasNext()) {
    +              advanceNextReader();
    +            } else {
    +              done = true;  // indicates the follow-up next() call will return IterOutcome.NONE.
    +            }
    +            return IterOutcome.OK_NEW_SCHEMA;
    +          } else { // not a new schema
    +            if (readers.hasNext()) {
    +              advanceNextReader();
    +              continue; // skip reader returning 0 row and having same schema.
    +                        // Skip to next loop iteration with next available reader.
    +            } else {
    +              releaseAssets(); // All data has been read. Release resource.
    +              return IterOutcome.NONE;
    +            }
    +          }
    +        } else { // recordCount > 0
    +          if (isNewSchema) {
    --- End diff --
    
    Looks like we have two copies of these two cases:
    
    * 0 rows and new schema
    * 0 rows and not new schema
    
    Can we use the state suggestion above to allow a single copy of the code for each path? Far easier to maintain (and review!) that way...


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] drill pull request #906: DRILL-5546: Handle schema change exception failure ...

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

    https://github.com/apache/drill/pull/906#discussion_r136151330
  
    --- Diff: exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/union/UnionAllRecordBatch.java ---
    @@ -130,562 +145,248 @@ public IterOutcome innerNext() {
       }
     
       @Override
    -  public WritableBatch getWritableBatch() {
    -    return WritableBatch.get(this);
    +  public int getRecordCount() {
    +    return recordCount;
       }
     
    -  private void setValueCount(int count) {
    -    for (ValueVector v : allocationVectors) {
    -      ValueVector.Mutator m = v.getMutator();
    -      m.setValueCount(count);
    -    }
    -  }
     
    -  private boolean doAlloc() {
    -    for (ValueVector v : allocationVectors) {
    -      try {
    -        AllocationHelper.allocateNew(v, current.getRecordCount());
    -      } catch (OutOfMemoryException ex) {
    -        return false;
    -      }
    +  @SuppressWarnings("resource")
    +  private IterOutcome doWork(RecordBatch inputBatch, boolean newSchema) throws ClassTransformationException, IOException, SchemaChangeException {
    +    if (inputBatch.getSchema().getFieldCount() != container.getSchema().getFieldCount()) {
    +      // wrong.
         }
    -    return true;
    -  }
     
    -  @SuppressWarnings("resource")
    -  private IterOutcome doWork() throws ClassTransformationException, IOException, SchemaChangeException {
    -    if (allocationVectors != null) {
    -      for (ValueVector v : allocationVectors) {
    -        v.clear();
    -      }
    +    if (newSchema) {
    +      createUnionAller(inputBatch);
         }
     
    -    allocationVectors = Lists.newArrayList();
    -    transfers.clear();
    +    container.zeroVectors();
     
    -    // If both sides of Union-All are empty
    -    if(unionAllInput.isBothSideEmpty()) {
    -      for(int i = 0; i < outputFields.size(); ++i) {
    -        final String colName = outputFields.get(i).getPath();
    -        final MajorType majorType = MajorType.newBuilder()
    -            .setMinorType(MinorType.INT)
    -            .setMode(DataMode.OPTIONAL)
    -            .build();
    -
    -        MaterializedField outputField = MaterializedField.create(colName, majorType);
    -        ValueVector vv = container.addOrGet(outputField, callBack);
    -        allocationVectors.add(vv);
    -      }
    +    for (final ValueVector v : this.allocationVectors) {
    +      AllocationHelper.allocateNew(v, inputBatch.getRecordCount());
    +    }
     
    -      container.buildSchema(BatchSchema.SelectionVectorMode.NONE);
    +    recordCount = unionall.unionRecords(0, inputBatch.getRecordCount(), 0);
    +    for (final ValueVector v : allocationVectors) {
    +      final ValueVector.Mutator m = v.getMutator();
    +      m.setValueCount(recordCount);
    +    }
    +
    +    if (callBack.getSchemaChangedAndReset()) {
           return IterOutcome.OK_NEW_SCHEMA;
    +    } else {
    +      return IterOutcome.OK;
         }
    +  }
    +
    +  private void createUnionAller(RecordBatch inputBatch) throws ClassTransformationException, IOException, SchemaChangeException {
    +    transfers.clear();
    +    allocationVectors.clear();;
     
         final ClassGenerator<UnionAller> cg = CodeGenerator.getRoot(UnionAller.TEMPLATE_DEFINITION, context.getFunctionRegistry(), context.getOptions());
         cg.getCodeGenerator().plainJavaCapable(true);
         // Uncomment out this line to debug the generated code.
    -//    cg.getCodeGenerator().saveCodeForDebugging(true);
    +    //    cg.getCodeGenerator().saveCodeForDebugging(true);
    +
         int index = 0;
    -    for(VectorWrapper<?> vw : current) {
    -       ValueVector vvIn = vw.getValueVector();
    -      // get the original input column names
    -      SchemaPath inputPath = SchemaPath.getSimplePath(vvIn.getField().getPath());
    -      // get the renamed column names
    -      SchemaPath outputPath = SchemaPath.getSimplePath(outputFields.get(index).getPath());
    +    for(VectorWrapper<?> vw : inputBatch) {
    +      ValueVector vvIn = vw.getValueVector();
    +      ValueVector vvOut = container.getValueVector(index).getValueVector();
     
           final ErrorCollector collector = new ErrorCollectorImpl();
           // According to input data names, Minortypes, Datamodes, choose to
           // transfer directly,
           // rename columns or
           // cast data types (Minortype or DataMode)
    -      if (hasSameTypeAndMode(outputFields.get(index), vw.getValueVector().getField())) {
    +      if (hasSameTypeAndMode(container.getSchema().getColumn(index), vvIn.getField())
    +          && vvIn.getField().getType().getMinorType() != TypeProtos.MinorType.MAP // Per DRILL-5521, existing bug for map transfer
    +          ) {
             // Transfer column
    -
    -        MajorType outputFieldType = outputFields.get(index).getType();
    -        MaterializedField outputField = MaterializedField.create(outputPath.getAsUnescapedPath(), outputFieldType);
    -
    -        /*
    -          todo: Fix if condition when DRILL-4824 is merged
    -          If condition should be changed to:
    -          `if (outputFields.get(index).getPath().equals(inputPath.getAsUnescapedPath())) {`
    -          DRILL-5419 has changed condition to correct one but this caused regression (DRILL-5521).
    -          Root cause is missing indication of child column in map types when it is null.
    -          DRILL-4824 is re-working json reader implementation, including map types and will fix this problem.
    -          Reverting condition to previous one to avoid regression till DRILL-4824 is merged.
    -          Unit test - TestJsonReader.testKvgenWithUnionAll().
    -         */
    -        if (outputFields.get(index).getPath().equals(inputPath)) {
    -          ValueVector vvOut = container.addOrGet(outputField);
    -          TransferPair tp = vvIn.makeTransferPair(vvOut);
    -          transfers.add(tp);
    +        TransferPair tp = vvIn.makeTransferPair(vvOut);
    +        transfers.add(tp);
             // Copy data in order to rename the column
    -        } else {
    -          final LogicalExpression expr = ExpressionTreeMaterializer.materialize(inputPath, current, collector, context.getFunctionRegistry() );
    -          if (collector.hasErrors()) {
    -            throw new SchemaChangeException(String.format("Failure while trying to materialize incoming schema.  Errors:\n %s.", collector.toErrorString()));
    -          }
    -
    -          ValueVector vv = container.addOrGet(outputField, callBack);
    -          allocationVectors.add(vv);
    -          TypedFieldId fid = container.getValueVectorId(SchemaPath.getSimplePath(outputField.getPath()));
    -          ValueVectorWriteExpression write = new ValueVectorWriteExpression(fid, expr, true);
    -          cg.addExpr(write);
    -        }
    -      // Cast is necessary
    +      } else if (vvIn.getField().getType().getMinorType() == TypeProtos.MinorType.NULL) {
    +        continue;
           } else {
    -        LogicalExpression expr = ExpressionTreeMaterializer.materialize(inputPath, current, collector, context.getFunctionRegistry());
    +        SchemaPath inputPath = SchemaPath.getSimplePath(vvIn.getField().getPath());
    +        MaterializedField inField = vvIn.getField();
    +        MaterializedField outputField = vvOut.getField();
    +
    +        LogicalExpression expr = ExpressionTreeMaterializer.materialize(inputPath, inputBatch, collector, context.getFunctionRegistry());
    +
             if (collector.hasErrors()) {
               throw new SchemaChangeException(String.format("Failure while trying to materialize incoming schema.  Errors:\n %s.", collector.toErrorString()));
             }
     
             // If the inputs' DataMode is required and the outputs' DataMode is not required
             // cast to the one with the least restriction
    -        if(vvIn.getField().getType().getMode() == DataMode.REQUIRED
    -            && outputFields.get(index).getType().getMode() != DataMode.REQUIRED) {
    -          expr = ExpressionTreeMaterializer.convertToNullableType(expr, vvIn.getField().getType().getMinorType(), context.getFunctionRegistry(), collector);
    +        if(inField.getType().getMode() == TypeProtos.DataMode.REQUIRED
    --- End diff --
    
    This logic actually was in the original UnionAll operator, I simply change the format. 
    
    On the other hand, if there is a `new` and `faster` way for non-nullable to nullable conversions proposed in DRILL-5709, will it make more sense for DRILL-5709 to take care of that, since it's  probably not the scope of DRILL-5546, and more appropriate for DRILL-5709? 



---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] drill pull request #906: DRILL-5546: Handle schema change exception failure ...

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

    https://github.com/apache/drill/pull/906#discussion_r136185139
  
    --- Diff: exec/java-exec/src/main/java/org/apache/drill/exec/record/AbstractSingleRecordBatch.java ---
    @@ -61,6 +63,10 @@ public IterOutcome innerNext() {
         }
         switch (upstream) {
         case NONE:
    +      if (state == BatchState.FIRST) {
    +        return handleFastNone();
    --- End diff --
    
    Changed to `handleNullInput`


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] drill pull request #906: DRILL-5546: Handle schema change exception failure ...

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

    https://github.com/apache/drill/pull/906#discussion_r135974310
  
    --- Diff: exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/project/ProjectRecordBatch.java ---
    @@ -768,4 +765,73 @@ else if (exprHasPrefix && refHasPrefix) {
           }
         }
       }
    +
    +  /**
    +   * handle FAST NONE specially when Project for query output. This happens when input returns a
    +   * FAST NONE directly ( input does not return any batch with schema/data).
    +   *
    +   * Project operator has to return a batch with schema derived using the following 3 rules:
    +   *  Case 1:  *  ==>  expand into an empty list of columns.
    +   *  Case 2:  regular column reference ==> treat as nullable-int column
    +   *  Case 3:  expressions => Call ExpressionTreeMaterialization over an empty vector contain.
    +   *           Once the expression is materialized without error, use the output type of materialized
    +   *           expression.
    +   * The batch is constructed with the above rules, and recordCount = 0.
    +   * Returned with OK_NEW_SCHEMA to down-stream operator.
    +   */
    +  @Override
    +  protected IterOutcome handleFastNone() {
    +    if (! popConfig.isOutputProj()) {
    +      return super.handleFastNone();
    +    }
    +
    +    allocationVectors = new ArrayList<>();
    +    final List<NamedExpression> exprs = getExpressionList();
    +    final ErrorCollector collector = new ErrorCollectorImpl();
    +    VectorContainer fakedIncomingVC = new VectorContainer();
    +
    +    for (NamedExpression namedExpression : exprs) {
    +      if (namedExpression.getExpr() instanceof SchemaPath) {
    +        final NameSegment expr = ((SchemaPath) namedExpression.getExpr()).getRootSegment();
    +        if (expr.getPath().contains(StarColumnHelper.STAR_COLUMN)) {
    --- End diff --
    
    This code has been removed in revised patch. 


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] drill pull request #906: DRILL-5546: Handle schema change exception failure ...

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

    https://github.com/apache/drill/pull/906#discussion_r136182353
  
    --- Diff: exec/java-exec/src/main/java/org/apache/drill/exec/planner/physical/ProjectPrel.java ---
    @@ -35,18 +35,43 @@
     import org.apache.calcite.rex.RexNode;
     import org.apache.calcite.sql.SqlKind;
     
    +/**
    + * A physical Prel node for Project operator.
    + */
     public class ProjectPrel extends DrillProjectRelBase implements Prel{
       static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(ProjectPrel.class);
     
    +  private final boolean outputProj;
     
       public ProjectPrel(RelOptCluster cluster, RelTraitSet traits, RelNode child, List<RexNode> exps,
           RelDataType rowType) {
    +    this(cluster, traits, child, exps, rowType, false);
    +  }
    +
    +  /**
    +   * Constructor for ProjectPrel.
    +   * @param cluster
    +   * @param traits traits of ProjectPrel node
    +   * @param child  input
    +   * @param exps   list of RexNode, representing expressions of projection.
    +   * @param rowType output rowType of projection expression.
    +   * @param outputProj true if ProjectPrel is inserted by {@link org.apache.drill.exec.planner.physical.visitor.TopProjectVisitor}
    +   *                   Such top Project operator does the following processing, before the result was presented to Screen/Writer
    +   *                   1) ensure final output field names are preserved,
    +   *                   2) handle cases where input does not return any batch (a fast NONE) (see ProjectRecordBatch.handleFastNone() method)
    +   *                   3) handle cases where expressions in upstream operator were evaluated to NULL type
    --- End diff --
    
    Done. 


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] drill pull request #906: DRILL-5546: Handle schema change exception failure ...

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/906#discussion_r134627033
  
    --- Diff: exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/project/ProjectRecordBatch.java ---
    @@ -768,4 +765,73 @@ else if (exprHasPrefix && refHasPrefix) {
           }
         }
       }
    +
    +  /**
    +   * handle FAST NONE specially when Project for query output. This happens when input returns a
    +   * FAST NONE directly ( input does not return any batch with schema/data).
    +   *
    --- End diff --
    
    \<p> between paragraphs, since Javadoc is HTML.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] drill pull request #906: DRILL-5546: Handle schema change exception failure ...

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

    https://github.com/apache/drill/pull/906#discussion_r136149953
  
    --- Diff: exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/union/UnionAllRecordBatch.java ---
    @@ -130,562 +145,248 @@ public IterOutcome innerNext() {
       }
     
       @Override
    -  public WritableBatch getWritableBatch() {
    -    return WritableBatch.get(this);
    +  public int getRecordCount() {
    +    return recordCount;
       }
     
    -  private void setValueCount(int count) {
    -    for (ValueVector v : allocationVectors) {
    -      ValueVector.Mutator m = v.getMutator();
    -      m.setValueCount(count);
    -    }
    -  }
     
    -  private boolean doAlloc() {
    -    for (ValueVector v : allocationVectors) {
    -      try {
    -        AllocationHelper.allocateNew(v, current.getRecordCount());
    -      } catch (OutOfMemoryException ex) {
    -        return false;
    -      }
    +  @SuppressWarnings("resource")
    +  private IterOutcome doWork(RecordBatch inputBatch, boolean newSchema) throws ClassTransformationException, IOException, SchemaChangeException {
    +    if (inputBatch.getSchema().getFieldCount() != container.getSchema().getFieldCount()) {
    +      // wrong.
         }
    -    return true;
    -  }
     
    -  @SuppressWarnings("resource")
    -  private IterOutcome doWork() throws ClassTransformationException, IOException, SchemaChangeException {
    -    if (allocationVectors != null) {
    -      for (ValueVector v : allocationVectors) {
    -        v.clear();
    -      }
    +    if (newSchema) {
    +      createUnionAller(inputBatch);
         }
     
    -    allocationVectors = Lists.newArrayList();
    -    transfers.clear();
    +    container.zeroVectors();
     
    -    // If both sides of Union-All are empty
    -    if(unionAllInput.isBothSideEmpty()) {
    -      for(int i = 0; i < outputFields.size(); ++i) {
    -        final String colName = outputFields.get(i).getPath();
    -        final MajorType majorType = MajorType.newBuilder()
    -            .setMinorType(MinorType.INT)
    -            .setMode(DataMode.OPTIONAL)
    -            .build();
    -
    -        MaterializedField outputField = MaterializedField.create(colName, majorType);
    -        ValueVector vv = container.addOrGet(outputField, callBack);
    -        allocationVectors.add(vv);
    -      }
    +    for (final ValueVector v : this.allocationVectors) {
    +      AllocationHelper.allocateNew(v, inputBatch.getRecordCount());
    +    }
     
    -      container.buildSchema(BatchSchema.SelectionVectorMode.NONE);
    +    recordCount = unionall.unionRecords(0, inputBatch.getRecordCount(), 0);
    +    for (final ValueVector v : allocationVectors) {
    +      final ValueVector.Mutator m = v.getMutator();
    +      m.setValueCount(recordCount);
    +    }
    --- End diff --
    
    New util method in `VectorUtil`. 


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] drill pull request #906: DRILL-5546: Handle schema change exception failure ...

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/906#discussion_r134295508
  
    --- Diff: exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/ScanBatch.java ---
    @@ -152,97 +157,75 @@ public void kill(boolean sendUpstream) {
         }
       }
     
    -  private void releaseAssets() {
    -    container.zeroVectors();
    -  }
    -
    -  private void clearFieldVectorMap() {
    -    for (final ValueVector v : mutator.fieldVectorMap().values()) {
    -      v.clear();
    -    }
    -  }
    -
       @Override
       public IterOutcome next() {
         if (done) {
           return IterOutcome.NONE;
         }
         oContext.getStats().startProcessing();
         try {
    -      try {
    -        injector.injectChecked(context.getExecutionControls(), "next-allocate", OutOfMemoryException.class);
    -
    -        currentReader.allocate(mutator.fieldVectorMap());
    -      } catch (OutOfMemoryException e) {
    -        clearFieldVectorMap();
    -        throw UserException.memoryError(e).build(logger);
    -      }
    -      while ((recordCount = currentReader.next()) == 0) {
    +      while (true) {
    --- End diff --
    
    The original logic allocates a batch once per call to `next()`. The new path allocates vectors once per pass though this loop. Is this desired? If we make a single pass though the loop, then all is fine. If we make two passes through the loop, which code releases the vectors allocated on the first pass?


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] drill pull request #906: DRILL-5546: Handle schema change exception failure ...

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

    https://github.com/apache/drill/pull/906#discussion_r135974133
  
    --- Diff: exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/project/ProjectRecordBatch.java ---
    @@ -768,4 +765,73 @@ else if (exprHasPrefix && refHasPrefix) {
           }
         }
       }
    +
    +  /**
    +   * handle FAST NONE specially when Project for query output. This happens when input returns a
    +   * FAST NONE directly ( input does not return any batch with schema/data).
    +   *
    +   * Project operator has to return a batch with schema derived using the following 3 rules:
    +   *  Case 1:  *  ==>  expand into an empty list of columns.
    +   *  Case 2:  regular column reference ==> treat as nullable-int column
    +   *  Case 3:  expressions => Call ExpressionTreeMaterialization over an empty vector contain.
    --- End diff --
    
    Done. 


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] drill pull request #906: DRILL-5546: Handle schema change exception failure ...

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/906#discussion_r135367166
  
    --- Diff: exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/union/UnionAllRecordBatch.java ---
    @@ -130,562 +145,248 @@ public IterOutcome innerNext() {
       }
     
       @Override
    -  public WritableBatch getWritableBatch() {
    -    return WritableBatch.get(this);
    +  public int getRecordCount() {
    +    return recordCount;
       }
     
    -  private void setValueCount(int count) {
    -    for (ValueVector v : allocationVectors) {
    -      ValueVector.Mutator m = v.getMutator();
    -      m.setValueCount(count);
    -    }
    -  }
     
    -  private boolean doAlloc() {
    -    for (ValueVector v : allocationVectors) {
    -      try {
    -        AllocationHelper.allocateNew(v, current.getRecordCount());
    -      } catch (OutOfMemoryException ex) {
    -        return false;
    -      }
    +  @SuppressWarnings("resource")
    +  private IterOutcome doWork(RecordBatch inputBatch, boolean newSchema) throws ClassTransformationException, IOException, SchemaChangeException {
    +    if (inputBatch.getSchema().getFieldCount() != container.getSchema().getFieldCount()) {
    +      // wrong.
         }
    -    return true;
    -  }
     
    -  @SuppressWarnings("resource")
    -  private IterOutcome doWork() throws ClassTransformationException, IOException, SchemaChangeException {
    -    if (allocationVectors != null) {
    -      for (ValueVector v : allocationVectors) {
    -        v.clear();
    -      }
    +    if (newSchema) {
    +      createUnionAller(inputBatch);
         }
     
    -    allocationVectors = Lists.newArrayList();
    -    transfers.clear();
    +    container.zeroVectors();
     
    -    // If both sides of Union-All are empty
    -    if(unionAllInput.isBothSideEmpty()) {
    -      for(int i = 0; i < outputFields.size(); ++i) {
    -        final String colName = outputFields.get(i).getPath();
    -        final MajorType majorType = MajorType.newBuilder()
    -            .setMinorType(MinorType.INT)
    -            .setMode(DataMode.OPTIONAL)
    -            .build();
    -
    -        MaterializedField outputField = MaterializedField.create(colName, majorType);
    -        ValueVector vv = container.addOrGet(outputField, callBack);
    -        allocationVectors.add(vv);
    -      }
    +    for (final ValueVector v : this.allocationVectors) {
    +      AllocationHelper.allocateNew(v, inputBatch.getRecordCount());
    +    }
     
    -      container.buildSchema(BatchSchema.SelectionVectorMode.NONE);
    +    recordCount = unionall.unionRecords(0, inputBatch.getRecordCount(), 0);
    +    for (final ValueVector v : allocationVectors) {
    +      final ValueVector.Mutator m = v.getMutator();
    +      m.setValueCount(recordCount);
    +    }
    +
    +    if (callBack.getSchemaChangedAndReset()) {
           return IterOutcome.OK_NEW_SCHEMA;
    +    } else {
    +      return IterOutcome.OK;
         }
    +  }
    +
    +  private void createUnionAller(RecordBatch inputBatch) throws ClassTransformationException, IOException, SchemaChangeException {
    +    transfers.clear();
    +    allocationVectors.clear();;
     
         final ClassGenerator<UnionAller> cg = CodeGenerator.getRoot(UnionAller.TEMPLATE_DEFINITION, context.getFunctionRegistry(), context.getOptions());
         cg.getCodeGenerator().plainJavaCapable(true);
         // Uncomment out this line to debug the generated code.
    -//    cg.getCodeGenerator().saveCodeForDebugging(true);
    +    //    cg.getCodeGenerator().saveCodeForDebugging(true);
    +
         int index = 0;
    -    for(VectorWrapper<?> vw : current) {
    -       ValueVector vvIn = vw.getValueVector();
    -      // get the original input column names
    -      SchemaPath inputPath = SchemaPath.getSimplePath(vvIn.getField().getPath());
    -      // get the renamed column names
    -      SchemaPath outputPath = SchemaPath.getSimplePath(outputFields.get(index).getPath());
    +    for(VectorWrapper<?> vw : inputBatch) {
    +      ValueVector vvIn = vw.getValueVector();
    +      ValueVector vvOut = container.getValueVector(index).getValueVector();
     
           final ErrorCollector collector = new ErrorCollectorImpl();
           // According to input data names, Minortypes, Datamodes, choose to
           // transfer directly,
           // rename columns or
           // cast data types (Minortype or DataMode)
    -      if (hasSameTypeAndMode(outputFields.get(index), vw.getValueVector().getField())) {
    +      if (hasSameTypeAndMode(container.getSchema().getColumn(index), vvIn.getField())
    +          && vvIn.getField().getType().getMinorType() != TypeProtos.MinorType.MAP // Per DRILL-5521, existing bug for map transfer
    +          ) {
             // Transfer column
    -
    -        MajorType outputFieldType = outputFields.get(index).getType();
    -        MaterializedField outputField = MaterializedField.create(outputPath.getAsUnescapedPath(), outputFieldType);
    -
    -        /*
    -          todo: Fix if condition when DRILL-4824 is merged
    -          If condition should be changed to:
    -          `if (outputFields.get(index).getPath().equals(inputPath.getAsUnescapedPath())) {`
    -          DRILL-5419 has changed condition to correct one but this caused regression (DRILL-5521).
    -          Root cause is missing indication of child column in map types when it is null.
    -          DRILL-4824 is re-working json reader implementation, including map types and will fix this problem.
    -          Reverting condition to previous one to avoid regression till DRILL-4824 is merged.
    -          Unit test - TestJsonReader.testKvgenWithUnionAll().
    -         */
    -        if (outputFields.get(index).getPath().equals(inputPath)) {
    -          ValueVector vvOut = container.addOrGet(outputField);
    -          TransferPair tp = vvIn.makeTransferPair(vvOut);
    -          transfers.add(tp);
    +        TransferPair tp = vvIn.makeTransferPair(vvOut);
    +        transfers.add(tp);
             // Copy data in order to rename the column
    -        } else {
    -          final LogicalExpression expr = ExpressionTreeMaterializer.materialize(inputPath, current, collector, context.getFunctionRegistry() );
    -          if (collector.hasErrors()) {
    -            throw new SchemaChangeException(String.format("Failure while trying to materialize incoming schema.  Errors:\n %s.", collector.toErrorString()));
    -          }
    -
    -          ValueVector vv = container.addOrGet(outputField, callBack);
    -          allocationVectors.add(vv);
    -          TypedFieldId fid = container.getValueVectorId(SchemaPath.getSimplePath(outputField.getPath()));
    -          ValueVectorWriteExpression write = new ValueVectorWriteExpression(fid, expr, true);
    -          cg.addExpr(write);
    -        }
    -      // Cast is necessary
    +      } else if (vvIn.getField().getType().getMinorType() == TypeProtos.MinorType.NULL) {
    +        continue;
           } else {
    -        LogicalExpression expr = ExpressionTreeMaterializer.materialize(inputPath, current, collector, context.getFunctionRegistry());
    +        SchemaPath inputPath = SchemaPath.getSimplePath(vvIn.getField().getPath());
    +        MaterializedField inField = vvIn.getField();
    +        MaterializedField outputField = vvOut.getField();
    +
    +        LogicalExpression expr = ExpressionTreeMaterializer.materialize(inputPath, inputBatch, collector, context.getFunctionRegistry());
    +
             if (collector.hasErrors()) {
               throw new SchemaChangeException(String.format("Failure while trying to materialize incoming schema.  Errors:\n %s.", collector.toErrorString()));
             }
     
             // If the inputs' DataMode is required and the outputs' DataMode is not required
             // cast to the one with the least restriction
    -        if(vvIn.getField().getType().getMode() == DataMode.REQUIRED
    -            && outputFields.get(index).getType().getMode() != DataMode.REQUIRED) {
    -          expr = ExpressionTreeMaterializer.convertToNullableType(expr, vvIn.getField().getType().getMinorType(), context.getFunctionRegistry(), collector);
    +        if(inField.getType().getMode() == TypeProtos.DataMode.REQUIRED
    +            && outputField.getType().getMode() != TypeProtos.DataMode.REQUIRED) {
    +          expr = ExpressionTreeMaterializer.convertToNullableType(expr, inField.getType().getMinorType(), context.getFunctionRegistry(), collector);
               if (collector.hasErrors()) {
                 throw new SchemaChangeException(String.format("Failure while trying to materialize incoming schema.  Errors:\n %s.", collector.toErrorString()));
               }
             }
     
             // If two inputs' MinorTypes are different,
             // Insert a cast before the Union operation
    -        if(vvIn.getField().getType().getMinorType() != outputFields.get(index).getType().getMinorType()) {
    -          expr = ExpressionTreeMaterializer.addCastExpression(expr, outputFields.get(index).getType(), context.getFunctionRegistry(), collector);
    +        if(inField.getType().getMinorType() != outputField.getType().getMinorType()) {
    +          expr = ExpressionTreeMaterializer.addCastExpression(expr, outputField.getType(), context.getFunctionRegistry(), collector);
               if (collector.hasErrors()) {
                 throw new SchemaChangeException(String.format("Failure while trying to materialize incoming schema.  Errors:\n %s.", collector.toErrorString()));
               }
             }
     
    -        final MaterializedField outputField = MaterializedField.create(outputPath.getAsUnescapedPath(), expr.getMajorType());
    -        ValueVector vector = container.addOrGet(outputField, callBack);
    -        allocationVectors.add(vector);
             TypedFieldId fid = container.getValueVectorId(SchemaPath.getSimplePath(outputField.getPath()));
     
    -        boolean useSetSafe = !(vector instanceof FixedWidthVector);
    +        boolean useSetSafe = !(vvOut instanceof FixedWidthVector);
             ValueVectorWriteExpression write = new ValueVectorWriteExpression(fid, expr, useSetSafe);
             cg.addExpr(write);
    +
    +        allocationVectors.add(vvOut);
           }
           ++index;
         }
     
         unionall = context.getImplementationClass(cg.getCodeGenerator());
    -    unionall.setup(context, current, this, transfers);
    -
    -    if(!schemaAvailable) {
    -      container.buildSchema(BatchSchema.SelectionVectorMode.NONE);
    -      schemaAvailable = true;
    -    }
    -
    -    if(!doAlloc()) {
    -      return IterOutcome.OUT_OF_MEMORY;
    -    }
    -
    -    recordCount = unionall.unionRecords(0, current.getRecordCount(), 0);
    -    setValueCount(recordCount);
    -    return IterOutcome.OK;
    -  }
    -
    -  public static boolean hasSameTypeAndMode(MaterializedField leftField, MaterializedField rightField) {
    -    return (leftField.getType().getMinorType() == rightField.getType().getMinorType())
    -        && (leftField.getType().getMode() == rightField.getType().getMode());
    -  }
    -
    -  // This method is used by inner class to point the reference `current` to the correct record batch
    -  private void setCurrentRecordBatch(RecordBatch target) {
    -    this.current = target;
    -  }
    -
    -  // This method is used by inner class to clear the current record batch
    -  private void clearCurrentRecordBatch() {
    -    for(VectorWrapper<?> v: current) {
    -      v.clear();
    -    }
    +    unionall.setup(context, inputBatch, this, transfers);
       }
     
    -  public static class UnionAllInput {
    -    private UnionAllRecordBatch unionAllRecordBatch;
    -    private List<MaterializedField> outputFields;
    -    private OneSideInput leftSide;
    -    private OneSideInput rightSide;
    -    private IterOutcome upstream = IterOutcome.NOT_YET;
    -    private boolean leftIsFinish = false;
    -    private boolean rightIsFinish = false;
    -
    -    // These two schemas are obtained from the first record batches of the left and right inputs
    -    // They are used to check if the schema is changed between recordbatches
    -    private BatchSchema leftSchema;
    -    private BatchSchema rightSchema;
    -    private boolean bothEmpty = false;
    -
    -    public UnionAllInput(UnionAllRecordBatch unionAllRecordBatch, RecordBatch left, RecordBatch right) {
    -      this.unionAllRecordBatch = unionAllRecordBatch;
    -      leftSide = new OneSideInput(left);
    -      rightSide = new OneSideInput(right);
    -    }
    -
    -    private void setBothSideEmpty(boolean bothEmpty) {
    -      this.bothEmpty = bothEmpty;
    -    }
    -
    -    private boolean isBothSideEmpty() {
    -      return bothEmpty;
    -    }
    -
    -    public IterOutcome nextBatch() throws SchemaChangeException {
    -      if(upstream == RecordBatch.IterOutcome.NOT_YET) {
    -        IterOutcome iterLeft = leftSide.nextBatch();
    -        switch(iterLeft) {
    -          case OK_NEW_SCHEMA:
    -            /*
    -             * If the first few record batches are all empty,
    -             * there is no way to tell whether these empty batches are coming from empty files.
    -             * It is incorrect to infer output types when either side could be coming from empty.
    -             *
    -             * Thus, while-loop is necessary to skip those empty batches.
    -             */
    -            whileLoop:
    -            while(leftSide.getRecordBatch().getRecordCount() == 0) {
    -              iterLeft = leftSide.nextBatch();
    -
    -              switch(iterLeft) {
    -                case STOP:
    -                case OUT_OF_MEMORY:
    -                  return iterLeft;
    -
    -                case NONE:
    -                  // Special Case: The left side was an empty input.
    -                  leftIsFinish = true;
    -                  break whileLoop;
    -
    -                case NOT_YET:
    -                case OK_NEW_SCHEMA:
    -                case OK:
    -                  continue whileLoop;
    -
    -                default:
    -                  throw new IllegalStateException(
    -                      String.format("Unexpected state %s.", iterLeft));
    -              }
    -            }
    -
    -            break;
    -          case STOP:
    -          case OUT_OF_MEMORY:
    -            return iterLeft;
    -
    -          default:
    -            throw new IllegalStateException(
    -                String.format("Unexpected state %s.", iterLeft));
    -        }
    -
    -        IterOutcome iterRight = rightSide.nextBatch();
    -        switch(iterRight) {
    -          case OK_NEW_SCHEMA:
    -            // Unless there is no record batch on the left side of the inputs,
    -            // always start processing from the left side.
    -            if(leftIsFinish) {
    -              unionAllRecordBatch.setCurrentRecordBatch(rightSide.getRecordBatch());
    -            } else {
    -              unionAllRecordBatch.setCurrentRecordBatch(leftSide.getRecordBatch());
    -            }
    -            // If the record count of the first batch from right input is zero,
    -            // there are two possibilities:
    -            // 1. The right side is an empty input (e.g., file).
    -            // 2. There will be more records carried by later batches.
    -
    -            /*
    -             * If the first few record batches are all empty,
    -             * there is no way to tell whether these empty batches are coming from empty files.
    -             * It is incorrect to infer output types when either side could be coming from empty.
    -             *
    -             * Thus, while-loop is necessary to skip those empty batches.
    -             */
    -            whileLoop:
    -            while(rightSide.getRecordBatch().getRecordCount() == 0) {
    -              iterRight = rightSide.nextBatch();
    -              switch(iterRight) {
    -                case STOP:
    -                case OUT_OF_MEMORY:
    -                  return iterRight;
    -
    -                case NONE:
    -                  // Special Case: The right side was an empty input.
    -                  rightIsFinish = true;
    -                  break whileLoop;
    -
    -                case NOT_YET:
    -                case OK_NEW_SCHEMA:
    -                case OK:
    -                  continue whileLoop;
    -
    -                default:
    -                  throw new IllegalStateException(
    -                      String.format("Unexpected state %s.", iterRight));
    -              }
    -            }
    -
    -            if(leftIsFinish && rightIsFinish) {
    -              setBothSideEmpty(true);
    -            }
    -
    -            inferOutputFields();
    -            break;
    -
    -          case STOP:
    -          case OUT_OF_MEMORY:
    -            return iterRight;
    -
    -          default:
    -            throw new IllegalStateException(
    -                String.format("Unexpected state %s.", iterRight));
    -        }
    -
    -
    -
    -        upstream = IterOutcome.OK_NEW_SCHEMA;
    -        return upstream;
    -      } else {
    -        if(isBothSideEmpty()) {
    -          return IterOutcome.NONE;
    -        }
    -
    -        unionAllRecordBatch.clearCurrentRecordBatch();
    -
    -        if(leftIsFinish && rightIsFinish) {
    -          upstream = IterOutcome.NONE;
    -          return upstream;
    -        } else if(leftIsFinish) {
    -          IterOutcome iterOutcome = rightSide.nextBatch();
    -
    -          switch(iterOutcome) {
    -            case NONE:
    -              rightIsFinish = true;
    -              // fall through
    -            case STOP:
    -            case OUT_OF_MEMORY:
    -              upstream = iterOutcome;
    -              return upstream;
    -
    -            case OK_NEW_SCHEMA:
    -              if(!rightSide.getRecordBatch().getSchema().equals(rightSchema)) {
    -                throw new SchemaChangeException("Schema change detected in the right input of Union-All. This is not currently supported");
    -              }
    -              iterOutcome = IterOutcome.OK;
    -              // fall through
    -            case OK:
    -              unionAllRecordBatch.setCurrentRecordBatch(rightSide.getRecordBatch());
    -              upstream = iterOutcome;
    -              return upstream;
    -
    -            default:
    -              throw new IllegalStateException(String.format("Unknown state %s.", upstream));
    -          }
    -        } else if(rightIsFinish) {
    -          IterOutcome iterOutcome = leftSide.nextBatch();
    -          switch(iterOutcome) {
    -            case STOP:
    -            case OUT_OF_MEMORY:
    -            case NONE:
    -              upstream = iterOutcome;
    -              return upstream;
    -
    -            case OK:
    -              unionAllRecordBatch.setCurrentRecordBatch(leftSide.getRecordBatch());
    -              upstream = iterOutcome;
    -              return upstream;
    -
    -            default:
    -              throw new IllegalStateException(String.format("Unknown state %s.", iterOutcome));
    -          }
    -        } else {
    -          IterOutcome iterOutcome = leftSide.nextBatch();
    -
    -          switch(iterOutcome) {
    -            case STOP:
    -            case OUT_OF_MEMORY:
    -              upstream = iterOutcome;
    -              return upstream;
    -
    -            case OK_NEW_SCHEMA:
    -              if(!leftSide.getRecordBatch().getSchema().equals(leftSchema)) {
    -                throw new SchemaChangeException("Schema change detected in the left input of Union-All. This is not currently supported");
    -              }
    -
    -              iterOutcome = IterOutcome.OK;
    -              // fall through
    -            case OK:
    -              unionAllRecordBatch.setCurrentRecordBatch(leftSide.getRecordBatch());
    -              upstream = iterOutcome;
    -              return upstream;
    -
    -            case NONE:
    -              unionAllRecordBatch.setCurrentRecordBatch(rightSide.getRecordBatch());
    -              upstream = IterOutcome.OK;
    -              leftIsFinish = true;
    -              return upstream;
    -
    -            default:
    -              throw new IllegalStateException(String.format("Unknown state %s.", upstream));
    -          }
    -        }
    -      }
    -    }
     
    -    /**
    -     *
    -     * Summarize the inference in the four different situations:
    -     * First of all, the field names are always determined by the left side
    -     * (Even when the left side is from an empty file, we have the column names.)
    -     *
    -     * Cases:
    -     * 1. Left: non-empty; Right: non-empty
    -     *      types determined by both sides with implicit casting involved
    -     * 2. Left: empty; Right: non-empty
    -     *      type from the right
    -     * 3. Left: non-empty; Right: empty
    -     *      types from the left
    -     * 4. Left: empty; Right: empty
    -     *      types are nullable integer
    -     */
    -    private void inferOutputFields() {
    -      if(!leftIsFinish && !rightIsFinish) {
    -        // Both sides are non-empty
    -        inferOutputFieldsBothSide();
    -      } else if(!rightIsFinish) {
    -        // Left side is non-empty
    -        // While use left side's column names as output column names,
    -        // use right side's column types as output column types.
    -        inferOutputFieldsFromSingleSide(
    -            leftSide.getRecordBatch().getSchema(),
    -            rightSide.getRecordBatch().getSchema());
    +  // The output table's column names always follow the left table,
    +  // where the output type is chosen based on DRILL's implicit casting rules
    +  private void inferOutputFieldsBothSide(final BatchSchema leftSchema, final BatchSchema rightSchema) {
    +//    outputFields = Lists.newArrayList();
    +    final Iterator<MaterializedField> leftIter = leftSchema.iterator();
    +    final Iterator<MaterializedField> rightIter = rightSchema.iterator();
    +
    +    int index = 1;
    +    while (leftIter.hasNext() && rightIter.hasNext()) {
    +      MaterializedField leftField  = leftIter.next();
    +      MaterializedField rightField = rightIter.next();
    +
    +      if (hasSameTypeAndMode(leftField, rightField)) {
    +        TypeProtos.MajorType.Builder builder = TypeProtos.MajorType.newBuilder().setMinorType(leftField.getType().getMinorType()).setMode(leftField.getDataMode());
    +        builder = Types.calculateTypePrecisionAndScale(leftField.getType(), rightField.getType(), builder);
    +        container.addOrGet(MaterializedField.create(leftField.getPath(), builder.build()), callBack);
    +      } else if (Types.isUntypedNull(rightField.getType())) {
    +        container.addOrGet(leftField, callBack);
    +      } else if (Types.isUntypedNull(leftField.getType())) {
    +        container.addOrGet(MaterializedField.create(leftField.getPath(), rightField.getType()), callBack);
           } else {
    -        // Either right side is empty or both are empty
    -        // Using left side's schema is sufficient
    -        inferOutputFieldsFromSingleSide(
    -            leftSide.getRecordBatch().getSchema(),
    -            leftSide.getRecordBatch().getSchema());
    -      }
    -    }
    -
    -    // The output table's column names always follow the left table,
    -    // where the output type is chosen based on DRILL's implicit casting rules
    -    private void inferOutputFieldsBothSide() {
    -      outputFields = Lists.newArrayList();
    -      leftSchema = leftSide.getRecordBatch().getSchema();
    -      rightSchema = rightSide.getRecordBatch().getSchema();
    -      Iterator<MaterializedField> leftIter = leftSchema.iterator();
    -      Iterator<MaterializedField> rightIter = rightSchema.iterator();
    -
    -      int index = 1;
    -      while (leftIter.hasNext() && rightIter.hasNext()) {
    -        MaterializedField leftField  = leftIter.next();
    -        MaterializedField rightField = rightIter.next();
    -
    -        if (hasSameTypeAndMode(leftField, rightField)) {
    -          MajorType.Builder builder = MajorType.newBuilder().setMinorType(leftField.getType().getMinorType()).setMode(leftField.getDataMode());
    +        // If the output type is not the same,
    +        // cast the column of one of the table to a data type which is the Least Restrictive
    +        TypeProtos.MajorType.Builder builder = TypeProtos.MajorType.newBuilder();
    +        if (leftField.getType().getMinorType() == rightField.getType().getMinorType()) {
    +          builder.setMinorType(leftField.getType().getMinorType());
               builder = Types.calculateTypePrecisionAndScale(leftField.getType(), rightField.getType(), builder);
    -          outputFields.add(MaterializedField.create(leftField.getPath(), builder.build()));
             } else {
    -          // If the output type is not the same,
    -          // cast the column of one of the table to a data type which is the Least Restrictive
    -          MajorType.Builder builder = MajorType.newBuilder();
    -          if (leftField.getType().getMinorType() == rightField.getType().getMinorType()) {
    -            builder.setMinorType(leftField.getType().getMinorType());
    -            builder = Types.calculateTypePrecisionAndScale(leftField.getType(), rightField.getType(), builder);
    -          } else {
    -            List<MinorType> types = Lists.newLinkedList();
    -            types.add(leftField.getType().getMinorType());
    -            types.add(rightField.getType().getMinorType());
    -            MinorType outputMinorType = TypeCastRules.getLeastRestrictiveType(types);
    -            if (outputMinorType == null) {
    -              throw new DrillRuntimeException("Type mismatch between " + leftField.getType().getMinorType().toString() +
    -                  " on the left side and " + rightField.getType().getMinorType().toString() +
    -                  " on the right side in column " + index + " of UNION ALL");
    -            }
    -            builder.setMinorType(outputMinorType);
    +          List<TypeProtos.MinorType> types = Lists.newLinkedList();
    +          types.add(leftField.getType().getMinorType());
    +          types.add(rightField.getType().getMinorType());
    +          TypeProtos.MinorType outputMinorType = TypeCastRules.getLeastRestrictiveType(types);
    +          if (outputMinorType == null) {
    +            throw new DrillRuntimeException("Type mismatch between " + leftField.getType().getMinorType().toString() +
    +                " on the left side and " + rightField.getType().getMinorType().toString() +
    +                " on the right side in column " + index + " of UNION ALL");
               }
    -
    -          // The output data mode should be as flexible as the more flexible one from the two input tables
    -          List<DataMode> dataModes = Lists.newLinkedList();
    -          dataModes.add(leftField.getType().getMode());
    -          dataModes.add(rightField.getType().getMode());
    -          builder.setMode(TypeCastRules.getLeastRestrictiveDataMode(dataModes));
    -
    -          outputFields.add(MaterializedField.create(leftField.getPath(), builder.build()));
    +          builder.setMinorType(outputMinorType);
             }
    -        ++index;
    -      }
    -
    -      assert !leftIter.hasNext() && ! rightIter.hasNext() : "Mis-match of column count should have been detected when validating sqlNode at planning";
    -    }
     
    -    private void inferOutputFieldsFromSingleSide(final BatchSchema schemaForNames, final BatchSchema schemaForTypes) {
    -      outputFields = Lists.newArrayList();
    +        // The output data mode should be as flexible as the more flexible one from the two input tables
    +        List<TypeProtos.DataMode> dataModes = Lists.newLinkedList();
    +        dataModes.add(leftField.getType().getMode());
    +        dataModes.add(rightField.getType().getMode());
    +        builder.setMode(TypeCastRules.getLeastRestrictiveDataMode(dataModes));
     
    -      final List<String> outputColumnNames = Lists.newArrayList();
    -      final Iterator<MaterializedField> iterForNames = schemaForNames.iterator();
    -      while(iterForNames.hasNext()) {
    -        outputColumnNames.add(iterForNames.next().getPath());
    -      }
    -
    -      final Iterator<MaterializedField> iterForTypes = schemaForTypes.iterator();
    -      for(int i = 0; iterForTypes.hasNext(); ++i) {
    -        MaterializedField field = iterForTypes.next();
    -        outputFields.add(MaterializedField.create(outputColumnNames.get(i), field.getType()));
    +        container.addOrGet(MaterializedField.create(leftField.getPath(), builder.build()), callBack);
           }
    +      ++index;
         }
     
    -    public List<MaterializedField> getOutputFields() {
    -      if(outputFields == null) {
    -        throw new NullPointerException("Output fields have not been inferred");
    -      }
    -
    -      return outputFields;
    -    }
    +    assert !leftIter.hasNext() && ! rightIter.hasNext() : "Mis-match of column count should have been detected when validating sqlNode at planning";
    +  }
     
    -    public void killIncoming(boolean sendUpstream) {
    -      leftSide.getRecordBatch().kill(sendUpstream);
    -      rightSide.getRecordBatch().kill(sendUpstream);
    +  private void inferOutputFieldsOneSide(final BatchSchema schema) {
    +    for (MaterializedField field : schema) {
    +      container.addOrGet(field, callBack);
         }
    +  }
     
    -    public RecordBatch getLeftRecordBatch() {
    -      return leftSide.getRecordBatch();
    -    }
    +  private static boolean hasSameTypeAndMode(MaterializedField leftField, MaterializedField rightField) {
    --- End diff --
    
    Seems like a generally useful query. Perhaps move it to a method on `MaterializedField`?


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] drill pull request #906: DRILL-5546: Handle schema change exception failure ...

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/906#discussion_r134628593
  
    --- Diff: exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/project/ProjectRecordBatch.java ---
    @@ -768,4 +765,73 @@ else if (exprHasPrefix && refHasPrefix) {
           }
         }
       }
    +
    +  /**
    +   * handle FAST NONE specially when Project for query output. This happens when input returns a
    +   * FAST NONE directly ( input does not return any batch with schema/data).
    +   *
    +   * Project operator has to return a batch with schema derived using the following 3 rules:
    +   *  Case 1:  *  ==>  expand into an empty list of columns.
    +   *  Case 2:  regular column reference ==> treat as nullable-int column
    +   *  Case 3:  expressions => Call ExpressionTreeMaterialization over an empty vector contain.
    +   *           Once the expression is materialized without error, use the output type of materialized
    +   *           expression.
    +   * The batch is constructed with the above rules, and recordCount = 0.
    +   * Returned with OK_NEW_SCHEMA to down-stream operator.
    +   */
    +  @Override
    +  protected IterOutcome handleFastNone() {
    +    if (! popConfig.isOutputProj()) {
    +      return super.handleFastNone();
    +    }
    +
    +    allocationVectors = new ArrayList<>();
    +    final List<NamedExpression> exprs = getExpressionList();
    +    final ErrorCollector collector = new ErrorCollectorImpl();
    +    VectorContainer fakedIncomingVC = new VectorContainer();
    +
    +    for (NamedExpression namedExpression : exprs) {
    +      if (namedExpression.getExpr() instanceof SchemaPath) {
    +        final NameSegment expr = ((SchemaPath) namedExpression.getExpr()).getRootSegment();
    +        if (expr.getPath().contains(StarColumnHelper.STAR_COLUMN)) {
    +          continue; // * would expand into an empty list.
    +        } else {
    +          final TypeProtos.MajorType majorType = TypeProtos.MajorType.newBuilder()
    +              .setMinorType(MinorType.INT)
    +              .setMode(TypeProtos.DataMode.OPTIONAL)
    +              .build();
    +
    +          MaterializedField outputField = MaterializedField.create(namedExpression.getRef().getRootSegment().getPath(), majorType);
    +          final ValueVector vv = container.addOrGet(outputField, callBack);
    --- End diff --
    
    Does this code handle map columns? `map1.nestedMap.col`?
    
    Does this code handle references to arrays, such as `columns[2]` for a CSV file?


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] drill pull request #906: DRILL-5546: Handle schema change exception failure ...

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

    https://github.com/apache/drill/pull/906#discussion_r135974302
  
    --- Diff: exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/project/ProjectRecordBatch.java ---
    @@ -768,4 +765,73 @@ else if (exprHasPrefix && refHasPrefix) {
           }
         }
       }
    +
    +  /**
    +   * handle FAST NONE specially when Project for query output. This happens when input returns a
    +   * FAST NONE directly ( input does not return any batch with schema/data).
    +   *
    +   * Project operator has to return a batch with schema derived using the following 3 rules:
    +   *  Case 1:  *  ==>  expand into an empty list of columns.
    +   *  Case 2:  regular column reference ==> treat as nullable-int column
    +   *  Case 3:  expressions => Call ExpressionTreeMaterialization over an empty vector contain.
    +   *           Once the expression is materialized without error, use the output type of materialized
    +   *           expression.
    +   * The batch is constructed with the above rules, and recordCount = 0.
    +   * Returned with OK_NEW_SCHEMA to down-stream operator.
    +   */
    +  @Override
    +  protected IterOutcome handleFastNone() {
    +    if (! popConfig.isOutputProj()) {
    +      return super.handleFastNone();
    +    }
    +
    +    allocationVectors = new ArrayList<>();
    +    final List<NamedExpression> exprs = getExpressionList();
    +    final ErrorCollector collector = new ErrorCollectorImpl();
    --- End diff --
    
    This code has been removed in revised patch. 


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] drill pull request #906: DRILL-5546: Handle schema change exception failure ...

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

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


---

[GitHub] drill pull request #906: DRILL-5546: Handle schema change exception failure ...

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

    https://github.com/apache/drill/pull/906#discussion_r135373441
  
    --- Diff: exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/ScanBatch.java ---
    @@ -152,97 +157,75 @@ public void kill(boolean sendUpstream) {
         }
       }
     
    -  private void releaseAssets() {
    -    container.zeroVectors();
    -  }
    -
    -  private void clearFieldVectorMap() {
    -    for (final ValueVector v : mutator.fieldVectorMap().values()) {
    -      v.clear();
    -    }
    -  }
    -
       @Override
       public IterOutcome next() {
         if (done) {
           return IterOutcome.NONE;
         }
         oContext.getStats().startProcessing();
         try {
    -      try {
    -        injector.injectChecked(context.getExecutionControls(), "next-allocate", OutOfMemoryException.class);
    -
    -        currentReader.allocate(mutator.fieldVectorMap());
    -      } catch (OutOfMemoryException e) {
    -        clearFieldVectorMap();
    -        throw UserException.memoryError(e).build(logger);
    -      }
    -      while ((recordCount = currentReader.next()) == 0) {
    +      while (true) {
             try {
    -          if (!readers.hasNext()) {
    -            // We're on the last reader, and it has no (more) rows.
    -            currentReader.close();
    -            releaseAssets();
    -            done = true;  // have any future call to next() return NONE
    -
    -            if (mutator.isNewSchema()) {
    -              // This last reader has a new schema (e.g., we have a zero-row
    -              // file or other source).  (Note that some sources have a non-
    -              // null/non-trivial schema even when there are no rows.)
    +          injector.injectChecked(context.getExecutionControls(), "next-allocate", OutOfMemoryException.class);
    +          currentReader.allocate(mutator.fieldVectorMap());
    +        } catch (OutOfMemoryException e) {
    +          clearFieldVectorMap();
    +          throw UserException.memoryError(e).build(logger);
    +        }
     
    -              container.buildSchema(SelectionVectorMode.NONE);
    -              schema = container.getSchema();
    +        recordCount = currentReader.next();
    +        Preconditions.checkArgument(recordCount >= 0,
    +            "recordCount from RecordReader.next() should not be negative");
     
    -              return IterOutcome.OK_NEW_SCHEMA;
    -            }
    -            return IterOutcome.NONE;
    -          }
    -          // At this point, the reader that hit its end is not the last reader.
    +        boolean isNewRegularSchema = mutator.isNewSchema();
    +        // We should skip the reader, when recordCount = 0 && ! isNewRegularSchema.
    +        // Add/set implicit column vectors, only when reader gets > 0 row, or
    +        // when reader gets 0 row but with a schema with new field added
    +        if (recordCount > 0 || isNewRegularSchema) {
    +          addImplicitVectors();
    +          populateImplicitVectors();
    +        }
     
    -          // If all the files we have read so far are just empty, the schema is not useful
    -          if (! hasReadNonEmptyFile) {
    -            container.clear();
    -            clearFieldVectorMap();
    -            mutator.clear();
    -          }
    +        boolean isNewImplicitSchema = mutator.isNewSchema();
    --- End diff --
    
    You are correct that implicit columns such as _filename_ etc are fixed, and only _dir0_, _dir1_ columns can change. 
    
    Previously,  I thought in the same way, in terms of how Drill handling _a/b/c.csv_,  _a/b/d/e.csv_. That is,  dir2 only exists for the second file, but not for the first file, and hence could lead to a schema change. Turns out that there is code to align the dir list [1].  As a result, across the record readers, the schema for implicit vectors should remain identical. In other words, we do not have to consider implicit vectors in terms of schema change for scan batch.
    
    In the revised patch, I dropped the isNewImplcitSchema logic, since the implicit schema is constant.  Also, I add the checking in ScanBatch's constructor, to make sure either the implicit column list is empty, or they should reman constant for all record readers, and throw setup exception if fails such check. 
    
    1. https://github.com/apache/drill/blob/master/exec/java-exec/src/main/java/org/apache/drill/exec/store/ColumnExplorer.java#L145-L147 


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] drill pull request #906: DRILL-5546: Handle schema change exception failure ...

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/906#discussion_r135579971
  
    --- Diff: exec/vector/src/main/java/org/apache/drill/exec/vector/UntypedNullVector.java ---
    @@ -0,0 +1,267 @@
    +/*
    + * 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;
    +
    +
    +import com.google.common.base.Preconditions;
    +import io.netty.buffer.DrillBuf;
    +import org.apache.drill.exec.memory.BufferAllocator;
    +import org.apache.drill.exec.proto.UserBitShared.SerializedField;
    +import org.apache.drill.exec.record.MaterializedField;
    +import org.apache.drill.exec.record.TransferPair;
    +import org.apache.drill.exec.vector.complex.reader.FieldReader;
    +
    +/** UntypedNullVector is to represent a value vector with {@link org.apache.drill.common.types.MinorType.NULL}
    + *  All values in the vector represent two semantic implications: 1) the value is unknown, 2) the type is unknown.
    + *  Because of this, we only have to keep track of the number of values in value vector,
    + *  and there is no allocated buffer to back up this value vector. Therefore, the majority of
    + *  methods in this class is either no-op, or throws {@link UnsupportedOperationException}.
    + *
    + */
    +public final class UntypedNullVector extends BaseDataValueVector implements FixedWidthVector {
    +  private static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(UntypedNullVector.class);
    +
    +  /**
    +   * Width of each fixed-width value.
    +   */
    +  public static final int VALUE_WIDTH = 0;
    +
    +  private final Accessor accessor = new Accessor();
    +  private final Mutator mutator = new Mutator();
    +  private int valueCount ;
    +
    +  public UntypedNullVector(MaterializedField field, BufferAllocator allocator) {
    +    super(field, allocator);
    +    valueCount = 0;
    +  }
    +
    +  @Override
    +  public FieldReader getReader() { throw new UnsupportedOperationException(); }
    +
    +  @Override
    +  public int getBufferSizeFor(final int valueCount) {
    +    return 0;
    +  }
    +
    +  @Override
    +  public int getValueCapacity(){
    +    return Character.MAX_VALUE;
    +  }
    +
    +  @Override
    +  public Accessor getAccessor() { return accessor; }
    +
    +  @Override
    +  public Mutator getMutator() { return mutator; }
    +
    +  @Override
    +  public void setInitialCapacity(final int valueCount) {
    +  }
    +
    +  @Override
    +  public void allocateNew() {
    +  }
    +
    +  @Override
    +  public boolean allocateNewSafe() {
    +    return true;
    +  }
    +
    +  @Override
    +  public void allocateNew(final int valueCount) {
    +  }
    +
    +  @Override
    +  public void reset() {
    +  }
    +
    +  /**
    +   * {@inheritDoc}
    +   */
    +  @Override
    +  public void zeroVector() {
    +  }
    +
    +  @Override
    +  public void load(SerializedField metadata, DrillBuf buffer) {
    +    Preconditions.checkArgument(this.field.getPath().equals(metadata.getNamePart().getName()),
    +        "The field %s doesn't match the provided metadata %s.", this.field, metadata);
    +    final int actualLength = metadata.getBufferLength();
    +    final int valueCount = metadata.getValueCount();
    +    final int expectedLength = valueCount * VALUE_WIDTH;
    +    assert actualLength == expectedLength : String.format("Expected to load %d bytes but actually loaded %d bytes", expectedLength, actualLength);
    +
    +    this.valueCount = valueCount;
    +  }
    +
    +  @Override
    +  public TransferPair getTransferPair(BufferAllocator allocator){
    +    return new TransferImpl(getField(), allocator);
    +  }
    +
    +  @Override
    +  public TransferPair getTransferPair(String ref, BufferAllocator allocator){
    +    return new TransferImpl(getField().withPath(ref), allocator);
    +  }
    +
    +  @Override
    +  public TransferPair makeTransferPair(ValueVector to) {
    +    return new TransferImpl((UntypedNullVector) to);
    +  }
    +
    +  public void transferTo(UntypedNullVector target){
    +  }
    +
    +  public void splitAndTransferTo(int startIndex, int length, UntypedNullVector target) {
    +  }
    +
    +  @Override
    +  public int getPayloadByteCount() {
    +    return 0;
    +  }
    +
    +  private class TransferImpl implements TransferPair{
    +    private UntypedNullVector to;
    +
    +    public TransferImpl(MaterializedField field, BufferAllocator allocator){
    +      to = new UntypedNullVector(field, allocator);
    +    }
    +
    +    public TransferImpl(UntypedNullVector to) {
    +      this.to = to;
    +    }
    +
    +    @Override
    +    public UntypedNullVector getTo(){
    +      return to;
    +    }
    +
    +    @Override
    +    public void transfer(){
    +      transferTo(to);
    +    }
    +
    +    @Override
    +    public void splitAndTransfer(int startIndex, int length) {
    +      checkBounds(startIndex);
    +      checkBounds(startIndex + length - 1);
    +      splitAndTransferTo(startIndex, length, to);
    +    }
    +
    +    @Override
    +    public void copyValueSafe(int fromIndex, int toIndex) {
    +      checkBounds(fromIndex);
    +      to.copyFromSafe(fromIndex, toIndex, UntypedNullVector.this);
    +    }
    +  }
    +
    +  public void copyFrom(int fromIndex, int thisIndex, UntypedNullVector from){
    +  }
    +
    +  public void copyFromSafe(int fromIndex, int thisIndex, UntypedNullVector from){
    +  }
    +
    +  private void checkBounds(int index) {
    +    if (index < 0 || index >= valueCount) {
    +      throw new IndexOutOfBoundsException(String.format(
    +          "index: %d, expected: range(0, %d-1))", index, valueCount));
    +    }
    +  }
    +  @Override
    +  public void copyEntry(int toIndex, ValueVector from, int fromIndex) {
    +    ((UntypedNullVector) from).data.getBytes(fromIndex * 4, data, toIndex * 4, 4);
    +  }
    +
    +  public final class Accessor extends BaseAccessor {
    +    @Override
    +    public int getValueCount() {
    +      return valueCount;
    +    }
    +
    +    @Override
    +    public boolean isNull(int index){
    +      checkBounds(index);
    +      return true;
    +    }
    +
    +    public int isSet(int index) {
    +      checkBounds(index);
    +      return 0;
    +    }
    +
    +    @Override
    +    public Object getObject(int index) {
    +      throw new UnsupportedOperationException();
    --- End diff --
    
    `return null`? Since the value is always null, shouldn't the Java Object value also be `null`?


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] drill pull request #906: DRILL-5546: Handle schema change exception failure ...

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

    https://github.com/apache/drill/pull/906#discussion_r136185479
  
    --- Diff: exec/java-exec/src/main/java/org/apache/drill/exec/record/AbstractSingleRecordBatch.java ---
    @@ -125,4 +131,19 @@ public BatchSchema getSchema() {
     
       protected abstract boolean setupNewSchema() throws SchemaChangeException;
       protected abstract IterOutcome doWork();
    +
    +  /**
    +   * Default behavior to handle fast NONE (incoming's first next() return NONE, in stead of OK_NEW_SCHEMA):
    +   * FAST NONE could happen when the underneath Scan operators do not produce any batch with schema.
    +   *
    +   * This behavior could be override in each individual operator, if the operator's semantics is to
    +   * inject a batch with schema.
    +   *
    +   * @return IterOutcome.NONE.
    +   */
    +  protected IterOutcome handleFastNone() {
    +    container.buildSchema(SelectionVectorMode.NONE);
    +    return IterOutcome.NONE;
    +  };
    --- End diff --
    
    done. 


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] drill pull request #906: DRILL-5546: Handle schema change exception failure ...

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/906#discussion_r135578594
  
    --- Diff: exec/vector/src/main/java/org/apache/drill/exec/vector/UntypedNullHolder.java ---
    @@ -0,0 +1,50 @@
    +/*
    + * 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;
    +
    +import org.apache.drill.common.types.TypeProtos;
    +import org.apache.drill.common.types.Types;
    +import org.apache.drill.exec.expr.holders.ValueHolder;
    +
    +public class UntypedNullHolder implements ValueHolder {
    +  public static final TypeProtos.MajorType TYPE = Types.optional(TypeProtos.MinorType.NULL);
    +
    +  public TypeProtos.MajorType getType() {return TYPE;}
    +
    +  public static final int WIDTH = 0;
    +
    +  public int isSet = 1;
    +
    +  @Deprecated
    +  public int hashCode(){
    +    throw new UnsupportedOperationException();
    +  }
    +
    +  /*
    +   * Reason for deprecation is that ValueHolders are potential scalar replacements
    +   * and hence we don't want any methods to be invoked on them.
    +   */
    +  @Deprecated
    +  public String toString(){
    +    throw new UnsupportedOperationException();
    --- End diff --
    
    Why unsupported? According to Java, all classes should implement toString. Else, if we are in the debugger, and try to look at an instance of this class, the debugger will encounter an exception.
    
    Also, it is not a good idea to deprecate a standard Java method.
    
    
    Either leave the default implementation, or simply return `getClass().getSimpleName()`.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] drill pull request #906: DRILL-5546: Handle schema change exception failure ...

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

    https://github.com/apache/drill/pull/906#discussion_r135977556
  
    --- Diff: exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/union/UnionAllRecordBatch.java ---
    @@ -39,88 +35,107 @@
     import org.apache.drill.exec.expr.ValueVectorWriteExpression;
     import org.apache.drill.exec.ops.FragmentContext;
     import org.apache.drill.exec.physical.config.UnionAll;
    -import org.apache.drill.exec.record.AbstractRecordBatch;
    +import org.apache.drill.exec.record.AbstractBinaryRecordBatch;
     import org.apache.drill.exec.record.BatchSchema;
     import org.apache.drill.exec.record.MaterializedField;
     import org.apache.drill.exec.record.RecordBatch;
     import org.apache.drill.exec.record.TransferPair;
     import org.apache.drill.exec.record.TypedFieldId;
     import org.apache.drill.exec.record.VectorWrapper;
    -import org.apache.drill.exec.record.WritableBatch;
    -import org.apache.drill.exec.record.selection.SelectionVector2;
    -import org.apache.drill.exec.record.selection.SelectionVector4;
     import org.apache.drill.exec.resolver.TypeCastRules;
     import org.apache.drill.exec.vector.AllocationHelper;
     import org.apache.drill.exec.vector.FixedWidthVector;
     import org.apache.drill.exec.vector.SchemaChangeCallBack;
     import org.apache.drill.exec.vector.ValueVector;
     
    -import com.google.common.collect.Lists;
    +import java.io.IOException;
    +import java.util.Iterator;
    +import java.util.List;
    +import java.util.NoSuchElementException;
    +import java.util.Stack;
     
    -public class UnionAllRecordBatch extends AbstractRecordBatch<UnionAll> {
    +public class UnionAllRecordBatch extends AbstractBinaryRecordBatch<UnionAll> {
       private static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(UnionAllRecordBatch.class);
     
    -  private List<MaterializedField> outputFields;
    +  private SchemaChangeCallBack callBack = new SchemaChangeCallBack();
       private UnionAller unionall;
    -  private UnionAllInput unionAllInput;
    -  private RecordBatch current;
    -
       private final List<TransferPair> transfers = Lists.newArrayList();
    -  private List<ValueVector> allocationVectors;
    -  protected SchemaChangeCallBack callBack = new SchemaChangeCallBack();
    +  private List<ValueVector> allocationVectors = Lists.newArrayList();
       private int recordCount = 0;
    -  private boolean schemaAvailable = false;
    +  private UnionInputIterator unionInputIterator;
     
       public UnionAllRecordBatch(UnionAll config, List<RecordBatch> children, FragmentContext context) throws OutOfMemoryException {
    -    super(config, context, false);
    -    assert (children.size() == 2) : "The number of the operands of Union must be 2";
    -    unionAllInput = new UnionAllInput(this, children.get(0), children.get(1));
    -  }
    -
    -  @Override
    -  public int getRecordCount() {
    -    return recordCount;
    +    super(config, context, true, children.get(0), children.get(1));
       }
     
       @Override
       protected void killIncoming(boolean sendUpstream) {
    -    unionAllInput.getLeftRecordBatch().kill(sendUpstream);
    -    unionAllInput.getRightRecordBatch().kill(sendUpstream);
    +    left.kill(sendUpstream);
    +    right.kill(sendUpstream);
       }
     
    -  @Override
    -  public SelectionVector2 getSelectionVector2() {
    -    throw new UnsupportedOperationException("UnionAllRecordBatch does not support selection vector");
    -  }
    +  protected void buildSchema() throws SchemaChangeException {
    +    if (! prefetchFirstBatchFromBothSides()) {
    +      return;
    +    }
     
    -  @Override
    -  public SelectionVector4 getSelectionVector4() {
    -    throw new UnsupportedOperationException("UnionAllRecordBatch does not support selection vector");
    +    unionInputIterator = new UnionInputIterator(leftUpstream, left, rightUpstream, right);
    +
    +    if (leftUpstream == IterOutcome.NONE && rightUpstream == IterOutcome.OK_NEW_SCHEMA) {
    +      inferOutputFieldsOneSide(right.getSchema());
    +    } else if (rightUpstream == IterOutcome.NONE && leftUpstream == IterOutcome.OK_NEW_SCHEMA) {
    +      inferOutputFieldsOneSide((left.getSchema()));
    +    } else if (leftUpstream == IterOutcome.OK_NEW_SCHEMA && rightUpstream == IterOutcome.OK_NEW_SCHEMA) {
    +      inferOutputFieldsBothSide(left.getSchema(), right.getSchema());
    +    }
    +
    +    container.buildSchema(BatchSchema.SelectionVectorMode.NONE);
    +
    +    for (VectorWrapper vv: container) {
    +      vv.getValueVector().allocateNew();
    +      vv.getValueVector().getMutator().setValueCount(0);
    +    }
    --- End diff --
    
    Why would you think the above the four lines of code are **_"copied"_** from somewhere else, while the two lines of code are NOT "copied"? You could say the two lines are shorter, but I could not see why the four lines are copied, while the two lines of code are NOT? 
    
    With that said, thanks for pointing to this new util method. I was not aware of that until now. In the revised patch, I used the util method to reduce 4 lines code into 2 lines code.



---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] drill pull request #906: DRILL-5546: Handle schema change exception failure ...

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

    https://github.com/apache/drill/pull/906#discussion_r136152074
  
    --- Diff: exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/union/UnionAllRecordBatch.java ---
    @@ -130,562 +145,248 @@ public IterOutcome innerNext() {
       }
     
       @Override
    -  public WritableBatch getWritableBatch() {
    -    return WritableBatch.get(this);
    +  public int getRecordCount() {
    +    return recordCount;
       }
     
    -  private void setValueCount(int count) {
    -    for (ValueVector v : allocationVectors) {
    -      ValueVector.Mutator m = v.getMutator();
    -      m.setValueCount(count);
    -    }
    -  }
     
    -  private boolean doAlloc() {
    -    for (ValueVector v : allocationVectors) {
    -      try {
    -        AllocationHelper.allocateNew(v, current.getRecordCount());
    -      } catch (OutOfMemoryException ex) {
    -        return false;
    -      }
    +  @SuppressWarnings("resource")
    +  private IterOutcome doWork(RecordBatch inputBatch, boolean newSchema) throws ClassTransformationException, IOException, SchemaChangeException {
    +    if (inputBatch.getSchema().getFieldCount() != container.getSchema().getFieldCount()) {
    +      // wrong.
         }
    -    return true;
    -  }
     
    -  @SuppressWarnings("resource")
    -  private IterOutcome doWork() throws ClassTransformationException, IOException, SchemaChangeException {
    -    if (allocationVectors != null) {
    -      for (ValueVector v : allocationVectors) {
    -        v.clear();
    -      }
    +    if (newSchema) {
    +      createUnionAller(inputBatch);
         }
     
    -    allocationVectors = Lists.newArrayList();
    -    transfers.clear();
    +    container.zeroVectors();
     
    -    // If both sides of Union-All are empty
    -    if(unionAllInput.isBothSideEmpty()) {
    -      for(int i = 0; i < outputFields.size(); ++i) {
    -        final String colName = outputFields.get(i).getPath();
    -        final MajorType majorType = MajorType.newBuilder()
    -            .setMinorType(MinorType.INT)
    -            .setMode(DataMode.OPTIONAL)
    -            .build();
    -
    -        MaterializedField outputField = MaterializedField.create(colName, majorType);
    -        ValueVector vv = container.addOrGet(outputField, callBack);
    -        allocationVectors.add(vv);
    -      }
    +    for (final ValueVector v : this.allocationVectors) {
    +      AllocationHelper.allocateNew(v, inputBatch.getRecordCount());
    +    }
     
    -      container.buildSchema(BatchSchema.SelectionVectorMode.NONE);
    +    recordCount = unionall.unionRecords(0, inputBatch.getRecordCount(), 0);
    +    for (final ValueVector v : allocationVectors) {
    +      final ValueVector.Mutator m = v.getMutator();
    +      m.setValueCount(recordCount);
    +    }
    +
    +    if (callBack.getSchemaChangedAndReset()) {
           return IterOutcome.OK_NEW_SCHEMA;
    +    } else {
    +      return IterOutcome.OK;
         }
    +  }
    +
    +  private void createUnionAller(RecordBatch inputBatch) throws ClassTransformationException, IOException, SchemaChangeException {
    +    transfers.clear();
    +    allocationVectors.clear();;
     
         final ClassGenerator<UnionAller> cg = CodeGenerator.getRoot(UnionAller.TEMPLATE_DEFINITION, context.getFunctionRegistry(), context.getOptions());
         cg.getCodeGenerator().plainJavaCapable(true);
         // Uncomment out this line to debug the generated code.
    -//    cg.getCodeGenerator().saveCodeForDebugging(true);
    +    //    cg.getCodeGenerator().saveCodeForDebugging(true);
    +
         int index = 0;
    -    for(VectorWrapper<?> vw : current) {
    -       ValueVector vvIn = vw.getValueVector();
    -      // get the original input column names
    -      SchemaPath inputPath = SchemaPath.getSimplePath(vvIn.getField().getPath());
    -      // get the renamed column names
    -      SchemaPath outputPath = SchemaPath.getSimplePath(outputFields.get(index).getPath());
    +    for(VectorWrapper<?> vw : inputBatch) {
    +      ValueVector vvIn = vw.getValueVector();
    +      ValueVector vvOut = container.getValueVector(index).getValueVector();
     
           final ErrorCollector collector = new ErrorCollectorImpl();
           // According to input data names, Minortypes, Datamodes, choose to
           // transfer directly,
           // rename columns or
           // cast data types (Minortype or DataMode)
    -      if (hasSameTypeAndMode(outputFields.get(index), vw.getValueVector().getField())) {
    +      if (hasSameTypeAndMode(container.getSchema().getColumn(index), vvIn.getField())
    +          && vvIn.getField().getType().getMinorType() != TypeProtos.MinorType.MAP // Per DRILL-5521, existing bug for map transfer
    +          ) {
             // Transfer column
    -
    -        MajorType outputFieldType = outputFields.get(index).getType();
    -        MaterializedField outputField = MaterializedField.create(outputPath.getAsUnescapedPath(), outputFieldType);
    -
    -        /*
    -          todo: Fix if condition when DRILL-4824 is merged
    -          If condition should be changed to:
    -          `if (outputFields.get(index).getPath().equals(inputPath.getAsUnescapedPath())) {`
    -          DRILL-5419 has changed condition to correct one but this caused regression (DRILL-5521).
    -          Root cause is missing indication of child column in map types when it is null.
    -          DRILL-4824 is re-working json reader implementation, including map types and will fix this problem.
    -          Reverting condition to previous one to avoid regression till DRILL-4824 is merged.
    -          Unit test - TestJsonReader.testKvgenWithUnionAll().
    -         */
    -        if (outputFields.get(index).getPath().equals(inputPath)) {
    -          ValueVector vvOut = container.addOrGet(outputField);
    -          TransferPair tp = vvIn.makeTransferPair(vvOut);
    -          transfers.add(tp);
    +        TransferPair tp = vvIn.makeTransferPair(vvOut);
    +        transfers.add(tp);
             // Copy data in order to rename the column
    -        } else {
    -          final LogicalExpression expr = ExpressionTreeMaterializer.materialize(inputPath, current, collector, context.getFunctionRegistry() );
    -          if (collector.hasErrors()) {
    -            throw new SchemaChangeException(String.format("Failure while trying to materialize incoming schema.  Errors:\n %s.", collector.toErrorString()));
    -          }
    -
    -          ValueVector vv = container.addOrGet(outputField, callBack);
    -          allocationVectors.add(vv);
    -          TypedFieldId fid = container.getValueVectorId(SchemaPath.getSimplePath(outputField.getPath()));
    -          ValueVectorWriteExpression write = new ValueVectorWriteExpression(fid, expr, true);
    -          cg.addExpr(write);
    -        }
    -      // Cast is necessary
    +      } else if (vvIn.getField().getType().getMinorType() == TypeProtos.MinorType.NULL) {
    +        continue;
           } else {
    -        LogicalExpression expr = ExpressionTreeMaterializer.materialize(inputPath, current, collector, context.getFunctionRegistry());
    +        SchemaPath inputPath = SchemaPath.getSimplePath(vvIn.getField().getPath());
    +        MaterializedField inField = vvIn.getField();
    +        MaterializedField outputField = vvOut.getField();
    +
    +        LogicalExpression expr = ExpressionTreeMaterializer.materialize(inputPath, inputBatch, collector, context.getFunctionRegistry());
    +
             if (collector.hasErrors()) {
               throw new SchemaChangeException(String.format("Failure while trying to materialize incoming schema.  Errors:\n %s.", collector.toErrorString()));
             }
     
             // If the inputs' DataMode is required and the outputs' DataMode is not required
             // cast to the one with the least restriction
    -        if(vvIn.getField().getType().getMode() == DataMode.REQUIRED
    -            && outputFields.get(index).getType().getMode() != DataMode.REQUIRED) {
    -          expr = ExpressionTreeMaterializer.convertToNullableType(expr, vvIn.getField().getType().getMinorType(), context.getFunctionRegistry(), collector);
    +        if(inField.getType().getMode() == TypeProtos.DataMode.REQUIRED
    +            && outputField.getType().getMode() != TypeProtos.DataMode.REQUIRED) {
    +          expr = ExpressionTreeMaterializer.convertToNullableType(expr, inField.getType().getMinorType(), context.getFunctionRegistry(), collector);
               if (collector.hasErrors()) {
                 throw new SchemaChangeException(String.format("Failure while trying to materialize incoming schema.  Errors:\n %s.", collector.toErrorString()));
               }
             }
     
             // If two inputs' MinorTypes are different,
             // Insert a cast before the Union operation
    -        if(vvIn.getField().getType().getMinorType() != outputFields.get(index).getType().getMinorType()) {
    -          expr = ExpressionTreeMaterializer.addCastExpression(expr, outputFields.get(index).getType(), context.getFunctionRegistry(), collector);
    +        if(inField.getType().getMinorType() != outputField.getType().getMinorType()) {
    +          expr = ExpressionTreeMaterializer.addCastExpression(expr, outputField.getType(), context.getFunctionRegistry(), collector);
               if (collector.hasErrors()) {
                 throw new SchemaChangeException(String.format("Failure while trying to materialize incoming schema.  Errors:\n %s.", collector.toErrorString()));
               }
             }
     
    -        final MaterializedField outputField = MaterializedField.create(outputPath.getAsUnescapedPath(), expr.getMajorType());
    -        ValueVector vector = container.addOrGet(outputField, callBack);
    -        allocationVectors.add(vector);
             TypedFieldId fid = container.getValueVectorId(SchemaPath.getSimplePath(outputField.getPath()));
     
    -        boolean useSetSafe = !(vector instanceof FixedWidthVector);
    +        boolean useSetSafe = !(vvOut instanceof FixedWidthVector);
             ValueVectorWriteExpression write = new ValueVectorWriteExpression(fid, expr, useSetSafe);
             cg.addExpr(write);
    +
    +        allocationVectors.add(vvOut);
           }
           ++index;
         }
     
         unionall = context.getImplementationClass(cg.getCodeGenerator());
    -    unionall.setup(context, current, this, transfers);
    -
    -    if(!schemaAvailable) {
    -      container.buildSchema(BatchSchema.SelectionVectorMode.NONE);
    -      schemaAvailable = true;
    -    }
    -
    -    if(!doAlloc()) {
    -      return IterOutcome.OUT_OF_MEMORY;
    -    }
    -
    -    recordCount = unionall.unionRecords(0, current.getRecordCount(), 0);
    -    setValueCount(recordCount);
    -    return IterOutcome.OK;
    -  }
    -
    -  public static boolean hasSameTypeAndMode(MaterializedField leftField, MaterializedField rightField) {
    -    return (leftField.getType().getMinorType() == rightField.getType().getMinorType())
    -        && (leftField.getType().getMode() == rightField.getType().getMode());
    -  }
    -
    -  // This method is used by inner class to point the reference `current` to the correct record batch
    -  private void setCurrentRecordBatch(RecordBatch target) {
    -    this.current = target;
    -  }
    -
    -  // This method is used by inner class to clear the current record batch
    -  private void clearCurrentRecordBatch() {
    -    for(VectorWrapper<?> v: current) {
    -      v.clear();
    -    }
    +    unionall.setup(context, inputBatch, this, transfers);
       }
     
    -  public static class UnionAllInput {
    -    private UnionAllRecordBatch unionAllRecordBatch;
    -    private List<MaterializedField> outputFields;
    -    private OneSideInput leftSide;
    -    private OneSideInput rightSide;
    -    private IterOutcome upstream = IterOutcome.NOT_YET;
    -    private boolean leftIsFinish = false;
    -    private boolean rightIsFinish = false;
    -
    -    // These two schemas are obtained from the first record batches of the left and right inputs
    -    // They are used to check if the schema is changed between recordbatches
    -    private BatchSchema leftSchema;
    -    private BatchSchema rightSchema;
    -    private boolean bothEmpty = false;
    -
    -    public UnionAllInput(UnionAllRecordBatch unionAllRecordBatch, RecordBatch left, RecordBatch right) {
    -      this.unionAllRecordBatch = unionAllRecordBatch;
    -      leftSide = new OneSideInput(left);
    -      rightSide = new OneSideInput(right);
    -    }
    -
    -    private void setBothSideEmpty(boolean bothEmpty) {
    -      this.bothEmpty = bothEmpty;
    -    }
    -
    -    private boolean isBothSideEmpty() {
    -      return bothEmpty;
    -    }
    -
    -    public IterOutcome nextBatch() throws SchemaChangeException {
    -      if(upstream == RecordBatch.IterOutcome.NOT_YET) {
    -        IterOutcome iterLeft = leftSide.nextBatch();
    -        switch(iterLeft) {
    -          case OK_NEW_SCHEMA:
    -            /*
    -             * If the first few record batches are all empty,
    -             * there is no way to tell whether these empty batches are coming from empty files.
    -             * It is incorrect to infer output types when either side could be coming from empty.
    -             *
    -             * Thus, while-loop is necessary to skip those empty batches.
    -             */
    -            whileLoop:
    -            while(leftSide.getRecordBatch().getRecordCount() == 0) {
    -              iterLeft = leftSide.nextBatch();
    -
    -              switch(iterLeft) {
    -                case STOP:
    -                case OUT_OF_MEMORY:
    -                  return iterLeft;
    -
    -                case NONE:
    -                  // Special Case: The left side was an empty input.
    -                  leftIsFinish = true;
    -                  break whileLoop;
    -
    -                case NOT_YET:
    -                case OK_NEW_SCHEMA:
    -                case OK:
    -                  continue whileLoop;
    -
    -                default:
    -                  throw new IllegalStateException(
    -                      String.format("Unexpected state %s.", iterLeft));
    -              }
    -            }
    -
    -            break;
    -          case STOP:
    -          case OUT_OF_MEMORY:
    -            return iterLeft;
    -
    -          default:
    -            throw new IllegalStateException(
    -                String.format("Unexpected state %s.", iterLeft));
    -        }
    -
    -        IterOutcome iterRight = rightSide.nextBatch();
    -        switch(iterRight) {
    -          case OK_NEW_SCHEMA:
    -            // Unless there is no record batch on the left side of the inputs,
    -            // always start processing from the left side.
    -            if(leftIsFinish) {
    -              unionAllRecordBatch.setCurrentRecordBatch(rightSide.getRecordBatch());
    -            } else {
    -              unionAllRecordBatch.setCurrentRecordBatch(leftSide.getRecordBatch());
    -            }
    -            // If the record count of the first batch from right input is zero,
    -            // there are two possibilities:
    -            // 1. The right side is an empty input (e.g., file).
    -            // 2. There will be more records carried by later batches.
    -
    -            /*
    -             * If the first few record batches are all empty,
    -             * there is no way to tell whether these empty batches are coming from empty files.
    -             * It is incorrect to infer output types when either side could be coming from empty.
    -             *
    -             * Thus, while-loop is necessary to skip those empty batches.
    -             */
    -            whileLoop:
    -            while(rightSide.getRecordBatch().getRecordCount() == 0) {
    -              iterRight = rightSide.nextBatch();
    -              switch(iterRight) {
    -                case STOP:
    -                case OUT_OF_MEMORY:
    -                  return iterRight;
    -
    -                case NONE:
    -                  // Special Case: The right side was an empty input.
    -                  rightIsFinish = true;
    -                  break whileLoop;
    -
    -                case NOT_YET:
    -                case OK_NEW_SCHEMA:
    -                case OK:
    -                  continue whileLoop;
    -
    -                default:
    -                  throw new IllegalStateException(
    -                      String.format("Unexpected state %s.", iterRight));
    -              }
    -            }
    -
    -            if(leftIsFinish && rightIsFinish) {
    -              setBothSideEmpty(true);
    -            }
    -
    -            inferOutputFields();
    -            break;
    -
    -          case STOP:
    -          case OUT_OF_MEMORY:
    -            return iterRight;
    -
    -          default:
    -            throw new IllegalStateException(
    -                String.format("Unexpected state %s.", iterRight));
    -        }
    -
    -
    -
    -        upstream = IterOutcome.OK_NEW_SCHEMA;
    -        return upstream;
    -      } else {
    -        if(isBothSideEmpty()) {
    -          return IterOutcome.NONE;
    -        }
    -
    -        unionAllRecordBatch.clearCurrentRecordBatch();
    -
    -        if(leftIsFinish && rightIsFinish) {
    -          upstream = IterOutcome.NONE;
    -          return upstream;
    -        } else if(leftIsFinish) {
    -          IterOutcome iterOutcome = rightSide.nextBatch();
    -
    -          switch(iterOutcome) {
    -            case NONE:
    -              rightIsFinish = true;
    -              // fall through
    -            case STOP:
    -            case OUT_OF_MEMORY:
    -              upstream = iterOutcome;
    -              return upstream;
    -
    -            case OK_NEW_SCHEMA:
    -              if(!rightSide.getRecordBatch().getSchema().equals(rightSchema)) {
    -                throw new SchemaChangeException("Schema change detected in the right input of Union-All. This is not currently supported");
    -              }
    -              iterOutcome = IterOutcome.OK;
    -              // fall through
    -            case OK:
    -              unionAllRecordBatch.setCurrentRecordBatch(rightSide.getRecordBatch());
    -              upstream = iterOutcome;
    -              return upstream;
    -
    -            default:
    -              throw new IllegalStateException(String.format("Unknown state %s.", upstream));
    -          }
    -        } else if(rightIsFinish) {
    -          IterOutcome iterOutcome = leftSide.nextBatch();
    -          switch(iterOutcome) {
    -            case STOP:
    -            case OUT_OF_MEMORY:
    -            case NONE:
    -              upstream = iterOutcome;
    -              return upstream;
    -
    -            case OK:
    -              unionAllRecordBatch.setCurrentRecordBatch(leftSide.getRecordBatch());
    -              upstream = iterOutcome;
    -              return upstream;
    -
    -            default:
    -              throw new IllegalStateException(String.format("Unknown state %s.", iterOutcome));
    -          }
    -        } else {
    -          IterOutcome iterOutcome = leftSide.nextBatch();
    -
    -          switch(iterOutcome) {
    -            case STOP:
    -            case OUT_OF_MEMORY:
    -              upstream = iterOutcome;
    -              return upstream;
    -
    -            case OK_NEW_SCHEMA:
    -              if(!leftSide.getRecordBatch().getSchema().equals(leftSchema)) {
    -                throw new SchemaChangeException("Schema change detected in the left input of Union-All. This is not currently supported");
    -              }
    -
    -              iterOutcome = IterOutcome.OK;
    -              // fall through
    -            case OK:
    -              unionAllRecordBatch.setCurrentRecordBatch(leftSide.getRecordBatch());
    -              upstream = iterOutcome;
    -              return upstream;
    -
    -            case NONE:
    -              unionAllRecordBatch.setCurrentRecordBatch(rightSide.getRecordBatch());
    -              upstream = IterOutcome.OK;
    -              leftIsFinish = true;
    -              return upstream;
    -
    -            default:
    -              throw new IllegalStateException(String.format("Unknown state %s.", upstream));
    -          }
    -        }
    -      }
    -    }
     
    -    /**
    -     *
    -     * Summarize the inference in the four different situations:
    -     * First of all, the field names are always determined by the left side
    -     * (Even when the left side is from an empty file, we have the column names.)
    -     *
    -     * Cases:
    -     * 1. Left: non-empty; Right: non-empty
    -     *      types determined by both sides with implicit casting involved
    -     * 2. Left: empty; Right: non-empty
    -     *      type from the right
    -     * 3. Left: non-empty; Right: empty
    -     *      types from the left
    -     * 4. Left: empty; Right: empty
    -     *      types are nullable integer
    -     */
    -    private void inferOutputFields() {
    -      if(!leftIsFinish && !rightIsFinish) {
    -        // Both sides are non-empty
    -        inferOutputFieldsBothSide();
    -      } else if(!rightIsFinish) {
    -        // Left side is non-empty
    -        // While use left side's column names as output column names,
    -        // use right side's column types as output column types.
    -        inferOutputFieldsFromSingleSide(
    -            leftSide.getRecordBatch().getSchema(),
    -            rightSide.getRecordBatch().getSchema());
    +  // The output table's column names always follow the left table,
    --- End diff --
    
    I actually did not change the logic of existing union operator to resolve the output name/type.  Please see the original design doc of union all ( it should be in one of JIRA for union all support).
    
    AFAIK, union operator uses position, in stead of names. 



---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] drill pull request #906: DRILL-5546: Handle schema change exception failure ...

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

    https://github.com/apache/drill/pull/906#discussion_r135351026
  
    --- Diff: exec/java-exec/src/main/java/org/apache/drill/exec/physical/config/Project.java ---
    @@ -35,9 +35,19 @@
       static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(Project.class);
     
       private final List<NamedExpression> exprs;
    +  /**
    +   * {@link org.apache.drill.exec.planner.physical.ProjectPrel for the meaning of flag 'outputProj'}
    +   */
    +  private boolean outputProj = false;
     
       @JsonCreator
    -  public Project(@JsonProperty("exprs") List<NamedExpression> exprs, @JsonProperty("child") PhysicalOperator child) {
    +  public Project(@JsonProperty("exprs") List<NamedExpression> exprs, @JsonProperty("child") PhysicalOperator child, @JsonProperty("outputproj") boolean outputProj) {
    --- End diff --
    
    Done.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] drill pull request #906: DRILL-5546: Handle schema change exception failure ...

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

    https://github.com/apache/drill/pull/906#discussion_r136154993
  
    --- Diff: exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/union/UnionAllRecordBatch.java ---
    @@ -130,562 +145,248 @@ public IterOutcome innerNext() {
       }
     
       @Override
    -  public WritableBatch getWritableBatch() {
    -    return WritableBatch.get(this);
    +  public int getRecordCount() {
    +    return recordCount;
       }
     
    -  private void setValueCount(int count) {
    -    for (ValueVector v : allocationVectors) {
    -      ValueVector.Mutator m = v.getMutator();
    -      m.setValueCount(count);
    -    }
    -  }
     
    -  private boolean doAlloc() {
    -    for (ValueVector v : allocationVectors) {
    -      try {
    -        AllocationHelper.allocateNew(v, current.getRecordCount());
    -      } catch (OutOfMemoryException ex) {
    -        return false;
    -      }
    +  @SuppressWarnings("resource")
    +  private IterOutcome doWork(RecordBatch inputBatch, boolean newSchema) throws ClassTransformationException, IOException, SchemaChangeException {
    +    if (inputBatch.getSchema().getFieldCount() != container.getSchema().getFieldCount()) {
    +      // wrong.
         }
    -    return true;
    -  }
     
    -  @SuppressWarnings("resource")
    -  private IterOutcome doWork() throws ClassTransformationException, IOException, SchemaChangeException {
    -    if (allocationVectors != null) {
    -      for (ValueVector v : allocationVectors) {
    -        v.clear();
    -      }
    +    if (newSchema) {
    +      createUnionAller(inputBatch);
         }
     
    -    allocationVectors = Lists.newArrayList();
    -    transfers.clear();
    +    container.zeroVectors();
     
    -    // If both sides of Union-All are empty
    -    if(unionAllInput.isBothSideEmpty()) {
    -      for(int i = 0; i < outputFields.size(); ++i) {
    -        final String colName = outputFields.get(i).getPath();
    -        final MajorType majorType = MajorType.newBuilder()
    -            .setMinorType(MinorType.INT)
    -            .setMode(DataMode.OPTIONAL)
    -            .build();
    -
    -        MaterializedField outputField = MaterializedField.create(colName, majorType);
    -        ValueVector vv = container.addOrGet(outputField, callBack);
    -        allocationVectors.add(vv);
    -      }
    +    for (final ValueVector v : this.allocationVectors) {
    +      AllocationHelper.allocateNew(v, inputBatch.getRecordCount());
    +    }
     
    -      container.buildSchema(BatchSchema.SelectionVectorMode.NONE);
    +    recordCount = unionall.unionRecords(0, inputBatch.getRecordCount(), 0);
    +    for (final ValueVector v : allocationVectors) {
    +      final ValueVector.Mutator m = v.getMutator();
    +      m.setValueCount(recordCount);
    +    }
    +
    +    if (callBack.getSchemaChangedAndReset()) {
           return IterOutcome.OK_NEW_SCHEMA;
    +    } else {
    +      return IterOutcome.OK;
         }
    +  }
    +
    +  private void createUnionAller(RecordBatch inputBatch) throws ClassTransformationException, IOException, SchemaChangeException {
    +    transfers.clear();
    +    allocationVectors.clear();;
     
         final ClassGenerator<UnionAller> cg = CodeGenerator.getRoot(UnionAller.TEMPLATE_DEFINITION, context.getFunctionRegistry(), context.getOptions());
         cg.getCodeGenerator().plainJavaCapable(true);
         // Uncomment out this line to debug the generated code.
    -//    cg.getCodeGenerator().saveCodeForDebugging(true);
    +    //    cg.getCodeGenerator().saveCodeForDebugging(true);
    +
         int index = 0;
    -    for(VectorWrapper<?> vw : current) {
    -       ValueVector vvIn = vw.getValueVector();
    -      // get the original input column names
    -      SchemaPath inputPath = SchemaPath.getSimplePath(vvIn.getField().getPath());
    -      // get the renamed column names
    -      SchemaPath outputPath = SchemaPath.getSimplePath(outputFields.get(index).getPath());
    +    for(VectorWrapper<?> vw : inputBatch) {
    +      ValueVector vvIn = vw.getValueVector();
    +      ValueVector vvOut = container.getValueVector(index).getValueVector();
     
           final ErrorCollector collector = new ErrorCollectorImpl();
           // According to input data names, Minortypes, Datamodes, choose to
           // transfer directly,
           // rename columns or
           // cast data types (Minortype or DataMode)
    -      if (hasSameTypeAndMode(outputFields.get(index), vw.getValueVector().getField())) {
    +      if (hasSameTypeAndMode(container.getSchema().getColumn(index), vvIn.getField())
    +          && vvIn.getField().getType().getMinorType() != TypeProtos.MinorType.MAP // Per DRILL-5521, existing bug for map transfer
    +          ) {
             // Transfer column
    -
    -        MajorType outputFieldType = outputFields.get(index).getType();
    -        MaterializedField outputField = MaterializedField.create(outputPath.getAsUnescapedPath(), outputFieldType);
    -
    -        /*
    -          todo: Fix if condition when DRILL-4824 is merged
    -          If condition should be changed to:
    -          `if (outputFields.get(index).getPath().equals(inputPath.getAsUnescapedPath())) {`
    -          DRILL-5419 has changed condition to correct one but this caused regression (DRILL-5521).
    -          Root cause is missing indication of child column in map types when it is null.
    -          DRILL-4824 is re-working json reader implementation, including map types and will fix this problem.
    -          Reverting condition to previous one to avoid regression till DRILL-4824 is merged.
    -          Unit test - TestJsonReader.testKvgenWithUnionAll().
    -         */
    -        if (outputFields.get(index).getPath().equals(inputPath)) {
    -          ValueVector vvOut = container.addOrGet(outputField);
    -          TransferPair tp = vvIn.makeTransferPair(vvOut);
    -          transfers.add(tp);
    +        TransferPair tp = vvIn.makeTransferPair(vvOut);
    +        transfers.add(tp);
             // Copy data in order to rename the column
    -        } else {
    -          final LogicalExpression expr = ExpressionTreeMaterializer.materialize(inputPath, current, collector, context.getFunctionRegistry() );
    -          if (collector.hasErrors()) {
    -            throw new SchemaChangeException(String.format("Failure while trying to materialize incoming schema.  Errors:\n %s.", collector.toErrorString()));
    -          }
    -
    -          ValueVector vv = container.addOrGet(outputField, callBack);
    -          allocationVectors.add(vv);
    -          TypedFieldId fid = container.getValueVectorId(SchemaPath.getSimplePath(outputField.getPath()));
    -          ValueVectorWriteExpression write = new ValueVectorWriteExpression(fid, expr, true);
    -          cg.addExpr(write);
    -        }
    -      // Cast is necessary
    +      } else if (vvIn.getField().getType().getMinorType() == TypeProtos.MinorType.NULL) {
    +        continue;
           } else {
    -        LogicalExpression expr = ExpressionTreeMaterializer.materialize(inputPath, current, collector, context.getFunctionRegistry());
    +        SchemaPath inputPath = SchemaPath.getSimplePath(vvIn.getField().getPath());
    +        MaterializedField inField = vvIn.getField();
    +        MaterializedField outputField = vvOut.getField();
    +
    +        LogicalExpression expr = ExpressionTreeMaterializer.materialize(inputPath, inputBatch, collector, context.getFunctionRegistry());
    +
             if (collector.hasErrors()) {
               throw new SchemaChangeException(String.format("Failure while trying to materialize incoming schema.  Errors:\n %s.", collector.toErrorString()));
             }
     
             // If the inputs' DataMode is required and the outputs' DataMode is not required
             // cast to the one with the least restriction
    -        if(vvIn.getField().getType().getMode() == DataMode.REQUIRED
    -            && outputFields.get(index).getType().getMode() != DataMode.REQUIRED) {
    -          expr = ExpressionTreeMaterializer.convertToNullableType(expr, vvIn.getField().getType().getMinorType(), context.getFunctionRegistry(), collector);
    +        if(inField.getType().getMode() == TypeProtos.DataMode.REQUIRED
    +            && outputField.getType().getMode() != TypeProtos.DataMode.REQUIRED) {
    +          expr = ExpressionTreeMaterializer.convertToNullableType(expr, inField.getType().getMinorType(), context.getFunctionRegistry(), collector);
               if (collector.hasErrors()) {
                 throw new SchemaChangeException(String.format("Failure while trying to materialize incoming schema.  Errors:\n %s.", collector.toErrorString()));
               }
             }
     
             // If two inputs' MinorTypes are different,
             // Insert a cast before the Union operation
    -        if(vvIn.getField().getType().getMinorType() != outputFields.get(index).getType().getMinorType()) {
    -          expr = ExpressionTreeMaterializer.addCastExpression(expr, outputFields.get(index).getType(), context.getFunctionRegistry(), collector);
    +        if(inField.getType().getMinorType() != outputField.getType().getMinorType()) {
    +          expr = ExpressionTreeMaterializer.addCastExpression(expr, outputField.getType(), context.getFunctionRegistry(), collector);
               if (collector.hasErrors()) {
                 throw new SchemaChangeException(String.format("Failure while trying to materialize incoming schema.  Errors:\n %s.", collector.toErrorString()));
               }
             }
     
    -        final MaterializedField outputField = MaterializedField.create(outputPath.getAsUnescapedPath(), expr.getMajorType());
    -        ValueVector vector = container.addOrGet(outputField, callBack);
    -        allocationVectors.add(vector);
             TypedFieldId fid = container.getValueVectorId(SchemaPath.getSimplePath(outputField.getPath()));
     
    -        boolean useSetSafe = !(vector instanceof FixedWidthVector);
    +        boolean useSetSafe = !(vvOut instanceof FixedWidthVector);
             ValueVectorWriteExpression write = new ValueVectorWriteExpression(fid, expr, useSetSafe);
             cg.addExpr(write);
    +
    +        allocationVectors.add(vvOut);
           }
           ++index;
         }
     
         unionall = context.getImplementationClass(cg.getCodeGenerator());
    -    unionall.setup(context, current, this, transfers);
    -
    -    if(!schemaAvailable) {
    -      container.buildSchema(BatchSchema.SelectionVectorMode.NONE);
    -      schemaAvailable = true;
    -    }
    -
    -    if(!doAlloc()) {
    -      return IterOutcome.OUT_OF_MEMORY;
    -    }
    -
    -    recordCount = unionall.unionRecords(0, current.getRecordCount(), 0);
    -    setValueCount(recordCount);
    -    return IterOutcome.OK;
    -  }
    -
    -  public static boolean hasSameTypeAndMode(MaterializedField leftField, MaterializedField rightField) {
    -    return (leftField.getType().getMinorType() == rightField.getType().getMinorType())
    -        && (leftField.getType().getMode() == rightField.getType().getMode());
    -  }
    -
    -  // This method is used by inner class to point the reference `current` to the correct record batch
    -  private void setCurrentRecordBatch(RecordBatch target) {
    -    this.current = target;
    -  }
    -
    -  // This method is used by inner class to clear the current record batch
    -  private void clearCurrentRecordBatch() {
    -    for(VectorWrapper<?> v: current) {
    -      v.clear();
    -    }
    +    unionall.setup(context, inputBatch, this, transfers);
       }
     
    -  public static class UnionAllInput {
    -    private UnionAllRecordBatch unionAllRecordBatch;
    -    private List<MaterializedField> outputFields;
    -    private OneSideInput leftSide;
    -    private OneSideInput rightSide;
    -    private IterOutcome upstream = IterOutcome.NOT_YET;
    -    private boolean leftIsFinish = false;
    -    private boolean rightIsFinish = false;
    -
    -    // These two schemas are obtained from the first record batches of the left and right inputs
    -    // They are used to check if the schema is changed between recordbatches
    -    private BatchSchema leftSchema;
    -    private BatchSchema rightSchema;
    -    private boolean bothEmpty = false;
    -
    -    public UnionAllInput(UnionAllRecordBatch unionAllRecordBatch, RecordBatch left, RecordBatch right) {
    -      this.unionAllRecordBatch = unionAllRecordBatch;
    -      leftSide = new OneSideInput(left);
    -      rightSide = new OneSideInput(right);
    -    }
    -
    -    private void setBothSideEmpty(boolean bothEmpty) {
    -      this.bothEmpty = bothEmpty;
    -    }
    -
    -    private boolean isBothSideEmpty() {
    -      return bothEmpty;
    -    }
    -
    -    public IterOutcome nextBatch() throws SchemaChangeException {
    -      if(upstream == RecordBatch.IterOutcome.NOT_YET) {
    -        IterOutcome iterLeft = leftSide.nextBatch();
    -        switch(iterLeft) {
    -          case OK_NEW_SCHEMA:
    -            /*
    -             * If the first few record batches are all empty,
    -             * there is no way to tell whether these empty batches are coming from empty files.
    -             * It is incorrect to infer output types when either side could be coming from empty.
    -             *
    -             * Thus, while-loop is necessary to skip those empty batches.
    -             */
    -            whileLoop:
    -            while(leftSide.getRecordBatch().getRecordCount() == 0) {
    -              iterLeft = leftSide.nextBatch();
    -
    -              switch(iterLeft) {
    -                case STOP:
    -                case OUT_OF_MEMORY:
    -                  return iterLeft;
    -
    -                case NONE:
    -                  // Special Case: The left side was an empty input.
    -                  leftIsFinish = true;
    -                  break whileLoop;
    -
    -                case NOT_YET:
    -                case OK_NEW_SCHEMA:
    -                case OK:
    -                  continue whileLoop;
    -
    -                default:
    -                  throw new IllegalStateException(
    -                      String.format("Unexpected state %s.", iterLeft));
    -              }
    -            }
    -
    -            break;
    -          case STOP:
    -          case OUT_OF_MEMORY:
    -            return iterLeft;
    -
    -          default:
    -            throw new IllegalStateException(
    -                String.format("Unexpected state %s.", iterLeft));
    -        }
    -
    -        IterOutcome iterRight = rightSide.nextBatch();
    -        switch(iterRight) {
    -          case OK_NEW_SCHEMA:
    -            // Unless there is no record batch on the left side of the inputs,
    -            // always start processing from the left side.
    -            if(leftIsFinish) {
    -              unionAllRecordBatch.setCurrentRecordBatch(rightSide.getRecordBatch());
    -            } else {
    -              unionAllRecordBatch.setCurrentRecordBatch(leftSide.getRecordBatch());
    -            }
    -            // If the record count of the first batch from right input is zero,
    -            // there are two possibilities:
    -            // 1. The right side is an empty input (e.g., file).
    -            // 2. There will be more records carried by later batches.
    -
    -            /*
    -             * If the first few record batches are all empty,
    -             * there is no way to tell whether these empty batches are coming from empty files.
    -             * It is incorrect to infer output types when either side could be coming from empty.
    -             *
    -             * Thus, while-loop is necessary to skip those empty batches.
    -             */
    -            whileLoop:
    -            while(rightSide.getRecordBatch().getRecordCount() == 0) {
    -              iterRight = rightSide.nextBatch();
    -              switch(iterRight) {
    -                case STOP:
    -                case OUT_OF_MEMORY:
    -                  return iterRight;
    -
    -                case NONE:
    -                  // Special Case: The right side was an empty input.
    -                  rightIsFinish = true;
    -                  break whileLoop;
    -
    -                case NOT_YET:
    -                case OK_NEW_SCHEMA:
    -                case OK:
    -                  continue whileLoop;
    -
    -                default:
    -                  throw new IllegalStateException(
    -                      String.format("Unexpected state %s.", iterRight));
    -              }
    -            }
    -
    -            if(leftIsFinish && rightIsFinish) {
    -              setBothSideEmpty(true);
    -            }
    -
    -            inferOutputFields();
    -            break;
    -
    -          case STOP:
    -          case OUT_OF_MEMORY:
    -            return iterRight;
    -
    -          default:
    -            throw new IllegalStateException(
    -                String.format("Unexpected state %s.", iterRight));
    -        }
    -
    -
    -
    -        upstream = IterOutcome.OK_NEW_SCHEMA;
    -        return upstream;
    -      } else {
    -        if(isBothSideEmpty()) {
    -          return IterOutcome.NONE;
    -        }
    -
    -        unionAllRecordBatch.clearCurrentRecordBatch();
    -
    -        if(leftIsFinish && rightIsFinish) {
    -          upstream = IterOutcome.NONE;
    -          return upstream;
    -        } else if(leftIsFinish) {
    -          IterOutcome iterOutcome = rightSide.nextBatch();
    -
    -          switch(iterOutcome) {
    -            case NONE:
    -              rightIsFinish = true;
    -              // fall through
    -            case STOP:
    -            case OUT_OF_MEMORY:
    -              upstream = iterOutcome;
    -              return upstream;
    -
    -            case OK_NEW_SCHEMA:
    -              if(!rightSide.getRecordBatch().getSchema().equals(rightSchema)) {
    -                throw new SchemaChangeException("Schema change detected in the right input of Union-All. This is not currently supported");
    -              }
    -              iterOutcome = IterOutcome.OK;
    -              // fall through
    -            case OK:
    -              unionAllRecordBatch.setCurrentRecordBatch(rightSide.getRecordBatch());
    -              upstream = iterOutcome;
    -              return upstream;
    -
    -            default:
    -              throw new IllegalStateException(String.format("Unknown state %s.", upstream));
    -          }
    -        } else if(rightIsFinish) {
    -          IterOutcome iterOutcome = leftSide.nextBatch();
    -          switch(iterOutcome) {
    -            case STOP:
    -            case OUT_OF_MEMORY:
    -            case NONE:
    -              upstream = iterOutcome;
    -              return upstream;
    -
    -            case OK:
    -              unionAllRecordBatch.setCurrentRecordBatch(leftSide.getRecordBatch());
    -              upstream = iterOutcome;
    -              return upstream;
    -
    -            default:
    -              throw new IllegalStateException(String.format("Unknown state %s.", iterOutcome));
    -          }
    -        } else {
    -          IterOutcome iterOutcome = leftSide.nextBatch();
    -
    -          switch(iterOutcome) {
    -            case STOP:
    -            case OUT_OF_MEMORY:
    -              upstream = iterOutcome;
    -              return upstream;
    -
    -            case OK_NEW_SCHEMA:
    -              if(!leftSide.getRecordBatch().getSchema().equals(leftSchema)) {
    -                throw new SchemaChangeException("Schema change detected in the left input of Union-All. This is not currently supported");
    -              }
    -
    -              iterOutcome = IterOutcome.OK;
    -              // fall through
    -            case OK:
    -              unionAllRecordBatch.setCurrentRecordBatch(leftSide.getRecordBatch());
    -              upstream = iterOutcome;
    -              return upstream;
    -
    -            case NONE:
    -              unionAllRecordBatch.setCurrentRecordBatch(rightSide.getRecordBatch());
    -              upstream = IterOutcome.OK;
    -              leftIsFinish = true;
    -              return upstream;
    -
    -            default:
    -              throw new IllegalStateException(String.format("Unknown state %s.", upstream));
    -          }
    -        }
    -      }
    -    }
     
    -    /**
    -     *
    -     * Summarize the inference in the four different situations:
    -     * First of all, the field names are always determined by the left side
    -     * (Even when the left side is from an empty file, we have the column names.)
    -     *
    -     * Cases:
    -     * 1. Left: non-empty; Right: non-empty
    -     *      types determined by both sides with implicit casting involved
    -     * 2. Left: empty; Right: non-empty
    -     *      type from the right
    -     * 3. Left: non-empty; Right: empty
    -     *      types from the left
    -     * 4. Left: empty; Right: empty
    -     *      types are nullable integer
    -     */
    -    private void inferOutputFields() {
    -      if(!leftIsFinish && !rightIsFinish) {
    -        // Both sides are non-empty
    -        inferOutputFieldsBothSide();
    -      } else if(!rightIsFinish) {
    -        // Left side is non-empty
    -        // While use left side's column names as output column names,
    -        // use right side's column types as output column types.
    -        inferOutputFieldsFromSingleSide(
    -            leftSide.getRecordBatch().getSchema(),
    -            rightSide.getRecordBatch().getSchema());
    +  // The output table's column names always follow the left table,
    +  // where the output type is chosen based on DRILL's implicit casting rules
    +  private void inferOutputFieldsBothSide(final BatchSchema leftSchema, final BatchSchema rightSchema) {
    +//    outputFields = Lists.newArrayList();
    +    final Iterator<MaterializedField> leftIter = leftSchema.iterator();
    +    final Iterator<MaterializedField> rightIter = rightSchema.iterator();
    +
    +    int index = 1;
    +    while (leftIter.hasNext() && rightIter.hasNext()) {
    +      MaterializedField leftField  = leftIter.next();
    +      MaterializedField rightField = rightIter.next();
    +
    +      if (hasSameTypeAndMode(leftField, rightField)) {
    +        TypeProtos.MajorType.Builder builder = TypeProtos.MajorType.newBuilder().setMinorType(leftField.getType().getMinorType()).setMode(leftField.getDataMode());
    +        builder = Types.calculateTypePrecisionAndScale(leftField.getType(), rightField.getType(), builder);
    +        container.addOrGet(MaterializedField.create(leftField.getPath(), builder.build()), callBack);
    +      } else if (Types.isUntypedNull(rightField.getType())) {
    +        container.addOrGet(leftField, callBack);
    +      } else if (Types.isUntypedNull(leftField.getType())) {
    +        container.addOrGet(MaterializedField.create(leftField.getPath(), rightField.getType()), callBack);
           } else {
    -        // Either right side is empty or both are empty
    -        // Using left side's schema is sufficient
    -        inferOutputFieldsFromSingleSide(
    -            leftSide.getRecordBatch().getSchema(),
    -            leftSide.getRecordBatch().getSchema());
    -      }
    -    }
    -
    -    // The output table's column names always follow the left table,
    -    // where the output type is chosen based on DRILL's implicit casting rules
    -    private void inferOutputFieldsBothSide() {
    -      outputFields = Lists.newArrayList();
    -      leftSchema = leftSide.getRecordBatch().getSchema();
    -      rightSchema = rightSide.getRecordBatch().getSchema();
    -      Iterator<MaterializedField> leftIter = leftSchema.iterator();
    -      Iterator<MaterializedField> rightIter = rightSchema.iterator();
    -
    -      int index = 1;
    -      while (leftIter.hasNext() && rightIter.hasNext()) {
    -        MaterializedField leftField  = leftIter.next();
    -        MaterializedField rightField = rightIter.next();
    -
    -        if (hasSameTypeAndMode(leftField, rightField)) {
    -          MajorType.Builder builder = MajorType.newBuilder().setMinorType(leftField.getType().getMinorType()).setMode(leftField.getDataMode());
    +        // If the output type is not the same,
    +        // cast the column of one of the table to a data type which is the Least Restrictive
    +        TypeProtos.MajorType.Builder builder = TypeProtos.MajorType.newBuilder();
    +        if (leftField.getType().getMinorType() == rightField.getType().getMinorType()) {
    +          builder.setMinorType(leftField.getType().getMinorType());
               builder = Types.calculateTypePrecisionAndScale(leftField.getType(), rightField.getType(), builder);
    -          outputFields.add(MaterializedField.create(leftField.getPath(), builder.build()));
             } else {
    -          // If the output type is not the same,
    -          // cast the column of one of the table to a data type which is the Least Restrictive
    -          MajorType.Builder builder = MajorType.newBuilder();
    -          if (leftField.getType().getMinorType() == rightField.getType().getMinorType()) {
    -            builder.setMinorType(leftField.getType().getMinorType());
    -            builder = Types.calculateTypePrecisionAndScale(leftField.getType(), rightField.getType(), builder);
    -          } else {
    -            List<MinorType> types = Lists.newLinkedList();
    -            types.add(leftField.getType().getMinorType());
    -            types.add(rightField.getType().getMinorType());
    -            MinorType outputMinorType = TypeCastRules.getLeastRestrictiveType(types);
    -            if (outputMinorType == null) {
    -              throw new DrillRuntimeException("Type mismatch between " + leftField.getType().getMinorType().toString() +
    -                  " on the left side and " + rightField.getType().getMinorType().toString() +
    -                  " on the right side in column " + index + " of UNION ALL");
    -            }
    -            builder.setMinorType(outputMinorType);
    +          List<TypeProtos.MinorType> types = Lists.newLinkedList();
    +          types.add(leftField.getType().getMinorType());
    +          types.add(rightField.getType().getMinorType());
    +          TypeProtos.MinorType outputMinorType = TypeCastRules.getLeastRestrictiveType(types);
    +          if (outputMinorType == null) {
    +            throw new DrillRuntimeException("Type mismatch between " + leftField.getType().getMinorType().toString() +
    +                " on the left side and " + rightField.getType().getMinorType().toString() +
    +                " on the right side in column " + index + " of UNION ALL");
               }
    -
    -          // The output data mode should be as flexible as the more flexible one from the two input tables
    -          List<DataMode> dataModes = Lists.newLinkedList();
    -          dataModes.add(leftField.getType().getMode());
    -          dataModes.add(rightField.getType().getMode());
    -          builder.setMode(TypeCastRules.getLeastRestrictiveDataMode(dataModes));
    -
    -          outputFields.add(MaterializedField.create(leftField.getPath(), builder.build()));
    +          builder.setMinorType(outputMinorType);
             }
    -        ++index;
    -      }
    -
    -      assert !leftIter.hasNext() && ! rightIter.hasNext() : "Mis-match of column count should have been detected when validating sqlNode at planning";
    -    }
     
    -    private void inferOutputFieldsFromSingleSide(final BatchSchema schemaForNames, final BatchSchema schemaForTypes) {
    -      outputFields = Lists.newArrayList();
    +        // The output data mode should be as flexible as the more flexible one from the two input tables
    +        List<TypeProtos.DataMode> dataModes = Lists.newLinkedList();
    +        dataModes.add(leftField.getType().getMode());
    +        dataModes.add(rightField.getType().getMode());
    +        builder.setMode(TypeCastRules.getLeastRestrictiveDataMode(dataModes));
     
    -      final List<String> outputColumnNames = Lists.newArrayList();
    -      final Iterator<MaterializedField> iterForNames = schemaForNames.iterator();
    -      while(iterForNames.hasNext()) {
    -        outputColumnNames.add(iterForNames.next().getPath());
    -      }
    -
    -      final Iterator<MaterializedField> iterForTypes = schemaForTypes.iterator();
    -      for(int i = 0; iterForTypes.hasNext(); ++i) {
    -        MaterializedField field = iterForTypes.next();
    -        outputFields.add(MaterializedField.create(outputColumnNames.get(i), field.getType()));
    +        container.addOrGet(MaterializedField.create(leftField.getPath(), builder.build()), callBack);
           }
    +      ++index;
         }
     
    -    public List<MaterializedField> getOutputFields() {
    -      if(outputFields == null) {
    -        throw new NullPointerException("Output fields have not been inferred");
    -      }
    -
    -      return outputFields;
    -    }
    +    assert !leftIter.hasNext() && ! rightIter.hasNext() : "Mis-match of column count should have been detected when validating sqlNode at planning";
    --- End diff --
    
    This is also in the original code. (Seems git diff did not do a good job). I think it might need cleanup, but it's probably out of scope for this JIRA. 
    
    1. https://github.com/apache/drill/blob/master/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/union/UnionAllRecordBatch.java#L620


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] drill pull request #906: DRILL-5546: Handle schema change exception failure ...

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

    https://github.com/apache/drill/pull/906#discussion_r136154585
  
    --- Diff: exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/union/UnionAllRecordBatch.java ---
    @@ -130,562 +145,248 @@ public IterOutcome innerNext() {
       }
     
       @Override
    -  public WritableBatch getWritableBatch() {
    -    return WritableBatch.get(this);
    +  public int getRecordCount() {
    +    return recordCount;
       }
     
    -  private void setValueCount(int count) {
    -    for (ValueVector v : allocationVectors) {
    -      ValueVector.Mutator m = v.getMutator();
    -      m.setValueCount(count);
    -    }
    -  }
     
    -  private boolean doAlloc() {
    -    for (ValueVector v : allocationVectors) {
    -      try {
    -        AllocationHelper.allocateNew(v, current.getRecordCount());
    -      } catch (OutOfMemoryException ex) {
    -        return false;
    -      }
    +  @SuppressWarnings("resource")
    +  private IterOutcome doWork(RecordBatch inputBatch, boolean newSchema) throws ClassTransformationException, IOException, SchemaChangeException {
    +    if (inputBatch.getSchema().getFieldCount() != container.getSchema().getFieldCount()) {
    +      // wrong.
         }
    -    return true;
    -  }
     
    -  @SuppressWarnings("resource")
    -  private IterOutcome doWork() throws ClassTransformationException, IOException, SchemaChangeException {
    -    if (allocationVectors != null) {
    -      for (ValueVector v : allocationVectors) {
    -        v.clear();
    -      }
    +    if (newSchema) {
    +      createUnionAller(inputBatch);
         }
     
    -    allocationVectors = Lists.newArrayList();
    -    transfers.clear();
    +    container.zeroVectors();
     
    -    // If both sides of Union-All are empty
    -    if(unionAllInput.isBothSideEmpty()) {
    -      for(int i = 0; i < outputFields.size(); ++i) {
    -        final String colName = outputFields.get(i).getPath();
    -        final MajorType majorType = MajorType.newBuilder()
    -            .setMinorType(MinorType.INT)
    -            .setMode(DataMode.OPTIONAL)
    -            .build();
    -
    -        MaterializedField outputField = MaterializedField.create(colName, majorType);
    -        ValueVector vv = container.addOrGet(outputField, callBack);
    -        allocationVectors.add(vv);
    -      }
    +    for (final ValueVector v : this.allocationVectors) {
    +      AllocationHelper.allocateNew(v, inputBatch.getRecordCount());
    +    }
     
    -      container.buildSchema(BatchSchema.SelectionVectorMode.NONE);
    +    recordCount = unionall.unionRecords(0, inputBatch.getRecordCount(), 0);
    +    for (final ValueVector v : allocationVectors) {
    +      final ValueVector.Mutator m = v.getMutator();
    +      m.setValueCount(recordCount);
    +    }
    +
    +    if (callBack.getSchemaChangedAndReset()) {
           return IterOutcome.OK_NEW_SCHEMA;
    +    } else {
    +      return IterOutcome.OK;
         }
    +  }
    +
    +  private void createUnionAller(RecordBatch inputBatch) throws ClassTransformationException, IOException, SchemaChangeException {
    +    transfers.clear();
    +    allocationVectors.clear();;
     
         final ClassGenerator<UnionAller> cg = CodeGenerator.getRoot(UnionAller.TEMPLATE_DEFINITION, context.getFunctionRegistry(), context.getOptions());
         cg.getCodeGenerator().plainJavaCapable(true);
         // Uncomment out this line to debug the generated code.
    -//    cg.getCodeGenerator().saveCodeForDebugging(true);
    +    //    cg.getCodeGenerator().saveCodeForDebugging(true);
    +
         int index = 0;
    -    for(VectorWrapper<?> vw : current) {
    -       ValueVector vvIn = vw.getValueVector();
    -      // get the original input column names
    -      SchemaPath inputPath = SchemaPath.getSimplePath(vvIn.getField().getPath());
    -      // get the renamed column names
    -      SchemaPath outputPath = SchemaPath.getSimplePath(outputFields.get(index).getPath());
    +    for(VectorWrapper<?> vw : inputBatch) {
    +      ValueVector vvIn = vw.getValueVector();
    +      ValueVector vvOut = container.getValueVector(index).getValueVector();
     
           final ErrorCollector collector = new ErrorCollectorImpl();
           // According to input data names, Minortypes, Datamodes, choose to
           // transfer directly,
           // rename columns or
           // cast data types (Minortype or DataMode)
    -      if (hasSameTypeAndMode(outputFields.get(index), vw.getValueVector().getField())) {
    +      if (hasSameTypeAndMode(container.getSchema().getColumn(index), vvIn.getField())
    +          && vvIn.getField().getType().getMinorType() != TypeProtos.MinorType.MAP // Per DRILL-5521, existing bug for map transfer
    +          ) {
             // Transfer column
    -
    -        MajorType outputFieldType = outputFields.get(index).getType();
    -        MaterializedField outputField = MaterializedField.create(outputPath.getAsUnescapedPath(), outputFieldType);
    -
    -        /*
    -          todo: Fix if condition when DRILL-4824 is merged
    -          If condition should be changed to:
    -          `if (outputFields.get(index).getPath().equals(inputPath.getAsUnescapedPath())) {`
    -          DRILL-5419 has changed condition to correct one but this caused regression (DRILL-5521).
    -          Root cause is missing indication of child column in map types when it is null.
    -          DRILL-4824 is re-working json reader implementation, including map types and will fix this problem.
    -          Reverting condition to previous one to avoid regression till DRILL-4824 is merged.
    -          Unit test - TestJsonReader.testKvgenWithUnionAll().
    -         */
    -        if (outputFields.get(index).getPath().equals(inputPath)) {
    -          ValueVector vvOut = container.addOrGet(outputField);
    -          TransferPair tp = vvIn.makeTransferPair(vvOut);
    -          transfers.add(tp);
    +        TransferPair tp = vvIn.makeTransferPair(vvOut);
    +        transfers.add(tp);
             // Copy data in order to rename the column
    -        } else {
    -          final LogicalExpression expr = ExpressionTreeMaterializer.materialize(inputPath, current, collector, context.getFunctionRegistry() );
    -          if (collector.hasErrors()) {
    -            throw new SchemaChangeException(String.format("Failure while trying to materialize incoming schema.  Errors:\n %s.", collector.toErrorString()));
    -          }
    -
    -          ValueVector vv = container.addOrGet(outputField, callBack);
    -          allocationVectors.add(vv);
    -          TypedFieldId fid = container.getValueVectorId(SchemaPath.getSimplePath(outputField.getPath()));
    -          ValueVectorWriteExpression write = new ValueVectorWriteExpression(fid, expr, true);
    -          cg.addExpr(write);
    -        }
    -      // Cast is necessary
    +      } else if (vvIn.getField().getType().getMinorType() == TypeProtos.MinorType.NULL) {
    +        continue;
           } else {
    -        LogicalExpression expr = ExpressionTreeMaterializer.materialize(inputPath, current, collector, context.getFunctionRegistry());
    +        SchemaPath inputPath = SchemaPath.getSimplePath(vvIn.getField().getPath());
    +        MaterializedField inField = vvIn.getField();
    +        MaterializedField outputField = vvOut.getField();
    +
    +        LogicalExpression expr = ExpressionTreeMaterializer.materialize(inputPath, inputBatch, collector, context.getFunctionRegistry());
    +
             if (collector.hasErrors()) {
               throw new SchemaChangeException(String.format("Failure while trying to materialize incoming schema.  Errors:\n %s.", collector.toErrorString()));
             }
     
             // If the inputs' DataMode is required and the outputs' DataMode is not required
             // cast to the one with the least restriction
    -        if(vvIn.getField().getType().getMode() == DataMode.REQUIRED
    -            && outputFields.get(index).getType().getMode() != DataMode.REQUIRED) {
    -          expr = ExpressionTreeMaterializer.convertToNullableType(expr, vvIn.getField().getType().getMinorType(), context.getFunctionRegistry(), collector);
    +        if(inField.getType().getMode() == TypeProtos.DataMode.REQUIRED
    +            && outputField.getType().getMode() != TypeProtos.DataMode.REQUIRED) {
    +          expr = ExpressionTreeMaterializer.convertToNullableType(expr, inField.getType().getMinorType(), context.getFunctionRegistry(), collector);
               if (collector.hasErrors()) {
                 throw new SchemaChangeException(String.format("Failure while trying to materialize incoming schema.  Errors:\n %s.", collector.toErrorString()));
               }
             }
     
             // If two inputs' MinorTypes are different,
             // Insert a cast before the Union operation
    -        if(vvIn.getField().getType().getMinorType() != outputFields.get(index).getType().getMinorType()) {
    -          expr = ExpressionTreeMaterializer.addCastExpression(expr, outputFields.get(index).getType(), context.getFunctionRegistry(), collector);
    +        if(inField.getType().getMinorType() != outputField.getType().getMinorType()) {
    +          expr = ExpressionTreeMaterializer.addCastExpression(expr, outputField.getType(), context.getFunctionRegistry(), collector);
               if (collector.hasErrors()) {
                 throw new SchemaChangeException(String.format("Failure while trying to materialize incoming schema.  Errors:\n %s.", collector.toErrorString()));
               }
             }
     
    -        final MaterializedField outputField = MaterializedField.create(outputPath.getAsUnescapedPath(), expr.getMajorType());
    -        ValueVector vector = container.addOrGet(outputField, callBack);
    -        allocationVectors.add(vector);
             TypedFieldId fid = container.getValueVectorId(SchemaPath.getSimplePath(outputField.getPath()));
     
    -        boolean useSetSafe = !(vector instanceof FixedWidthVector);
    +        boolean useSetSafe = !(vvOut instanceof FixedWidthVector);
             ValueVectorWriteExpression write = new ValueVectorWriteExpression(fid, expr, useSetSafe);
             cg.addExpr(write);
    +
    +        allocationVectors.add(vvOut);
           }
           ++index;
         }
     
         unionall = context.getImplementationClass(cg.getCodeGenerator());
    -    unionall.setup(context, current, this, transfers);
    -
    -    if(!schemaAvailable) {
    -      container.buildSchema(BatchSchema.SelectionVectorMode.NONE);
    -      schemaAvailable = true;
    -    }
    -
    -    if(!doAlloc()) {
    -      return IterOutcome.OUT_OF_MEMORY;
    -    }
    -
    -    recordCount = unionall.unionRecords(0, current.getRecordCount(), 0);
    -    setValueCount(recordCount);
    -    return IterOutcome.OK;
    -  }
    -
    -  public static boolean hasSameTypeAndMode(MaterializedField leftField, MaterializedField rightField) {
    -    return (leftField.getType().getMinorType() == rightField.getType().getMinorType())
    -        && (leftField.getType().getMode() == rightField.getType().getMode());
    -  }
    -
    -  // This method is used by inner class to point the reference `current` to the correct record batch
    -  private void setCurrentRecordBatch(RecordBatch target) {
    -    this.current = target;
    -  }
    -
    -  // This method is used by inner class to clear the current record batch
    -  private void clearCurrentRecordBatch() {
    -    for(VectorWrapper<?> v: current) {
    -      v.clear();
    -    }
    +    unionall.setup(context, inputBatch, this, transfers);
       }
     
    -  public static class UnionAllInput {
    -    private UnionAllRecordBatch unionAllRecordBatch;
    -    private List<MaterializedField> outputFields;
    -    private OneSideInput leftSide;
    -    private OneSideInput rightSide;
    -    private IterOutcome upstream = IterOutcome.NOT_YET;
    -    private boolean leftIsFinish = false;
    -    private boolean rightIsFinish = false;
    -
    -    // These two schemas are obtained from the first record batches of the left and right inputs
    -    // They are used to check if the schema is changed between recordbatches
    -    private BatchSchema leftSchema;
    -    private BatchSchema rightSchema;
    -    private boolean bothEmpty = false;
    -
    -    public UnionAllInput(UnionAllRecordBatch unionAllRecordBatch, RecordBatch left, RecordBatch right) {
    -      this.unionAllRecordBatch = unionAllRecordBatch;
    -      leftSide = new OneSideInput(left);
    -      rightSide = new OneSideInput(right);
    -    }
    -
    -    private void setBothSideEmpty(boolean bothEmpty) {
    -      this.bothEmpty = bothEmpty;
    -    }
    -
    -    private boolean isBothSideEmpty() {
    -      return bothEmpty;
    -    }
    -
    -    public IterOutcome nextBatch() throws SchemaChangeException {
    -      if(upstream == RecordBatch.IterOutcome.NOT_YET) {
    -        IterOutcome iterLeft = leftSide.nextBatch();
    -        switch(iterLeft) {
    -          case OK_NEW_SCHEMA:
    -            /*
    -             * If the first few record batches are all empty,
    -             * there is no way to tell whether these empty batches are coming from empty files.
    -             * It is incorrect to infer output types when either side could be coming from empty.
    -             *
    -             * Thus, while-loop is necessary to skip those empty batches.
    -             */
    -            whileLoop:
    -            while(leftSide.getRecordBatch().getRecordCount() == 0) {
    -              iterLeft = leftSide.nextBatch();
    -
    -              switch(iterLeft) {
    -                case STOP:
    -                case OUT_OF_MEMORY:
    -                  return iterLeft;
    -
    -                case NONE:
    -                  // Special Case: The left side was an empty input.
    -                  leftIsFinish = true;
    -                  break whileLoop;
    -
    -                case NOT_YET:
    -                case OK_NEW_SCHEMA:
    -                case OK:
    -                  continue whileLoop;
    -
    -                default:
    -                  throw new IllegalStateException(
    -                      String.format("Unexpected state %s.", iterLeft));
    -              }
    -            }
    -
    -            break;
    -          case STOP:
    -          case OUT_OF_MEMORY:
    -            return iterLeft;
    -
    -          default:
    -            throw new IllegalStateException(
    -                String.format("Unexpected state %s.", iterLeft));
    -        }
    -
    -        IterOutcome iterRight = rightSide.nextBatch();
    -        switch(iterRight) {
    -          case OK_NEW_SCHEMA:
    -            // Unless there is no record batch on the left side of the inputs,
    -            // always start processing from the left side.
    -            if(leftIsFinish) {
    -              unionAllRecordBatch.setCurrentRecordBatch(rightSide.getRecordBatch());
    -            } else {
    -              unionAllRecordBatch.setCurrentRecordBatch(leftSide.getRecordBatch());
    -            }
    -            // If the record count of the first batch from right input is zero,
    -            // there are two possibilities:
    -            // 1. The right side is an empty input (e.g., file).
    -            // 2. There will be more records carried by later batches.
    -
    -            /*
    -             * If the first few record batches are all empty,
    -             * there is no way to tell whether these empty batches are coming from empty files.
    -             * It is incorrect to infer output types when either side could be coming from empty.
    -             *
    -             * Thus, while-loop is necessary to skip those empty batches.
    -             */
    -            whileLoop:
    -            while(rightSide.getRecordBatch().getRecordCount() == 0) {
    -              iterRight = rightSide.nextBatch();
    -              switch(iterRight) {
    -                case STOP:
    -                case OUT_OF_MEMORY:
    -                  return iterRight;
    -
    -                case NONE:
    -                  // Special Case: The right side was an empty input.
    -                  rightIsFinish = true;
    -                  break whileLoop;
    -
    -                case NOT_YET:
    -                case OK_NEW_SCHEMA:
    -                case OK:
    -                  continue whileLoop;
    -
    -                default:
    -                  throw new IllegalStateException(
    -                      String.format("Unexpected state %s.", iterRight));
    -              }
    -            }
    -
    -            if(leftIsFinish && rightIsFinish) {
    -              setBothSideEmpty(true);
    -            }
    -
    -            inferOutputFields();
    -            break;
    -
    -          case STOP:
    -          case OUT_OF_MEMORY:
    -            return iterRight;
    -
    -          default:
    -            throw new IllegalStateException(
    -                String.format("Unexpected state %s.", iterRight));
    -        }
    -
    -
    -
    -        upstream = IterOutcome.OK_NEW_SCHEMA;
    -        return upstream;
    -      } else {
    -        if(isBothSideEmpty()) {
    -          return IterOutcome.NONE;
    -        }
    -
    -        unionAllRecordBatch.clearCurrentRecordBatch();
    -
    -        if(leftIsFinish && rightIsFinish) {
    -          upstream = IterOutcome.NONE;
    -          return upstream;
    -        } else if(leftIsFinish) {
    -          IterOutcome iterOutcome = rightSide.nextBatch();
    -
    -          switch(iterOutcome) {
    -            case NONE:
    -              rightIsFinish = true;
    -              // fall through
    -            case STOP:
    -            case OUT_OF_MEMORY:
    -              upstream = iterOutcome;
    -              return upstream;
    -
    -            case OK_NEW_SCHEMA:
    -              if(!rightSide.getRecordBatch().getSchema().equals(rightSchema)) {
    -                throw new SchemaChangeException("Schema change detected in the right input of Union-All. This is not currently supported");
    -              }
    -              iterOutcome = IterOutcome.OK;
    -              // fall through
    -            case OK:
    -              unionAllRecordBatch.setCurrentRecordBatch(rightSide.getRecordBatch());
    -              upstream = iterOutcome;
    -              return upstream;
    -
    -            default:
    -              throw new IllegalStateException(String.format("Unknown state %s.", upstream));
    -          }
    -        } else if(rightIsFinish) {
    -          IterOutcome iterOutcome = leftSide.nextBatch();
    -          switch(iterOutcome) {
    -            case STOP:
    -            case OUT_OF_MEMORY:
    -            case NONE:
    -              upstream = iterOutcome;
    -              return upstream;
    -
    -            case OK:
    -              unionAllRecordBatch.setCurrentRecordBatch(leftSide.getRecordBatch());
    -              upstream = iterOutcome;
    -              return upstream;
    -
    -            default:
    -              throw new IllegalStateException(String.format("Unknown state %s.", iterOutcome));
    -          }
    -        } else {
    -          IterOutcome iterOutcome = leftSide.nextBatch();
    -
    -          switch(iterOutcome) {
    -            case STOP:
    -            case OUT_OF_MEMORY:
    -              upstream = iterOutcome;
    -              return upstream;
    -
    -            case OK_NEW_SCHEMA:
    -              if(!leftSide.getRecordBatch().getSchema().equals(leftSchema)) {
    -                throw new SchemaChangeException("Schema change detected in the left input of Union-All. This is not currently supported");
    -              }
    -
    -              iterOutcome = IterOutcome.OK;
    -              // fall through
    -            case OK:
    -              unionAllRecordBatch.setCurrentRecordBatch(leftSide.getRecordBatch());
    -              upstream = iterOutcome;
    -              return upstream;
    -
    -            case NONE:
    -              unionAllRecordBatch.setCurrentRecordBatch(rightSide.getRecordBatch());
    -              upstream = IterOutcome.OK;
    -              leftIsFinish = true;
    -              return upstream;
    -
    -            default:
    -              throw new IllegalStateException(String.format("Unknown state %s.", upstream));
    -          }
    -        }
    -      }
    -    }
     
    -    /**
    -     *
    -     * Summarize the inference in the four different situations:
    -     * First of all, the field names are always determined by the left side
    -     * (Even when the left side is from an empty file, we have the column names.)
    -     *
    -     * Cases:
    -     * 1. Left: non-empty; Right: non-empty
    -     *      types determined by both sides with implicit casting involved
    -     * 2. Left: empty; Right: non-empty
    -     *      type from the right
    -     * 3. Left: non-empty; Right: empty
    -     *      types from the left
    -     * 4. Left: empty; Right: empty
    -     *      types are nullable integer
    -     */
    -    private void inferOutputFields() {
    -      if(!leftIsFinish && !rightIsFinish) {
    -        // Both sides are non-empty
    -        inferOutputFieldsBothSide();
    -      } else if(!rightIsFinish) {
    -        // Left side is non-empty
    -        // While use left side's column names as output column names,
    -        // use right side's column types as output column types.
    -        inferOutputFieldsFromSingleSide(
    -            leftSide.getRecordBatch().getSchema(),
    -            rightSide.getRecordBatch().getSchema());
    +  // The output table's column names always follow the left table,
    +  // where the output type is chosen based on DRILL's implicit casting rules
    +  private void inferOutputFieldsBothSide(final BatchSchema leftSchema, final BatchSchema rightSchema) {
    +//    outputFields = Lists.newArrayList();
    +    final Iterator<MaterializedField> leftIter = leftSchema.iterator();
    +    final Iterator<MaterializedField> rightIter = rightSchema.iterator();
    +
    +    int index = 1;
    +    while (leftIter.hasNext() && rightIter.hasNext()) {
    +      MaterializedField leftField  = leftIter.next();
    +      MaterializedField rightField = rightIter.next();
    +
    +      if (hasSameTypeAndMode(leftField, rightField)) {
    --- End diff --
    
    If you are referring `createUnionAller` and `inferOutputFieldsBothSide`,  the majority of code actually are identical from the original one. AFASIK, the later one is to determine the output field name/type, while the first one is to generate code (may require implicit cast, depending on the future incoming batch's schema and output field type).  
    
    1. https://github.com/apache/drill/blob/master/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/union/UnionAllRecordBatch.java#L243-L244
    2. https://github.com/apache/drill/blob/master/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/union/UnionAllRecordBatch.java#L582-L585


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] drill pull request #906: DRILL-5546: Handle schema change exception failure ...

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

    https://github.com/apache/drill/pull/906#discussion_r135709915
  
    --- Diff: exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/ScanBatch.java ---
    @@ -252,14 +235,28 @@ public IterOutcome next() {
         }
       }
     
    +  private void releaseAssets() {
    +    container.zeroVectors();
    +  }
    +
    +  private void clearFieldVectorMap() {
    +    for (final ValueVector v : mutator.fieldVectorMap().values()) {
    +      v.clear();
    +    }
    +  }
    +
    +  private void advanceNextReader() throws ExecutionSetupException {
    +    currentReader = readers.next();
    +    implicitValues = implicitColumns.hasNext() ? implicitColumns.next() : null;
    +    currentReader.setup(oContext, mutator);
    --- End diff --
    
    Nice catch. You are right we should not advance to next reader if we need pass a batch to downstream, since `reader.setup()` for new reader could modify both mutator and container.  This seems to be true for original ScanBatch code as well. 
    
    In the revised batch,  I remove the code to get next reader in constructor, and put the code to get next reader in the beginning of loop when needed. In case of return a `OK_NEW_SCHEMA`, we will only get next reader in the next `next()` method call after `OK_NEW_SCHEMA` is returned.  
     


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] drill pull request #906: DRILL-5546: Handle schema change exception failure ...

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/906#discussion_r135578735
  
    --- Diff: exec/vector/src/main/java/org/apache/drill/exec/vector/UntypedNullHolder.java ---
    @@ -0,0 +1,50 @@
    +/*
    + * 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;
    +
    +import org.apache.drill.common.types.TypeProtos;
    +import org.apache.drill.common.types.Types;
    +import org.apache.drill.exec.expr.holders.ValueHolder;
    +
    +public class UntypedNullHolder implements ValueHolder {
    +  public static final TypeProtos.MajorType TYPE = Types.optional(TypeProtos.MinorType.NULL);
    +
    +  public TypeProtos.MajorType getType() {return TYPE;}
    --- End diff --
    
    The typical Java convention is to list fields first, then methods, so maybe move this after the fields.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] drill pull request #906: DRILL-5546: Handle schema change exception failure ...

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/906#discussion_r135370016
  
    --- Diff: exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/union/UnionAllRecordBatch.java ---
    @@ -130,562 +145,248 @@ public IterOutcome innerNext() {
       }
     
       @Override
    -  public WritableBatch getWritableBatch() {
    -    return WritableBatch.get(this);
    +  public int getRecordCount() {
    +    return recordCount;
       }
     
    -  private void setValueCount(int count) {
    -    for (ValueVector v : allocationVectors) {
    -      ValueVector.Mutator m = v.getMutator();
    -      m.setValueCount(count);
    -    }
    -  }
     
    -  private boolean doAlloc() {
    -    for (ValueVector v : allocationVectors) {
    -      try {
    -        AllocationHelper.allocateNew(v, current.getRecordCount());
    -      } catch (OutOfMemoryException ex) {
    -        return false;
    -      }
    +  @SuppressWarnings("resource")
    +  private IterOutcome doWork(RecordBatch inputBatch, boolean newSchema) throws ClassTransformationException, IOException, SchemaChangeException {
    +    if (inputBatch.getSchema().getFieldCount() != container.getSchema().getFieldCount()) {
    +      // wrong.
         }
    -    return true;
    -  }
     
    -  @SuppressWarnings("resource")
    -  private IterOutcome doWork() throws ClassTransformationException, IOException, SchemaChangeException {
    -    if (allocationVectors != null) {
    -      for (ValueVector v : allocationVectors) {
    -        v.clear();
    -      }
    +    if (newSchema) {
    +      createUnionAller(inputBatch);
         }
     
    -    allocationVectors = Lists.newArrayList();
    -    transfers.clear();
    +    container.zeroVectors();
     
    -    // If both sides of Union-All are empty
    -    if(unionAllInput.isBothSideEmpty()) {
    -      for(int i = 0; i < outputFields.size(); ++i) {
    -        final String colName = outputFields.get(i).getPath();
    -        final MajorType majorType = MajorType.newBuilder()
    -            .setMinorType(MinorType.INT)
    -            .setMode(DataMode.OPTIONAL)
    -            .build();
    -
    -        MaterializedField outputField = MaterializedField.create(colName, majorType);
    -        ValueVector vv = container.addOrGet(outputField, callBack);
    -        allocationVectors.add(vv);
    -      }
    +    for (final ValueVector v : this.allocationVectors) {
    +      AllocationHelper.allocateNew(v, inputBatch.getRecordCount());
    +    }
     
    -      container.buildSchema(BatchSchema.SelectionVectorMode.NONE);
    +    recordCount = unionall.unionRecords(0, inputBatch.getRecordCount(), 0);
    +    for (final ValueVector v : allocationVectors) {
    +      final ValueVector.Mutator m = v.getMutator();
    +      m.setValueCount(recordCount);
    +    }
    +
    +    if (callBack.getSchemaChangedAndReset()) {
           return IterOutcome.OK_NEW_SCHEMA;
    +    } else {
    +      return IterOutcome.OK;
         }
    +  }
    +
    +  private void createUnionAller(RecordBatch inputBatch) throws ClassTransformationException, IOException, SchemaChangeException {
    +    transfers.clear();
    +    allocationVectors.clear();;
     
         final ClassGenerator<UnionAller> cg = CodeGenerator.getRoot(UnionAller.TEMPLATE_DEFINITION, context.getFunctionRegistry(), context.getOptions());
         cg.getCodeGenerator().plainJavaCapable(true);
         // Uncomment out this line to debug the generated code.
    -//    cg.getCodeGenerator().saveCodeForDebugging(true);
    +    //    cg.getCodeGenerator().saveCodeForDebugging(true);
    +
         int index = 0;
    -    for(VectorWrapper<?> vw : current) {
    -       ValueVector vvIn = vw.getValueVector();
    -      // get the original input column names
    -      SchemaPath inputPath = SchemaPath.getSimplePath(vvIn.getField().getPath());
    -      // get the renamed column names
    -      SchemaPath outputPath = SchemaPath.getSimplePath(outputFields.get(index).getPath());
    +    for(VectorWrapper<?> vw : inputBatch) {
    +      ValueVector vvIn = vw.getValueVector();
    +      ValueVector vvOut = container.getValueVector(index).getValueVector();
     
           final ErrorCollector collector = new ErrorCollectorImpl();
           // According to input data names, Minortypes, Datamodes, choose to
           // transfer directly,
           // rename columns or
           // cast data types (Minortype or DataMode)
    -      if (hasSameTypeAndMode(outputFields.get(index), vw.getValueVector().getField())) {
    +      if (hasSameTypeAndMode(container.getSchema().getColumn(index), vvIn.getField())
    +          && vvIn.getField().getType().getMinorType() != TypeProtos.MinorType.MAP // Per DRILL-5521, existing bug for map transfer
    +          ) {
             // Transfer column
    -
    -        MajorType outputFieldType = outputFields.get(index).getType();
    -        MaterializedField outputField = MaterializedField.create(outputPath.getAsUnescapedPath(), outputFieldType);
    -
    -        /*
    -          todo: Fix if condition when DRILL-4824 is merged
    -          If condition should be changed to:
    -          `if (outputFields.get(index).getPath().equals(inputPath.getAsUnescapedPath())) {`
    -          DRILL-5419 has changed condition to correct one but this caused regression (DRILL-5521).
    -          Root cause is missing indication of child column in map types when it is null.
    -          DRILL-4824 is re-working json reader implementation, including map types and will fix this problem.
    -          Reverting condition to previous one to avoid regression till DRILL-4824 is merged.
    -          Unit test - TestJsonReader.testKvgenWithUnionAll().
    -         */
    -        if (outputFields.get(index).getPath().equals(inputPath)) {
    -          ValueVector vvOut = container.addOrGet(outputField);
    -          TransferPair tp = vvIn.makeTransferPair(vvOut);
    -          transfers.add(tp);
    +        TransferPair tp = vvIn.makeTransferPair(vvOut);
    +        transfers.add(tp);
             // Copy data in order to rename the column
    -        } else {
    -          final LogicalExpression expr = ExpressionTreeMaterializer.materialize(inputPath, current, collector, context.getFunctionRegistry() );
    -          if (collector.hasErrors()) {
    -            throw new SchemaChangeException(String.format("Failure while trying to materialize incoming schema.  Errors:\n %s.", collector.toErrorString()));
    -          }
    -
    -          ValueVector vv = container.addOrGet(outputField, callBack);
    -          allocationVectors.add(vv);
    -          TypedFieldId fid = container.getValueVectorId(SchemaPath.getSimplePath(outputField.getPath()));
    -          ValueVectorWriteExpression write = new ValueVectorWriteExpression(fid, expr, true);
    -          cg.addExpr(write);
    -        }
    -      // Cast is necessary
    +      } else if (vvIn.getField().getType().getMinorType() == TypeProtos.MinorType.NULL) {
    --- End diff --
    
    Why do we need to copy data to *rename* a column? The underlying buffer is identical regardless of name? Can't rename be done with a transfer? Or, is the original comment simply wrong?


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] drill pull request #906: DRILL-5546: Handle schema change exception failure ...

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/906#discussion_r136440163
  
    --- Diff: exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/union/UnionAllRecordBatch.java ---
    @@ -130,562 +145,248 @@ public IterOutcome innerNext() {
       }
     
       @Override
    -  public WritableBatch getWritableBatch() {
    -    return WritableBatch.get(this);
    +  public int getRecordCount() {
    +    return recordCount;
       }
     
    -  private void setValueCount(int count) {
    -    for (ValueVector v : allocationVectors) {
    -      ValueVector.Mutator m = v.getMutator();
    -      m.setValueCount(count);
    -    }
    -  }
     
    -  private boolean doAlloc() {
    -    for (ValueVector v : allocationVectors) {
    -      try {
    -        AllocationHelper.allocateNew(v, current.getRecordCount());
    -      } catch (OutOfMemoryException ex) {
    -        return false;
    -      }
    +  @SuppressWarnings("resource")
    +  private IterOutcome doWork(RecordBatch inputBatch, boolean newSchema) throws ClassTransformationException, IOException, SchemaChangeException {
    +    if (inputBatch.getSchema().getFieldCount() != container.getSchema().getFieldCount()) {
    +      // wrong.
         }
    -    return true;
    -  }
     
    -  @SuppressWarnings("resource")
    -  private IterOutcome doWork() throws ClassTransformationException, IOException, SchemaChangeException {
    -    if (allocationVectors != null) {
    -      for (ValueVector v : allocationVectors) {
    -        v.clear();
    -      }
    +    if (newSchema) {
    +      createUnionAller(inputBatch);
         }
     
    -    allocationVectors = Lists.newArrayList();
    -    transfers.clear();
    +    container.zeroVectors();
     
    -    // If both sides of Union-All are empty
    -    if(unionAllInput.isBothSideEmpty()) {
    -      for(int i = 0; i < outputFields.size(); ++i) {
    -        final String colName = outputFields.get(i).getPath();
    -        final MajorType majorType = MajorType.newBuilder()
    -            .setMinorType(MinorType.INT)
    -            .setMode(DataMode.OPTIONAL)
    -            .build();
    -
    -        MaterializedField outputField = MaterializedField.create(colName, majorType);
    -        ValueVector vv = container.addOrGet(outputField, callBack);
    -        allocationVectors.add(vv);
    -      }
    +    for (final ValueVector v : this.allocationVectors) {
    +      AllocationHelper.allocateNew(v, inputBatch.getRecordCount());
    +    }
     
    -      container.buildSchema(BatchSchema.SelectionVectorMode.NONE);
    +    recordCount = unionall.unionRecords(0, inputBatch.getRecordCount(), 0);
    +    for (final ValueVector v : allocationVectors) {
    +      final ValueVector.Mutator m = v.getMutator();
    +      m.setValueCount(recordCount);
    +    }
    +
    +    if (callBack.getSchemaChangedAndReset()) {
           return IterOutcome.OK_NEW_SCHEMA;
    +    } else {
    +      return IterOutcome.OK;
         }
    +  }
    +
    +  private void createUnionAller(RecordBatch inputBatch) throws ClassTransformationException, IOException, SchemaChangeException {
    +    transfers.clear();
    +    allocationVectors.clear();;
     
         final ClassGenerator<UnionAller> cg = CodeGenerator.getRoot(UnionAller.TEMPLATE_DEFINITION, context.getFunctionRegistry(), context.getOptions());
         cg.getCodeGenerator().plainJavaCapable(true);
         // Uncomment out this line to debug the generated code.
    -//    cg.getCodeGenerator().saveCodeForDebugging(true);
    +    //    cg.getCodeGenerator().saveCodeForDebugging(true);
    +
         int index = 0;
    -    for(VectorWrapper<?> vw : current) {
    -       ValueVector vvIn = vw.getValueVector();
    -      // get the original input column names
    -      SchemaPath inputPath = SchemaPath.getSimplePath(vvIn.getField().getPath());
    -      // get the renamed column names
    -      SchemaPath outputPath = SchemaPath.getSimplePath(outputFields.get(index).getPath());
    +    for(VectorWrapper<?> vw : inputBatch) {
    +      ValueVector vvIn = vw.getValueVector();
    +      ValueVector vvOut = container.getValueVector(index).getValueVector();
     
           final ErrorCollector collector = new ErrorCollectorImpl();
           // According to input data names, Minortypes, Datamodes, choose to
           // transfer directly,
           // rename columns or
           // cast data types (Minortype or DataMode)
    -      if (hasSameTypeAndMode(outputFields.get(index), vw.getValueVector().getField())) {
    +      if (hasSameTypeAndMode(container.getSchema().getColumn(index), vvIn.getField())
    +          && vvIn.getField().getType().getMinorType() != TypeProtos.MinorType.MAP // Per DRILL-5521, existing bug for map transfer
    +          ) {
             // Transfer column
    -
    -        MajorType outputFieldType = outputFields.get(index).getType();
    -        MaterializedField outputField = MaterializedField.create(outputPath.getAsUnescapedPath(), outputFieldType);
    -
    -        /*
    -          todo: Fix if condition when DRILL-4824 is merged
    -          If condition should be changed to:
    -          `if (outputFields.get(index).getPath().equals(inputPath.getAsUnescapedPath())) {`
    -          DRILL-5419 has changed condition to correct one but this caused regression (DRILL-5521).
    -          Root cause is missing indication of child column in map types when it is null.
    -          DRILL-4824 is re-working json reader implementation, including map types and will fix this problem.
    -          Reverting condition to previous one to avoid regression till DRILL-4824 is merged.
    -          Unit test - TestJsonReader.testKvgenWithUnionAll().
    -         */
    -        if (outputFields.get(index).getPath().equals(inputPath)) {
    -          ValueVector vvOut = container.addOrGet(outputField);
    -          TransferPair tp = vvIn.makeTransferPair(vvOut);
    -          transfers.add(tp);
    +        TransferPair tp = vvIn.makeTransferPair(vvOut);
    +        transfers.add(tp);
             // Copy data in order to rename the column
    -        } else {
    -          final LogicalExpression expr = ExpressionTreeMaterializer.materialize(inputPath, current, collector, context.getFunctionRegistry() );
    -          if (collector.hasErrors()) {
    -            throw new SchemaChangeException(String.format("Failure while trying to materialize incoming schema.  Errors:\n %s.", collector.toErrorString()));
    -          }
    -
    -          ValueVector vv = container.addOrGet(outputField, callBack);
    -          allocationVectors.add(vv);
    -          TypedFieldId fid = container.getValueVectorId(SchemaPath.getSimplePath(outputField.getPath()));
    -          ValueVectorWriteExpression write = new ValueVectorWriteExpression(fid, expr, true);
    -          cg.addExpr(write);
    -        }
    -      // Cast is necessary
    +      } else if (vvIn.getField().getType().getMinorType() == TypeProtos.MinorType.NULL) {
    +        continue;
           } else {
    -        LogicalExpression expr = ExpressionTreeMaterializer.materialize(inputPath, current, collector, context.getFunctionRegistry());
    +        SchemaPath inputPath = SchemaPath.getSimplePath(vvIn.getField().getPath());
    +        MaterializedField inField = vvIn.getField();
    +        MaterializedField outputField = vvOut.getField();
    +
    +        LogicalExpression expr = ExpressionTreeMaterializer.materialize(inputPath, inputBatch, collector, context.getFunctionRegistry());
    +
             if (collector.hasErrors()) {
               throw new SchemaChangeException(String.format("Failure while trying to materialize incoming schema.  Errors:\n %s.", collector.toErrorString()));
             }
     
             // If the inputs' DataMode is required and the outputs' DataMode is not required
             // cast to the one with the least restriction
    -        if(vvIn.getField().getType().getMode() == DataMode.REQUIRED
    -            && outputFields.get(index).getType().getMode() != DataMode.REQUIRED) {
    -          expr = ExpressionTreeMaterializer.convertToNullableType(expr, vvIn.getField().getType().getMinorType(), context.getFunctionRegistry(), collector);
    +        if(inField.getType().getMode() == TypeProtos.DataMode.REQUIRED
    --- End diff --
    
    Makes sense. Thanks for the explanation.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] drill pull request #906: DRILL-5546: Handle schema change exception failure ...

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

    https://github.com/apache/drill/pull/906#discussion_r136187804
  
    --- Diff: exec/vector/src/main/java/org/apache/drill/exec/vector/UntypedNullHolder.java ---
    @@ -0,0 +1,50 @@
    +/*
    + * 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;
    +
    +import org.apache.drill.common.types.TypeProtos;
    +import org.apache.drill.common.types.Types;
    +import org.apache.drill.exec.expr.holders.ValueHolder;
    +
    +public class UntypedNullHolder implements ValueHolder {
    +  public static final TypeProtos.MajorType TYPE = Types.optional(TypeProtos.MinorType.NULL);
    +
    +  public TypeProtos.MajorType getType() {return TYPE;}
    +
    +  public static final int WIDTH = 0;
    +
    +  public int isSet = 1;
    +
    +  @Deprecated
    +  public int hashCode(){
    +    throw new UnsupportedOperationException();
    +  }
    +
    +  /*
    +   * Reason for deprecation is that ValueHolders are potential scalar replacements
    +   * and hence we don't want any methods to be invoked on them.
    +   */
    +  @Deprecated
    +  public String toString(){
    +    throw new UnsupportedOperationException();
    +  }
    +
    +
    +
    --- End diff --
    
    done.



---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] drill pull request #906: DRILL-5546: Handle schema change exception failure ...

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/906#discussion_r134298357
  
    --- Diff: exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/ScanBatch.java ---
    @@ -152,97 +157,75 @@ public void kill(boolean sendUpstream) {
         }
       }
     
    -  private void releaseAssets() {
    -    container.zeroVectors();
    -  }
    -
    -  private void clearFieldVectorMap() {
    -    for (final ValueVector v : mutator.fieldVectorMap().values()) {
    -      v.clear();
    -    }
    -  }
    -
       @Override
       public IterOutcome next() {
         if (done) {
           return IterOutcome.NONE;
         }
         oContext.getStats().startProcessing();
         try {
    -      try {
    -        injector.injectChecked(context.getExecutionControls(), "next-allocate", OutOfMemoryException.class);
    -
    -        currentReader.allocate(mutator.fieldVectorMap());
    -      } catch (OutOfMemoryException e) {
    -        clearFieldVectorMap();
    -        throw UserException.memoryError(e).build(logger);
    -      }
    -      while ((recordCount = currentReader.next()) == 0) {
    +      while (true) {
             try {
    -          if (!readers.hasNext()) {
    -            // We're on the last reader, and it has no (more) rows.
    -            currentReader.close();
    -            releaseAssets();
    -            done = true;  // have any future call to next() return NONE
    -
    -            if (mutator.isNewSchema()) {
    -              // This last reader has a new schema (e.g., we have a zero-row
    -              // file or other source).  (Note that some sources have a non-
    -              // null/non-trivial schema even when there are no rows.)
    +          injector.injectChecked(context.getExecutionControls(), "next-allocate", OutOfMemoryException.class);
    +          currentReader.allocate(mutator.fieldVectorMap());
    +        } catch (OutOfMemoryException e) {
    +          clearFieldVectorMap();
    +          throw UserException.memoryError(e).build(logger);
    +        }
     
    -              container.buildSchema(SelectionVectorMode.NONE);
    -              schema = container.getSchema();
    +        recordCount = currentReader.next();
    +        Preconditions.checkArgument(recordCount >= 0,
    +            "recordCount from RecordReader.next() should not be negative");
     
    -              return IterOutcome.OK_NEW_SCHEMA;
    -            }
    -            return IterOutcome.NONE;
    -          }
    -          // At this point, the reader that hit its end is not the last reader.
    +        boolean isNewRegularSchema = mutator.isNewSchema();
    +        // We should skip the reader, when recordCount = 0 && ! isNewRegularSchema.
    +        // Add/set implicit column vectors, only when reader gets > 0 row, or
    +        // when reader gets 0 row but with a schema with new field added
    +        if (recordCount > 0 || isNewRegularSchema) {
    +          addImplicitVectors();
    +          populateImplicitVectors();
    +        }
     
    -          // If all the files we have read so far are just empty, the schema is not useful
    -          if (! hasReadNonEmptyFile) {
    -            container.clear();
    -            clearFieldVectorMap();
    -            mutator.clear();
    -          }
    +        boolean isNewImplicitSchema = mutator.isNewSchema();
    +        for (VectorWrapper<?> w : container) {
    +          w.getValueVector().getMutator().setValueCount(recordCount);
    +        }
    +        final boolean isNewSchema = isNewRegularSchema || isNewImplicitSchema;
    +        oContext.getStats().batchReceived(0, recordCount, isNewSchema);
     
    +        if (recordCount == 0) {
               currentReader.close();
    -          currentReader = readers.next();
    -          implicitValues = implicitColumns.hasNext() ? implicitColumns.next() : null;
    -          currentReader.setup(oContext, mutator);
    -          try {
    -            currentReader.allocate(mutator.fieldVectorMap());
    -          } catch (OutOfMemoryException e) {
    -            clearFieldVectorMap();
    -            throw UserException.memoryError(e).build(logger);
    +          if (isNewSchema) {
    +            // current reader presents a new schema in mutator even though it has 0 row.
    +            // This could happen when data sources have a non-trivial schema with 0 row.
    +            container.buildSchema(SelectionVectorMode.NONE);
    +            schema = container.getSchema();
    +            if (readers.hasNext()) {
    --- End diff --
    
    This code is pretty convoluted. Would be clearer to have an internal state such as NO_READER, ACTIVE_READER, EOF.
    
    Note that the revised scan batch, which will be used with the new size-aware mutator, has taken this approach. So, anything we do here is temporary. (Or, more realistic, will be used by community-provided readers that we chose not to convert to use the new scan version...)


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] drill pull request #906: DRILL-5546: Handle schema change exception failure ...

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/906#discussion_r135370205
  
    --- Diff: exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/union/UnionAllRecordBatch.java ---
    @@ -130,562 +145,248 @@ public IterOutcome innerNext() {
       }
     
       @Override
    -  public WritableBatch getWritableBatch() {
    -    return WritableBatch.get(this);
    +  public int getRecordCount() {
    +    return recordCount;
       }
     
    -  private void setValueCount(int count) {
    -    for (ValueVector v : allocationVectors) {
    -      ValueVector.Mutator m = v.getMutator();
    -      m.setValueCount(count);
    -    }
    -  }
     
    -  private boolean doAlloc() {
    -    for (ValueVector v : allocationVectors) {
    -      try {
    -        AllocationHelper.allocateNew(v, current.getRecordCount());
    -      } catch (OutOfMemoryException ex) {
    -        return false;
    -      }
    +  @SuppressWarnings("resource")
    +  private IterOutcome doWork(RecordBatch inputBatch, boolean newSchema) throws ClassTransformationException, IOException, SchemaChangeException {
    +    if (inputBatch.getSchema().getFieldCount() != container.getSchema().getFieldCount()) {
    +      // wrong.
         }
    -    return true;
    -  }
     
    -  @SuppressWarnings("resource")
    -  private IterOutcome doWork() throws ClassTransformationException, IOException, SchemaChangeException {
    -    if (allocationVectors != null) {
    -      for (ValueVector v : allocationVectors) {
    -        v.clear();
    -      }
    +    if (newSchema) {
    +      createUnionAller(inputBatch);
         }
     
    -    allocationVectors = Lists.newArrayList();
    -    transfers.clear();
    +    container.zeroVectors();
     
    -    // If both sides of Union-All are empty
    -    if(unionAllInput.isBothSideEmpty()) {
    -      for(int i = 0; i < outputFields.size(); ++i) {
    -        final String colName = outputFields.get(i).getPath();
    -        final MajorType majorType = MajorType.newBuilder()
    -            .setMinorType(MinorType.INT)
    -            .setMode(DataMode.OPTIONAL)
    -            .build();
    -
    -        MaterializedField outputField = MaterializedField.create(colName, majorType);
    -        ValueVector vv = container.addOrGet(outputField, callBack);
    -        allocationVectors.add(vv);
    -      }
    +    for (final ValueVector v : this.allocationVectors) {
    +      AllocationHelper.allocateNew(v, inputBatch.getRecordCount());
    +    }
     
    -      container.buildSchema(BatchSchema.SelectionVectorMode.NONE);
    +    recordCount = unionall.unionRecords(0, inputBatch.getRecordCount(), 0);
    +    for (final ValueVector v : allocationVectors) {
    +      final ValueVector.Mutator m = v.getMutator();
    +      m.setValueCount(recordCount);
    +    }
    +
    +    if (callBack.getSchemaChangedAndReset()) {
           return IterOutcome.OK_NEW_SCHEMA;
    +    } else {
    +      return IterOutcome.OK;
         }
    +  }
    +
    +  private void createUnionAller(RecordBatch inputBatch) throws ClassTransformationException, IOException, SchemaChangeException {
    +    transfers.clear();
    +    allocationVectors.clear();;
     
         final ClassGenerator<UnionAller> cg = CodeGenerator.getRoot(UnionAller.TEMPLATE_DEFINITION, context.getFunctionRegistry(), context.getOptions());
         cg.getCodeGenerator().plainJavaCapable(true);
         // Uncomment out this line to debug the generated code.
    -//    cg.getCodeGenerator().saveCodeForDebugging(true);
    +    //    cg.getCodeGenerator().saveCodeForDebugging(true);
    +
         int index = 0;
    -    for(VectorWrapper<?> vw : current) {
    -       ValueVector vvIn = vw.getValueVector();
    -      // get the original input column names
    -      SchemaPath inputPath = SchemaPath.getSimplePath(vvIn.getField().getPath());
    -      // get the renamed column names
    -      SchemaPath outputPath = SchemaPath.getSimplePath(outputFields.get(index).getPath());
    +    for(VectorWrapper<?> vw : inputBatch) {
    +      ValueVector vvIn = vw.getValueVector();
    +      ValueVector vvOut = container.getValueVector(index).getValueVector();
     
           final ErrorCollector collector = new ErrorCollectorImpl();
           // According to input data names, Minortypes, Datamodes, choose to
           // transfer directly,
           // rename columns or
           // cast data types (Minortype or DataMode)
    -      if (hasSameTypeAndMode(outputFields.get(index), vw.getValueVector().getField())) {
    +      if (hasSameTypeAndMode(container.getSchema().getColumn(index), vvIn.getField())
    +          && vvIn.getField().getType().getMinorType() != TypeProtos.MinorType.MAP // Per DRILL-5521, existing bug for map transfer
    +          ) {
             // Transfer column
    -
    -        MajorType outputFieldType = outputFields.get(index).getType();
    -        MaterializedField outputField = MaterializedField.create(outputPath.getAsUnescapedPath(), outputFieldType);
    -
    -        /*
    -          todo: Fix if condition when DRILL-4824 is merged
    -          If condition should be changed to:
    -          `if (outputFields.get(index).getPath().equals(inputPath.getAsUnescapedPath())) {`
    -          DRILL-5419 has changed condition to correct one but this caused regression (DRILL-5521).
    -          Root cause is missing indication of child column in map types when it is null.
    -          DRILL-4824 is re-working json reader implementation, including map types and will fix this problem.
    -          Reverting condition to previous one to avoid regression till DRILL-4824 is merged.
    -          Unit test - TestJsonReader.testKvgenWithUnionAll().
    -         */
    -        if (outputFields.get(index).getPath().equals(inputPath)) {
    -          ValueVector vvOut = container.addOrGet(outputField);
    -          TransferPair tp = vvIn.makeTransferPair(vvOut);
    -          transfers.add(tp);
    +        TransferPair tp = vvIn.makeTransferPair(vvOut);
    +        transfers.add(tp);
             // Copy data in order to rename the column
    -        } else {
    -          final LogicalExpression expr = ExpressionTreeMaterializer.materialize(inputPath, current, collector, context.getFunctionRegistry() );
    -          if (collector.hasErrors()) {
    -            throw new SchemaChangeException(String.format("Failure while trying to materialize incoming schema.  Errors:\n %s.", collector.toErrorString()));
    -          }
    -
    -          ValueVector vv = container.addOrGet(outputField, callBack);
    -          allocationVectors.add(vv);
    -          TypedFieldId fid = container.getValueVectorId(SchemaPath.getSimplePath(outputField.getPath()));
    -          ValueVectorWriteExpression write = new ValueVectorWriteExpression(fid, expr, true);
    -          cg.addExpr(write);
    -        }
    -      // Cast is necessary
    +      } else if (vvIn.getField().getType().getMinorType() == TypeProtos.MinorType.NULL) {
    +        continue;
           } else {
    -        LogicalExpression expr = ExpressionTreeMaterializer.materialize(inputPath, current, collector, context.getFunctionRegistry());
    +        SchemaPath inputPath = SchemaPath.getSimplePath(vvIn.getField().getPath());
    +        MaterializedField inField = vvIn.getField();
    +        MaterializedField outputField = vvOut.getField();
    +
    +        LogicalExpression expr = ExpressionTreeMaterializer.materialize(inputPath, inputBatch, collector, context.getFunctionRegistry());
    +
             if (collector.hasErrors()) {
               throw new SchemaChangeException(String.format("Failure while trying to materialize incoming schema.  Errors:\n %s.", collector.toErrorString()));
             }
     
             // If the inputs' DataMode is required and the outputs' DataMode is not required
             // cast to the one with the least restriction
    -        if(vvIn.getField().getType().getMode() == DataMode.REQUIRED
    -            && outputFields.get(index).getType().getMode() != DataMode.REQUIRED) {
    -          expr = ExpressionTreeMaterializer.convertToNullableType(expr, vvIn.getField().getType().getMinorType(), context.getFunctionRegistry(), collector);
    +        if(inField.getType().getMode() == TypeProtos.DataMode.REQUIRED
    --- End diff --
    
    See DRILL-5709, PR #901 for a faster way to do non-nullable to nullable conversions.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] drill pull request #906: DRILL-5546: Handle schema change exception failure ...

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/906#discussion_r135579027
  
    --- Diff: exec/vector/src/main/java/org/apache/drill/exec/vector/UntypedNullHolder.java ---
    @@ -0,0 +1,50 @@
    +/*
    + * 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;
    +
    +import org.apache.drill.common.types.TypeProtos;
    +import org.apache.drill.common.types.Types;
    +import org.apache.drill.exec.expr.holders.ValueHolder;
    +
    +public class UntypedNullHolder implements ValueHolder {
    +  public static final TypeProtos.MajorType TYPE = Types.optional(TypeProtos.MinorType.NULL);
    +
    +  public TypeProtos.MajorType getType() {return TYPE;}
    +
    +  public static final int WIDTH = 0;
    +
    +  public int isSet = 1;
    +
    +  @Deprecated
    +  public int hashCode(){
    +    throw new UnsupportedOperationException();
    +  }
    +
    +  /*
    +   * Reason for deprecation is that ValueHolders are potential scalar replacements
    +   * and hence we don't want any methods to be invoked on them.
    +   */
    +  @Deprecated
    +  public String toString(){
    +    throw new UnsupportedOperationException();
    +  }
    +
    +
    +
    --- End diff --
    
    Remove extra blank lines.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] drill pull request #906: DRILL-5546: Handle schema change exception failure ...

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/906#discussion_r135579157
  
    --- Diff: exec/vector/src/main/java/org/apache/drill/exec/vector/UntypedNullVector.java ---
    @@ -0,0 +1,267 @@
    +/*
    + * 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;
    +
    +
    +import com.google.common.base.Preconditions;
    +import io.netty.buffer.DrillBuf;
    +import org.apache.drill.exec.memory.BufferAllocator;
    +import org.apache.drill.exec.proto.UserBitShared.SerializedField;
    +import org.apache.drill.exec.record.MaterializedField;
    +import org.apache.drill.exec.record.TransferPair;
    +import org.apache.drill.exec.vector.complex.reader.FieldReader;
    +
    +/** UntypedNullVector is to represent a value vector with {@link org.apache.drill.common.types.MinorType.NULL}
    + *  All values in the vector represent two semantic implications: 1) the value is unknown, 2) the type is unknown.
    + *  Because of this, we only have to keep track of the number of values in value vector,
    + *  and there is no allocated buffer to back up this value vector. Therefore, the majority of
    + *  methods in this class is either no-op, or throws {@link UnsupportedOperationException}.
    + *
    + */
    +public final class UntypedNullVector extends BaseDataValueVector implements FixedWidthVector {
    +  private static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(UntypedNullVector.class);
    +
    +  /**
    +   * Width of each fixed-width value.
    +   */
    +  public static final int VALUE_WIDTH = 0;
    +
    +  private final Accessor accessor = new Accessor();
    +  private final Mutator mutator = new Mutator();
    +  private int valueCount ;
    --- End diff --
    
    Remove space before ;


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] drill pull request #906: DRILL-5546: Handle schema change exception failure ...

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

    https://github.com/apache/drill/pull/906#discussion_r135423794
  
    --- Diff: exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/ScanBatch.java ---
    @@ -152,97 +157,75 @@ public void kill(boolean sendUpstream) {
         }
       }
     
    -  private void releaseAssets() {
    -    container.zeroVectors();
    -  }
    -
    -  private void clearFieldVectorMap() {
    -    for (final ValueVector v : mutator.fieldVectorMap().values()) {
    -      v.clear();
    -    }
    -  }
    -
       @Override
       public IterOutcome next() {
         if (done) {
           return IterOutcome.NONE;
         }
         oContext.getStats().startProcessing();
         try {
    -      try {
    -        injector.injectChecked(context.getExecutionControls(), "next-allocate", OutOfMemoryException.class);
    -
    -        currentReader.allocate(mutator.fieldVectorMap());
    -      } catch (OutOfMemoryException e) {
    -        clearFieldVectorMap();
    -        throw UserException.memoryError(e).build(logger);
    -      }
    -      while ((recordCount = currentReader.next()) == 0) {
    +      while (true) {
             try {
    -          if (!readers.hasNext()) {
    -            // We're on the last reader, and it has no (more) rows.
    -            currentReader.close();
    -            releaseAssets();
    -            done = true;  // have any future call to next() return NONE
    -
    -            if (mutator.isNewSchema()) {
    -              // This last reader has a new schema (e.g., we have a zero-row
    -              // file or other source).  (Note that some sources have a non-
    -              // null/non-trivial schema even when there are no rows.)
    +          injector.injectChecked(context.getExecutionControls(), "next-allocate", OutOfMemoryException.class);
    +          currentReader.allocate(mutator.fieldVectorMap());
    +        } catch (OutOfMemoryException e) {
    +          clearFieldVectorMap();
    +          throw UserException.memoryError(e).build(logger);
    +        }
     
    -              container.buildSchema(SelectionVectorMode.NONE);
    -              schema = container.getSchema();
    +        recordCount = currentReader.next();
    +        Preconditions.checkArgument(recordCount >= 0,
    +            "recordCount from RecordReader.next() should not be negative");
     
    -              return IterOutcome.OK_NEW_SCHEMA;
    -            }
    -            return IterOutcome.NONE;
    -          }
    -          // At this point, the reader that hit its end is not the last reader.
    +        boolean isNewRegularSchema = mutator.isNewSchema();
    --- End diff --
    
    Since implicit schema is constant, we do not need to distinguish "regular" vs. "implicit".  The revised patch simply uses "isNewSchema".


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] drill pull request #906: DRILL-5546: Handle schema change exception failure ...

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/906#discussion_r135576938
  
    --- Diff: exec/java-exec/src/main/java/org/apache/drill/exec/record/AbstractBinaryRecordBatch.java ---
    @@ -0,0 +1,75 @@
    +/**
    + * 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.record;
    +
    +import org.apache.drill.exec.exception.OutOfMemoryException;
    +import org.apache.drill.exec.ops.FragmentContext;
    +import org.apache.drill.exec.physical.base.PhysicalOperator;
    +
    +public abstract class AbstractBinaryRecordBatch<T extends PhysicalOperator> extends  AbstractRecordBatch<T> {
    +  protected final RecordBatch left;
    +  protected final RecordBatch right;
    +
    +  // state (IterOutcome) of the left input
    +  protected IterOutcome leftUpstream = IterOutcome.NONE;
    +
    +  // state (IterOutcome) of the right input
    +  protected IterOutcome rightUpstream = IterOutcome.NONE;
    --- End diff --
    
    Nice improvement!
    
    Consider wrapping the two sides in a new class, say, BatchIterator, that holds the batch and iter outcome. Then, the code here is just a bit cleaner because we don't have to repeat variables for left and right.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] drill pull request #906: DRILL-5546: Handle schema change exception failure ...

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/906#discussion_r134297430
  
    --- Diff: exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/ScanBatch.java ---
    @@ -152,97 +157,75 @@ public void kill(boolean sendUpstream) {
         }
       }
     
    -  private void releaseAssets() {
    -    container.zeroVectors();
    -  }
    -
    -  private void clearFieldVectorMap() {
    -    for (final ValueVector v : mutator.fieldVectorMap().values()) {
    -      v.clear();
    -    }
    -  }
    -
       @Override
       public IterOutcome next() {
         if (done) {
           return IterOutcome.NONE;
         }
         oContext.getStats().startProcessing();
         try {
    -      try {
    -        injector.injectChecked(context.getExecutionControls(), "next-allocate", OutOfMemoryException.class);
    -
    -        currentReader.allocate(mutator.fieldVectorMap());
    -      } catch (OutOfMemoryException e) {
    -        clearFieldVectorMap();
    -        throw UserException.memoryError(e).build(logger);
    -      }
    -      while ((recordCount = currentReader.next()) == 0) {
    +      while (true) {
             try {
    -          if (!readers.hasNext()) {
    -            // We're on the last reader, and it has no (more) rows.
    -            currentReader.close();
    -            releaseAssets();
    -            done = true;  // have any future call to next() return NONE
    -
    -            if (mutator.isNewSchema()) {
    -              // This last reader has a new schema (e.g., we have a zero-row
    -              // file or other source).  (Note that some sources have a non-
    -              // null/non-trivial schema even when there are no rows.)
    +          injector.injectChecked(context.getExecutionControls(), "next-allocate", OutOfMemoryException.class);
    +          currentReader.allocate(mutator.fieldVectorMap());
    +        } catch (OutOfMemoryException e) {
    +          clearFieldVectorMap();
    +          throw UserException.memoryError(e).build(logger);
    +        }
     
    -              container.buildSchema(SelectionVectorMode.NONE);
    -              schema = container.getSchema();
    +        recordCount = currentReader.next();
    +        Preconditions.checkArgument(recordCount >= 0,
    +            "recordCount from RecordReader.next() should not be negative");
     
    -              return IterOutcome.OK_NEW_SCHEMA;
    -            }
    -            return IterOutcome.NONE;
    -          }
    -          // At this point, the reader that hit its end is not the last reader.
    +        boolean isNewRegularSchema = mutator.isNewSchema();
    +        // We should skip the reader, when recordCount = 0 && ! isNewRegularSchema.
    +        // Add/set implicit column vectors, only when reader gets > 0 row, or
    +        // when reader gets 0 row but with a schema with new field added
    +        if (recordCount > 0 || isNewRegularSchema) {
    +          addImplicitVectors();
    +          populateImplicitVectors();
    +        }
     
    -          // If all the files we have read so far are just empty, the schema is not useful
    -          if (! hasReadNonEmptyFile) {
    -            container.clear();
    -            clearFieldVectorMap();
    -            mutator.clear();
    -          }
    +        boolean isNewImplicitSchema = mutator.isNewSchema();
    --- End diff --
    
    The implicit schema will change only if the new file is in a different directory than the previous file. The implicit columns are fixed (`filename`, etc.) Only the `dir0`, `dir1` columns can change.
    
    Does Drill allow combining files from different directory levels into a single scan? If so, don't we have a trivial schema change problem? If the scan decides to scan `a/b/c.csv` before, say, `a/b/d/e.csv`, then we get a trivial schema change on the second file when we add the `dir2` column. Better analysis up front of the collection of paths will avoid this problem.
    
    If we avoid the `dirx` problem, then the implicit schema is constant for all readers (the values of the columns, of course, differs), so the `isNewImplicitSchema` logic can be dropped.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] drill pull request #906: DRILL-5546: Handle schema change exception failure ...

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

    https://github.com/apache/drill/pull/906#discussion_r135351043
  
    --- Diff: exec/java-exec/src/main/java/org/apache/drill/exec/physical/config/Project.java ---
    @@ -46,14 +56,18 @@ public Project(@JsonProperty("exprs") List<NamedExpression> exprs, @JsonProperty
         return exprs;
       }
     
    +  public boolean isOutputProj() {
    --- End diff --
    
    Done. 


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] drill pull request #906: DRILL-5546: Handle schema change exception failure ...

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/906#discussion_r134628709
  
    --- Diff: exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/project/ProjectRecordBatch.java ---
    @@ -768,4 +765,73 @@ else if (exprHasPrefix && refHasPrefix) {
           }
         }
       }
    +
    +  /**
    +   * handle FAST NONE specially when Project for query output. This happens when input returns a
    +   * FAST NONE directly ( input does not return any batch with schema/data).
    +   *
    +   * Project operator has to return a batch with schema derived using the following 3 rules:
    +   *  Case 1:  *  ==>  expand into an empty list of columns.
    +   *  Case 2:  regular column reference ==> treat as nullable-int column
    +   *  Case 3:  expressions => Call ExpressionTreeMaterialization over an empty vector contain.
    +   *           Once the expression is materialized without error, use the output type of materialized
    +   *           expression.
    +   * The batch is constructed with the above rules, and recordCount = 0.
    +   * Returned with OK_NEW_SCHEMA to down-stream operator.
    +   */
    +  @Override
    +  protected IterOutcome handleFastNone() {
    +    if (! popConfig.isOutputProj()) {
    +      return super.handleFastNone();
    +    }
    +
    +    allocationVectors = new ArrayList<>();
    +    final List<NamedExpression> exprs = getExpressionList();
    +    final ErrorCollector collector = new ErrorCollectorImpl();
    --- End diff --
    
    Maybe move this down to just above where it is used.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] drill issue #906: DRILL-5546: Handle schema change exception failure caused ...

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

    https://github.com/apache/drill/pull/906
  
    @paul-rogers , thank you for your detailed review comments. I revised the patch based on your comments, and rebased on latest master. Please see the 2nd commit for the revised patch. Let me know if you have further comments. 
     


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] drill pull request #906: DRILL-5546: Handle schema change exception failure ...

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

    https://github.com/apache/drill/pull/906#discussion_r135974916
  
    --- Diff: exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/project/ProjectRecordBatch.java ---
    @@ -768,4 +765,73 @@ else if (exprHasPrefix && refHasPrefix) {
           }
         }
       }
    +
    +  /**
    +   * handle FAST NONE specially when Project for query output. This happens when input returns a
    +   * FAST NONE directly ( input does not return any batch with schema/data).
    +   *
    +   * Project operator has to return a batch with schema derived using the following 3 rules:
    +   *  Case 1:  *  ==>  expand into an empty list of columns.
    +   *  Case 2:  regular column reference ==> treat as nullable-int column
    +   *  Case 3:  expressions => Call ExpressionTreeMaterialization over an empty vector contain.
    +   *           Once the expression is materialized without error, use the output type of materialized
    +   *           expression.
    +   * The batch is constructed with the above rules, and recordCount = 0.
    +   * Returned with OK_NEW_SCHEMA to down-stream operator.
    +   */
    +  @Override
    +  protected IterOutcome handleFastNone() {
    +    if (! popConfig.isOutputProj()) {
    +      return super.handleFastNone();
    +    }
    +
    +    allocationVectors = new ArrayList<>();
    +    final List<NamedExpression> exprs = getExpressionList();
    +    final ErrorCollector collector = new ErrorCollectorImpl();
    +    VectorContainer fakedIncomingVC = new VectorContainer();
    +
    +    for (NamedExpression namedExpression : exprs) {
    +      if (namedExpression.getExpr() instanceof SchemaPath) {
    +        final NameSegment expr = ((SchemaPath) namedExpression.getExpr()).getRootSegment();
    +        if (expr.getPath().contains(StarColumnHelper.STAR_COLUMN)) {
    +          continue; // * would expand into an empty list.
    +        } else {
    +          final TypeProtos.MajorType majorType = TypeProtos.MajorType.newBuilder()
    +              .setMinorType(MinorType.INT)
    +              .setMode(TypeProtos.DataMode.OPTIONAL)
    +              .build();
    +
    +          MaterializedField outputField = MaterializedField.create(namedExpression.getRef().getRootSegment().getPath(), majorType);
    +          final ValueVector vv = container.addOrGet(outputField, callBack);
    --- End diff --
    
    This code has been removed in revised patch.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] drill pull request #906: DRILL-5546: Handle schema change exception failure ...

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

    https://github.com/apache/drill/pull/906#discussion_r136157330
  
    --- Diff: exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/union/UnionAllRecordBatch.java ---
    @@ -130,562 +145,248 @@ public IterOutcome innerNext() {
       }
     
       @Override
    -  public WritableBatch getWritableBatch() {
    -    return WritableBatch.get(this);
    +  public int getRecordCount() {
    +    return recordCount;
       }
     
    -  private void setValueCount(int count) {
    -    for (ValueVector v : allocationVectors) {
    -      ValueVector.Mutator m = v.getMutator();
    -      m.setValueCount(count);
    -    }
    -  }
     
    -  private boolean doAlloc() {
    -    for (ValueVector v : allocationVectors) {
    -      try {
    -        AllocationHelper.allocateNew(v, current.getRecordCount());
    -      } catch (OutOfMemoryException ex) {
    -        return false;
    -      }
    +  @SuppressWarnings("resource")
    +  private IterOutcome doWork(RecordBatch inputBatch, boolean newSchema) throws ClassTransformationException, IOException, SchemaChangeException {
    +    if (inputBatch.getSchema().getFieldCount() != container.getSchema().getFieldCount()) {
    +      // wrong.
         }
    -    return true;
    -  }
     
    -  @SuppressWarnings("resource")
    -  private IterOutcome doWork() throws ClassTransformationException, IOException, SchemaChangeException {
    -    if (allocationVectors != null) {
    -      for (ValueVector v : allocationVectors) {
    -        v.clear();
    -      }
    +    if (newSchema) {
    +      createUnionAller(inputBatch);
         }
     
    -    allocationVectors = Lists.newArrayList();
    -    transfers.clear();
    +    container.zeroVectors();
     
    -    // If both sides of Union-All are empty
    -    if(unionAllInput.isBothSideEmpty()) {
    -      for(int i = 0; i < outputFields.size(); ++i) {
    -        final String colName = outputFields.get(i).getPath();
    -        final MajorType majorType = MajorType.newBuilder()
    -            .setMinorType(MinorType.INT)
    -            .setMode(DataMode.OPTIONAL)
    -            .build();
    -
    -        MaterializedField outputField = MaterializedField.create(colName, majorType);
    -        ValueVector vv = container.addOrGet(outputField, callBack);
    -        allocationVectors.add(vv);
    -      }
    +    for (final ValueVector v : this.allocationVectors) {
    +      AllocationHelper.allocateNew(v, inputBatch.getRecordCount());
    +    }
     
    -      container.buildSchema(BatchSchema.SelectionVectorMode.NONE);
    +    recordCount = unionall.unionRecords(0, inputBatch.getRecordCount(), 0);
    +    for (final ValueVector v : allocationVectors) {
    +      final ValueVector.Mutator m = v.getMutator();
    +      m.setValueCount(recordCount);
    +    }
    +
    +    if (callBack.getSchemaChangedAndReset()) {
           return IterOutcome.OK_NEW_SCHEMA;
    +    } else {
    +      return IterOutcome.OK;
         }
    +  }
    +
    +  private void createUnionAller(RecordBatch inputBatch) throws ClassTransformationException, IOException, SchemaChangeException {
    +    transfers.clear();
    +    allocationVectors.clear();;
    --- End diff --
    
    I thought the more `;` , the better.  :-)
    
    Removed. 


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] drill pull request #906: DRILL-5546: Handle schema change exception failure ...

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/906#discussion_r134627805
  
    --- Diff: exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/project/ProjectRecordBatch.java ---
    @@ -768,4 +765,73 @@ else if (exprHasPrefix && refHasPrefix) {
           }
         }
       }
    +
    +  /**
    +   * handle FAST NONE specially when Project for query output. This happens when input returns a
    +   * FAST NONE directly ( input does not return any batch with schema/data).
    +   *
    +   * Project operator has to return a batch with schema derived using the following 3 rules:
    +   *  Case 1:  *  ==>  expand into an empty list of columns.
    +   *  Case 2:  regular column reference ==> treat as nullable-int column
    +   *  Case 3:  expressions => Call ExpressionTreeMaterialization over an empty vector contain.
    --- End diff --
    
    Is this description confusing two different scenarios?
    
    1. Empty result set, but a schema is provided. (The Scan Batch changes go out of their way to provide a schema when possible.)
    2. Null result set: no rows and no schema.
    
    The rules in the Javadoc seem to relate to the second case: there are no columns to project.
    
    But, what do we do in the first case (when we have a schema, but no rows?) We should do exactly what we'd do if we had data: matching up columns, inserting nullable ints for missing columns, etc.
    
    Now, visualize the null result set as the same as an empty result set with no schema. *Exactly the same* rules apply. We match up columns (for wildcard or a project list), but will find none. So, we'll replace all reference with a nullable int.
    
    The point is, there should be only one code path; not two, and the one code path should gracefully handle the case in which the schema is empty.
    
    That said, it is likely true that debugging the existing code path may be tedious, and it may be faster to create a new code path. I wonder what that does for ongoing maintenance costs, however, as future developers have to not only understand the original path, but now must maintain the parallel "fast none" path.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] drill pull request #906: DRILL-5546: Handle schema change exception failure ...

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/906#discussion_r134287696
  
    --- Diff: exec/java-exec/src/main/java/org/apache/drill/exec/physical/config/Project.java ---
    @@ -35,9 +35,19 @@
       static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(Project.class);
     
       private final List<NamedExpression> exprs;
    +  /**
    +   * {@link org.apache.drill.exec.planner.physical.ProjectPrel for the meaning of flag 'outputProj'}
    +   */
    +  private boolean outputProj = false;
     
       @JsonCreator
    -  public Project(@JsonProperty("exprs") List<NamedExpression> exprs, @JsonProperty("child") PhysicalOperator child) {
    +  public Project(@JsonProperty("exprs") List<NamedExpression> exprs, @JsonProperty("child") PhysicalOperator child, @JsonProperty("outputproj") boolean outputProj) {
    --- End diff --
    
    `outputProj`?


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] drill pull request #906: DRILL-5546: Handle schema change exception failure ...

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/906#discussion_r135577279
  
    --- Diff: exec/java-exec/src/main/java/org/apache/drill/exec/record/AbstractSingleRecordBatch.java ---
    @@ -61,6 +63,10 @@ public IterOutcome innerNext() {
         }
         switch (upstream) {
         case NONE:
    +      if (state == BatchState.FIRST) {
    +        return handleFastNone();
    --- End diff --
    
    Again, consider a more descriptive name, such as `handleEmptyInput()`. Otherwise, everyone has to remember that "fast none" == "empty input".


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---