You are viewing a plain text version of this content. The canonical link for it is here.
Posted to dev@drill.apache.org by gparai <gi...@git.apache.org> on 2017/01/25 23:06:41 UTC

[GitHub] drill pull request #729: Drill 1328 r4

GitHub user gparai opened a pull request:

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

    Drill 1328 r4

    

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

    $ git pull https://github.com/gparai/drill Drill-1328-r4

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

    https://github.com/apache/drill/pull/729.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 #729
    
----
commit 7ed459d340d99ba1e4a8df6b66465a272ce51f02
Author: Cliff Buchanan <cb...@maprtech.com>
Date:   2014-08-21T21:59:53Z

    DRILL-1328: Support table statistics
    
    PRE: Add "append" concept to directory write.
    
    * This is so stats can be stored in [table].stats.drill and be appended to be writing a new file into the directory.
    
    FUNCS: Statistics functions as UDFs:
    Currently using FieldReader to ensure consistent output type so that Unpivot doesn't get confused. All stats columns should be Nullable, so that stats functions can return NULL when N/A.
    * custom versions of "count" that always return BigInt
    * HyperLogLog based NDV that returns BigInt that works only on VarChars
    * HyperLogLog with binary output that only works on VarChars
    
    OPS: Updated protobufs for new ops
    
    OPS: Implemented StatisticsAggregate
    
    OPS: Implemented StatisticsUnpivot
    
    ANALYZE: AnalyzeTable functionality
    * JavaCC syntax more-or-less copied from LucidDB.
    * (Basic) AnalyzePrule: DrillAnalyzeRel -> UnpivotPrel and StatsAggPrel
    
    ANALYZE: Add getMetadataTable() to AbstractSchema
    
    USAGE: Change field access in QueryWrapper
    
    USAGE: Add getDrillTable() to DrillScanRelBase and ScanPrel
    * since ScanPrel does not inherit from DrillScanRelBase, this requires adding a DrillTable to the constructor
    * This is done so that a custom ReflectiveRelMetadataProvider can access the DrillTable associated with Logical/Physical scans.
    
    USAGE: Attach DrillTableMetadata to DrillTable.
    * DrillTableMetadata represents the data scanned from a corresponding ".stats.drill" table
    * In order to avoid doing query execution right after the ".stats.drill" table is found, metadata is not actually collected until the MaterializationVisitor is used.
    ** Currently, the metadata source must be a string (so that a SQL query can be created). Doing this with a table is probably more complicated.
    ** Query is set up to extract only the most recent statistics results for each column.
    
    USAGE: Configure DrillJoinRelBase to use NDV metadata when available.
    
    USAGE: attach metadata to table
    
    USAGE: implement optiq provider

commit 9771a732ad9d266937c5f5a263cca2e09ee6f4f6
Author: Gautam Parai <gp...@maprtech.com>
Date:   2014-08-21T21:59:53Z

    DRILL-1328: Support table statistics
    
    PRE: Add "append" concept to directory write.
    
    * This is so stats can be stored in [table].stats.drill and be appended to be writing a new file into the directory.
    
    FUNCS: Statistics functions as UDFs:
    Currently using FieldReader to ensure consistent output type so that Unpivot doesn't get confused. All stats columns should be Nullable, so that stats functions can return NULL when N/A.
    * custom versions of "count" that always return BigInt
    * HyperLogLog based NDV that returns BigInt that works only on VarChars
    * HyperLogLog with binary output that only works on VarChars
    
    OPS: Updated protobufs for new ops
    
    OPS: Implemented StatisticsAggregate
    
    OPS: Implemented StatisticsUnpivot
    
    ANALYZE: AnalyzeTable functionality
    * JavaCC syntax more-or-less copied from LucidDB.
    * (Basic) AnalyzePrule: DrillAnalyzeRel -> UnpivotPrel and StatsAggPrel
    
    ANALYZE: Add getMetadataTable() to AbstractSchema
    
    USAGE: Change field access in QueryWrapper
    
    USAGE: Add getDrillTable() to DrillScanRelBase and ScanPrel
    * since ScanPrel does not inherit from DrillScanRelBase, this requires adding a DrillTable to the constructor
    * This is done so that a custom ReflectiveRelMetadataProvider can access the DrillTable associated with Logical/Physical scans.
    
    USAGE: Attach DrillTableMetadata to DrillTable.
    * DrillTableMetadata represents the data scanned from a corresponding ".stats.drill" table
    * In order to avoid doing query execution right after the ".stats.drill" table is found, metadata is not actually collected until the MaterializationVisitor is used.
    ** Currently, the metadata source must be a string (so that a SQL query can be created). Doing this with a table is probably more complicated.
    ** Query is set up to extract only the most recent statistics results for each column.
    
    USAGE: Configure DrillJoinRelBase to use NDV metadata when available.
    
    USAGE: attach metadata to table
    
    USAGE: implement optiq provider

commit 2bd531a3885415496c0c5f5ea445aedbd7aad07c
Author: Gautam Parai <gp...@maprtech.com>
Date:   2016-11-24T03:30:18Z

    Parallel statistics computation

commit 84189392719325e25a87c65ac7fb46e38b503882
Author: Aman Sinha <as...@maprtech.com>
Date:   2016-11-30T01:53:52Z

    Fix distribution traits for 2 phase analyze.  Add statistics_merge operator to CoreOperatorType and fix enum values.

commit 32123eeeeb9649c37ba90bb06b9ecabd62323b67
Author: Gautam Parai <gp...@maprtech.com>
Date:   2016-12-02T21:23:34Z

    Add support for all stats

commit 218ab2d7c9ffe7263a9f458d0fa6025824b62e97
Author: Gautam Parai <gp...@maprtech.com>
Date:   2016-12-09T22:48:04Z

    Old costing without statistics and new costing with statistics

commit 2f4ad909326b9020d87cfa70169d8d403aaa005c
Author: Gautam Parai <gp...@maprtech.com>
Date:   2017-01-05T23:48:07Z

    Fix Calcite AnalyzeSimpleEquiJoin

commit a92ddacfd84b378eb375f207990a8dbebba705cd
Author: Gautam Parai <gp...@maprtech.com>
Date:   2017-01-06T21:40:23Z

    Fix NDV overestimate when groupKey relies on one side of the join

commit af94db7a4e3d91d276be0202406dbc1cd748345e
Author: Gautam Parai <gp...@maprtech.com>
Date:   2017-01-25T00:13:35Z

    Code cleanup

----


---
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 #729: Drill 1328: Support table statistics for Parquet

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

    https://github.com/apache/drill/pull/729#discussion_r100709733
  
    --- Diff: exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/statistics/StatisticsAggBatch.java ---
    @@ -0,0 +1,256 @@
    +/**
    + * 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.impl.statistics;
    +
    +import com.google.common.collect.Lists;
    +import com.sun.codemodel.JExpr;
    +import org.apache.drill.common.expression.ErrorCollector;
    +import org.apache.drill.common.expression.ErrorCollectorImpl;
    +import org.apache.drill.common.expression.FunctionCallFactory;
    +import org.apache.drill.common.expression.LogicalExpression;
    +import org.apache.drill.common.expression.SchemaPath;
    +import org.apache.drill.common.expression.ValueExpressions;
    +import org.apache.drill.exec.exception.ClassTransformationException;
    +import org.apache.drill.exec.exception.OutOfMemoryException;
    +import org.apache.drill.exec.exception.SchemaChangeException;
    +import org.apache.drill.exec.expr.ClassGenerator;
    +import org.apache.drill.exec.expr.CodeGenerator;
    +import org.apache.drill.exec.expr.ExpressionTreeMaterializer;
    +import org.apache.drill.exec.expr.TypeHelper;
    +import org.apache.drill.exec.expr.ValueVectorWriteExpression;
    +import org.apache.drill.exec.ops.FragmentContext;
    +import org.apache.drill.exec.physical.config.StatisticsAggregate;
    +import org.apache.drill.exec.physical.impl.aggregate.StreamingAggBatch;
    +import org.apache.drill.exec.physical.impl.aggregate.StreamingAggTemplate;
    +import org.apache.drill.exec.physical.impl.aggregate.StreamingAggregator;
    +import org.apache.drill.exec.planner.physical.StatsAggPrel.OperatorPhase;
    +import org.apache.drill.exec.record.BatchSchema.SelectionVectorMode;
    +import org.apache.drill.exec.record.MaterializedField;
    +import org.apache.drill.exec.record.RecordBatch;
    +import org.apache.drill.exec.record.TypedFieldId;
    +import org.apache.drill.exec.store.ImplicitColumnExplorer;
    +import org.apache.drill.exec.vector.ValueVector;
    +import org.apache.drill.exec.vector.complex.FieldIdUtil;
    +import org.apache.drill.exec.vector.complex.MapVector;
    +
    +import java.io.IOException;
    +import java.util.GregorianCalendar;
    +import java.util.List;
    +import java.util.TimeZone;
    +
    +/**
    + * TODO: This needs cleanup. Currently the key values are constants and we compare the constants
    + * for every record. Seems unnecessary.
    + *
    + * Example input and output:
    + * Schema of incoming batch: region_id (VARCHAR), sales_city (VARCHAR), cnt (BIGINT)
    + * Schema of output:
    + *    "schema" : BIGINT - Schema number. For each schema change this number is incremented.
    + *    "computed" : BIGINT - What time is it computed?
    + *    "columns"       : MAP - Column names
    + *       "region_id"  : VARCHAR
    + *       "sales_city" : VARCHAR
    + *       "cnt"        : VARCHAR
    + *    "statscount" : MAP
    + *       "region_id"  : BIGINT - statscount(region_id) - aggregation over all values of region_id
    + *                      in incoming batch
    + *       "sales_city" : BIGINT - statscount(sales_city)
    + *       "cnt"        : BIGINT - statscount(cnt)
    + *    "nonnullstatcount" : MAP
    + *       "region_id"  : BIGINT - nonnullstatcount(region_id)
    + *       "sales_city" : BIGINT - nonnullstatcount(sales_city)
    + *       "cnt"        : BIGINT - nonnullstatcount(cnt)
    + *   .... another map for next stats function ....
    + */
    +public class StatisticsAggBatch extends StreamingAggBatch {
    +  private List<String> functions;
    +  private int schema = 0;
    +
    +  public StatisticsAggBatch(StatisticsAggregate popConfig, RecordBatch incoming,
    +      FragmentContext context) throws OutOfMemoryException {
    +    super(popConfig, incoming, context);
    +    this.functions = popConfig.getFunctions();
    +  }
    +
    +  private void createKeyColumn(String name, LogicalExpression expr, List<LogicalExpression> keyExprs,
    +      List<TypedFieldId> keyOutputIds) throws SchemaChangeException {
    +    ErrorCollector collector = new ErrorCollectorImpl();
    +
    +    LogicalExpression mle = ExpressionTreeMaterializer.materialize(expr, incoming, collector,
    +        context.getFunctionRegistry());
    +
    +    MaterializedField outputField = MaterializedField.create(name, mle.getMajorType());
    +    ValueVector vector = TypeHelper.getNewVector(outputField, oContext.getAllocator());
    +
    +    keyExprs.add(mle);
    +    keyOutputIds.add(container.add(vector));
    +
    +    if (collector.hasErrors()) {
    +      throw new SchemaChangeException("Failure while materializing expression. "
    +          + collector.toErrorString());
    +    }
    +  }
    +
    +  private void createNestedKeyColumn(MapVector parent, String name, LogicalExpression expr,
    +      List<LogicalExpression> keyExprs, List<TypedFieldId> keyOutputIds)
    +          throws SchemaChangeException {
    +    ErrorCollector collector = new ErrorCollectorImpl();
    +
    +    LogicalExpression mle = ExpressionTreeMaterializer.materialize(expr, incoming, collector,
    +        context.getFunctionRegistry());
    +
    +    Class<? extends ValueVector> vvc =
    +        TypeHelper.getValueVectorClass(mle.getMajorType().getMinorType(),
    +            mle.getMajorType().getMode());
    +
    +    ValueVector vv = parent.addOrGet(name, mle.getMajorType(), vvc);
    +
    +    TypedFieldId pfid = container.getValueVectorId(SchemaPath.getSimplePath(parent.getField().getPath()));
    +    assert pfid.getFieldIds().length == 1;
    +    TypedFieldId.Builder builder = TypedFieldId.newBuilder();
    +    builder.addId(pfid.getFieldIds()[0]);
    +    TypedFieldId id =
    +        FieldIdUtil.getFieldIdIfMatches(parent, builder, true,
    +            SchemaPath.getSimplePath(vv.getField().getPath()).getRootSegment());
    +
    +    keyExprs.add(mle);
    +    keyOutputIds.add(id);
    +
    +    if (collector.hasErrors()) {
    +      throw new SchemaChangeException("Failure while materializing expression. "
    +          + collector.toErrorString());
    +    }
    +  }
    +
    +  private void addMapVector(String name, MapVector parent, LogicalExpression expr,
    +      List<LogicalExpression> valueExprs) throws SchemaChangeException {
    +    ErrorCollector collector = new ErrorCollectorImpl();
    +
    +    LogicalExpression mle = ExpressionTreeMaterializer.materialize(expr, incoming, collector,
    +        context.getFunctionRegistry());
    +
    +    Class<? extends ValueVector> vvc =
    +        TypeHelper.getValueVectorClass(mle.getMajorType().getMinorType(), mle.getMajorType().getMode());
    +    ValueVector vv = parent.addOrGet(name, mle.getMajorType(), vvc);
    +
    +    TypedFieldId pfid = container.getValueVectorId(SchemaPath.getSimplePath(parent.getField().getPath()));
    +    assert pfid.getFieldIds().length == 1;
    +    TypedFieldId.Builder builder = TypedFieldId.newBuilder();
    +    builder.addId(pfid.getFieldIds()[0]);
    +    TypedFieldId id = FieldIdUtil.getFieldIdIfMatches(parent, builder, true,
    +        SchemaPath.getSimplePath(vv.getField().getPath()).getRootSegment());
    +
    +    valueExprs.add(new ValueVectorWriteExpression(id, mle, true));
    +
    +    if (collector.hasErrors()) {
    +      throw new SchemaChangeException("Failure while materializing expression. "
    +          + collector.toErrorString());
    +    }
    +  }
    +
    +  private StreamingAggregator codegenAggregator(List<LogicalExpression> keyExprs,
    +      List<LogicalExpression> valueExprs, List<TypedFieldId> keyOutputIds)
    +          throws SchemaChangeException, ClassTransformationException, IOException {
    +    ClassGenerator<StreamingAggregator> cg =
    +        CodeGenerator.getRoot(StreamingAggTemplate.TEMPLATE_DEFINITION, context.getFunctionRegistry(),
    +            context.getOptions());
    +
    +    LogicalExpression[] keyExprsArray = new LogicalExpression[keyExprs.size()];
    +    LogicalExpression[] valueExprsArray = new LogicalExpression[valueExprs.size()];
    +    TypedFieldId[] keyOutputIdsArray = new TypedFieldId[keyOutputIds.size()];
    +
    +    keyExprs.toArray(keyExprsArray);
    +    valueExprs.toArray(valueExprsArray);
    +    keyOutputIds.toArray(keyOutputIdsArray);
    +
    +    setupIsSame(cg, keyExprsArray);
    +    setupIsSameApart(cg, keyExprsArray);
    +    addRecordValues(cg, valueExprsArray);
    +    outputRecordKeys(cg, keyOutputIdsArray, keyExprsArray);
    +    outputRecordKeysPrev(cg, keyOutputIdsArray, keyExprsArray);
    +
    +    cg.getBlock("resetValues")._return(JExpr.TRUE);
    +    getIndex(cg);
    +
    +    container.buildSchema(SelectionVectorMode.NONE);
    +    StreamingAggregator agg = context.getImplementationClass(cg);
    +    agg.setup(oContext, incoming, this);
    +    return agg;
    +  }
    +
    +  private boolean isImplicitFileColumn(MaterializedField mf) {
    +    return ImplicitColumnExplorer.initImplicitFileColumns(context.getOptions()).get(mf.getName()) != null;
    +  }
    +
    +  protected StreamingAggregator createAggregatorInternal()
    +      throws SchemaChangeException, ClassTransformationException, IOException {
    +    container.clear();
    +
    +    List<LogicalExpression> keyExprs = Lists.newArrayList();
    +    List<LogicalExpression> valueExprs = Lists.newArrayList();
    +    List<TypedFieldId> keyOutputIds = Lists.newArrayList();
    +    GregorianCalendar calendar = new GregorianCalendar(TimeZone.getTimeZone("UTC"));
    +    calendar.setTimeInMillis(System.currentTimeMillis());
    +
    +    if (this.getPopConfig() instanceof StatisticsAggregate
    +        && (((StatisticsAggregate) this.getPopConfig()).getPhase() == OperatorPhase.PHASE_1of1
    +            || ((StatisticsAggregate) this.getPopConfig()).getPhase() == OperatorPhase.PHASE_2of2)) {
    +      createKeyColumn("schema",
    +          ValueExpressions.getBigInt(schema++),
    +          keyExprs,
    +          keyOutputIds
    +      );
    +      createKeyColumn("computed",
    +          ValueExpressions.getDate(calendar),
    --- End diff --
    
    This code was refactored out (no longer present). We only do this in the merging phase (which is serial - on the foreman). Also, we did not do it prior to the refactoring. Using the phases we determined if the plan was serial or not and only did this computation for the stats aggregate for the serial plan. For the parallel case, we do it in the stats merge which is serial (done on the foreman)


---
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 #729: Drill 1328 r4

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/729#discussion_r99392582
  
    --- Diff: exec/java-exec/src/main/java/org/apache/drill/exec/ops/FragmentContext.java ---
    @@ -392,6 +402,11 @@ public String getQueryUserName() {
         return fragment.getCredentials().getUserName();
       }
     
    +  /**
    +   * @return ID {@link java.util.UUID} of the current query
    +   */
    +  public String getQueryId() { return QueryIdHelper.getQueryId(fragment.getHandle().getQueryId());}
    --- End diff --
    
    We use the term "query ID" for the numeric version of the query ID as shown by the `getHandle().getQueryId()` method. Suggestion: rename this to 'getQueryIdString`. And, if we do that, provide a version that returns the original query ID. That is:
    ```
    public QueryId getQueryId() { return fragment.getHandle().getQueryId(); }
    public String getQueryIdString() { return QueryIdHelper.getQueryId(getQueryId()); }
    ```


---
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 #729: Drill 1328: Support table statistics for Parquet

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/729#discussion_r102866270
  
    --- Diff: protocol/src/main/java/org/apache/drill/exec/proto/beans/QueryContextInformation.java ---
    @@ -51,6 +51,7 @@ public static QueryContextInformation getDefaultInstance()
         private int timeZone;
         private String defaultSchemaName;
         private String sessionId;
    +    private int hllAccuracy;
    --- End diff --
    
    This need not be set here. Instead, get it from the option manager available to each fragment. Looks like this class is for static info common to all queries, not for caching system options.


---
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 #729: Drill 1328: Support table statistics for Parquet

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/729#discussion_r102872916
  
    --- Diff: exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/statistics/AbstractMergedStatistic.java ---
    @@ -0,0 +1,52 @@
    +/*
    + * 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.impl.statistics;
    +
    +import org.apache.drill.exec.vector.ValueVector;
    +
    +public abstract class AbstractMergedStatistic extends Statistic implements MergedStatistic {
    --- End diff --
    
    In general, I like how this is working out. Having the core logic in these classes is much clearer. See below for a view Java-related refinements.


---
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 #729: Drill 1328 r4

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/729#discussion_r99391822
  
    --- Diff: exec/java-exec/src/main/java/org/apache/drill/exec/ops/ContextInformation.java ---
    @@ -63,4 +65,11 @@ public long getQueryStartTime() {
       public int getRootFragmentTimeZone() {
         return rootFragmentTimeZone;
       }
    +
    +  /**
    +   * @return HLL accuracy parameter
    +   */
    +  public int getHllMemoryLimit() {
    --- End diff --
    
    The comment says this is an accuracy parameter, but the method name says it is a memory limit. Perhaps explain how the two relate?


---
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 #729: Drill 1328: Support table statistics for Parquet

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

    https://github.com/apache/drill/pull/729#discussion_r100707677
  
    --- Diff: exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/statistics/StatisticsAggBatch.java ---
    @@ -0,0 +1,256 @@
    +/**
    + * 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.impl.statistics;
    +
    +import com.google.common.collect.Lists;
    +import com.sun.codemodel.JExpr;
    +import org.apache.drill.common.expression.ErrorCollector;
    +import org.apache.drill.common.expression.ErrorCollectorImpl;
    +import org.apache.drill.common.expression.FunctionCallFactory;
    +import org.apache.drill.common.expression.LogicalExpression;
    +import org.apache.drill.common.expression.SchemaPath;
    +import org.apache.drill.common.expression.ValueExpressions;
    +import org.apache.drill.exec.exception.ClassTransformationException;
    +import org.apache.drill.exec.exception.OutOfMemoryException;
    +import org.apache.drill.exec.exception.SchemaChangeException;
    +import org.apache.drill.exec.expr.ClassGenerator;
    +import org.apache.drill.exec.expr.CodeGenerator;
    +import org.apache.drill.exec.expr.ExpressionTreeMaterializer;
    +import org.apache.drill.exec.expr.TypeHelper;
    +import org.apache.drill.exec.expr.ValueVectorWriteExpression;
    +import org.apache.drill.exec.ops.FragmentContext;
    +import org.apache.drill.exec.physical.config.StatisticsAggregate;
    +import org.apache.drill.exec.physical.impl.aggregate.StreamingAggBatch;
    +import org.apache.drill.exec.physical.impl.aggregate.StreamingAggTemplate;
    +import org.apache.drill.exec.physical.impl.aggregate.StreamingAggregator;
    +import org.apache.drill.exec.planner.physical.StatsAggPrel.OperatorPhase;
    +import org.apache.drill.exec.record.BatchSchema.SelectionVectorMode;
    +import org.apache.drill.exec.record.MaterializedField;
    +import org.apache.drill.exec.record.RecordBatch;
    +import org.apache.drill.exec.record.TypedFieldId;
    +import org.apache.drill.exec.store.ImplicitColumnExplorer;
    +import org.apache.drill.exec.vector.ValueVector;
    +import org.apache.drill.exec.vector.complex.FieldIdUtil;
    +import org.apache.drill.exec.vector.complex.MapVector;
    +
    +import java.io.IOException;
    +import java.util.GregorianCalendar;
    +import java.util.List;
    +import java.util.TimeZone;
    +
    +/**
    + * TODO: This needs cleanup. Currently the key values are constants and we compare the constants
    + * for every record. Seems unnecessary.
    + *
    + * Example input and output:
    + * Schema of incoming batch: region_id (VARCHAR), sales_city (VARCHAR), cnt (BIGINT)
    + * Schema of output:
    + *    "schema" : BIGINT - Schema number. For each schema change this number is incremented.
    + *    "computed" : BIGINT - What time is it computed?
    + *    "columns"       : MAP - Column names
    + *       "region_id"  : VARCHAR
    + *       "sales_city" : VARCHAR
    + *       "cnt"        : VARCHAR
    + *    "statscount" : MAP
    + *       "region_id"  : BIGINT - statscount(region_id) - aggregation over all values of region_id
    + *                      in incoming batch
    + *       "sales_city" : BIGINT - statscount(sales_city)
    + *       "cnt"        : BIGINT - statscount(cnt)
    + *    "nonnullstatcount" : MAP
    + *       "region_id"  : BIGINT - nonnullstatcount(region_id)
    + *       "sales_city" : BIGINT - nonnullstatcount(sales_city)
    + *       "cnt"        : BIGINT - nonnullstatcount(cnt)
    + *   .... another map for next stats function ....
    + */
    +public class StatisticsAggBatch extends StreamingAggBatch {
    +  private List<String> functions;
    +  private int schema = 0;
    +
    +  public StatisticsAggBatch(StatisticsAggregate popConfig, RecordBatch incoming,
    +      FragmentContext context) throws OutOfMemoryException {
    +    super(popConfig, incoming, context);
    +    this.functions = popConfig.getFunctions();
    +  }
    +
    --- End diff --
    
    Added comments when declaring and using `functions`.


---
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 #729: Drill 1328: Support table statistics for Parquet

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/729#discussion_r100680661
  
    --- Diff: exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/statistics/StatisticsMergeBatch.java ---
    @@ -0,0 +1,532 @@
    +/**
    + * 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.impl.statistics;
    +
    +import com.google.common.collect.Lists;
    +import com.google.common.collect.Maps;
    +import org.apache.drill.common.expression.LogicalExpression;
    +import org.apache.drill.common.expression.ErrorCollector;
    +import org.apache.drill.common.expression.ErrorCollectorImpl;
    +import org.apache.drill.common.expression.FunctionCallFactory;
    +import org.apache.drill.common.expression.SchemaPath;
    +import org.apache.drill.common.expression.ValueExpressions;
    +import org.apache.drill.common.types.TypeProtos;
    +import org.apache.drill.common.types.TypeProtos.MajorType;
    +import org.apache.drill.exec.exception.OutOfMemoryException;
    +import org.apache.drill.exec.exception.SchemaChangeException;
    +import org.apache.drill.exec.expr.ExpressionTreeMaterializer;
    +import org.apache.drill.exec.expr.TypeHelper;
    +import org.apache.drill.exec.expr.holders.BigIntHolder;
    +import org.apache.drill.exec.expr.holders.NullableFloat8Holder;
    +import org.apache.drill.exec.expr.holders.ObjectHolder;
    +import org.apache.drill.exec.expr.holders.ValueHolder;
    +import org.apache.drill.exec.ops.FragmentContext;
    +import org.apache.drill.exec.physical.config.StatisticsMerge;
    +import org.apache.drill.exec.record.AbstractSingleRecordBatch;
    +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.VectorContainer;
    +import org.apache.drill.exec.record.VectorWrapper;
    +import org.apache.drill.exec.vector.ValueVector;
    +import org.apache.drill.exec.vector.DateVector;
    +import org.apache.drill.exec.vector.BigIntVector;
    +import org.apache.drill.exec.vector.NullableBigIntVector;
    +import org.apache.drill.exec.vector.NullableFloat8Vector;
    +import org.apache.drill.exec.vector.NullableVarBinaryVector;
    +import org.apache.drill.exec.vector.VarCharVector;
    +import org.apache.drill.exec.vector.complex.MapVector;
    +
    +import java.io.ByteArrayInputStream;
    +import java.io.DataInputStream;
    +import java.io.IOException;
    +import java.util.List;
    +import java.util.GregorianCalendar;
    +import java.util.HashMap;
    +import java.util.Map;
    +import java.util.TimeZone;
    +
    +import com.clearspring.analytics.stream.cardinality.HyperLogLog;
    +
    +public class StatisticsMergeBatch extends AbstractSingleRecordBatch<StatisticsMerge> {
    +  private static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(StatisticsMergeBatch.class);
    +  private Map<String, String> functions;
    +  private boolean first = true;
    +  private boolean finished = false;
    +  private int schema = 0;
    +  private int recordCount = 0;
    +  private List<String> keyList = null;
    +  private Map<MaterializedField, ValueVector> dataSrcVecMap = null;
    +  // Map of non-map fields to VV in the incoming schema
    +  private Map<MaterializedField, ValueVector> copySrcVecMap = null;
    +  private Map<String, Map<String, ValueHolder>> aggregationMap = null;
    +  public StatisticsMergeBatch(StatisticsMerge popConfig, RecordBatch incoming,
    +                              FragmentContext context) throws OutOfMemoryException {
    +    super(popConfig, context, incoming);
    +    this.functions = new HashMap<>();
    +    this.aggregationMap = new HashMap<>();
    +
    +    /*for (String key : popConfig.getFunctions()) {
    +      aggregationMap.put(key, new HashMap<String, ValueHolder>());
    +      if (key.equalsIgnoreCase("statcount") || key.equalsIgnoreCase("nonnullstatcount")) {
    +        functions.put(key, "sum");
    +      } else if (key.equalsIgnoreCase("hll")) {
    +        functions.put(key, "hll_merge");
    +      } else if (key.equalsIgnoreCase("sum_width")) {
    +        functions.put(key, "avg_width");
    +      }
    +    }*/
    +    for (String key : popConfig.getFunctions()) {
    +      if (key.equalsIgnoreCase("sum_width")) {
    +        functions.put(key, "avg_width");
    +      } else if (key.equalsIgnoreCase("hll")) {
    +        functions.put(key, "hll_merge");
    +      } else {
    +        functions.put(key, key);
    +      }
    +      aggregationMap.put(functions.get(key), new HashMap<String, ValueHolder>());
    +    }
    +  }
    +
    +  private void createKeyColumn(String name, LogicalExpression expr, Map<MaterializedField, ValueVector> parentMap)
    +      throws SchemaChangeException {
    +    ErrorCollector collector = new ErrorCollectorImpl();
    +
    +    LogicalExpression mle = ExpressionTreeMaterializer.materialize(expr, incoming, collector,
    +        context.getFunctionRegistry());
    +
    +    MaterializedField outputField = MaterializedField.create(name, mle.getMajorType());
    +    ValueVector vector = TypeHelper.getNewVector(outputField, oContext.getAllocator());
    +    container.add(vector);
    +
    +    if (collector.hasErrors()) {
    +      throw new SchemaChangeException("Failure while materializing expression. "
    +          + collector.toErrorString());
    +    }
    +    parentMap.put(outputField, vector);
    +  }
    +
    +  private ValueVector addMapVector(String name, MapVector parent, LogicalExpression expr) throws SchemaChangeException {
    +    ErrorCollector collector = new ErrorCollectorImpl();
    +
    +    LogicalExpression mle = ExpressionTreeMaterializer.materialize(expr, incoming, collector,
    +        context.getFunctionRegistry());
    +
    +    Class<? extends ValueVector> vvc =
    +        TypeHelper.getValueVectorClass(mle.getMajorType().getMinorType(), mle.getMajorType().getMode());
    +    ValueVector vector = parent.addOrGet(name, mle.getMajorType(), vvc);
    +
    +    if (collector.hasErrors()) {
    +      throw new SchemaChangeException("Failure while materializing expression. "
    +          + collector.toErrorString());
    +    }
    +    return vector;
    +  }
    +
    +  /**
    +   * Identify the list of fields within a map which are unpivoted as columns in output
    +   */
    +  private void buildKeyList() {
    +    List<String> lastMapKeyList = null;
    +    for (VectorWrapper<?> vw : incoming) {
    +      if (vw.getField().getType().getMinorType() != TypeProtos.MinorType.MAP) {
    +        continue;
    +      }
    +
    +      keyList = Lists.newArrayList();
    +
    +      for (ValueVector vv : vw.getValueVector()) {
    +        keyList.add(vv.getField().getLastName());
    +      }
    +
    +      if (lastMapKeyList == null) {
    +        lastMapKeyList = keyList;
    +      } else {
    +        if (keyList.size() != lastMapKeyList.size() || !lastMapKeyList.containsAll(keyList)) {
    +          throw new UnsupportedOperationException("Maps have different fields");
    +        }
    +      }
    +    }
    +  }
    +
    +  private void buildOutputContainer() throws SchemaChangeException {
    +    dataSrcVecMap = Maps.newHashMap();
    +    copySrcVecMap = Maps.newHashMap();
    +    MajorType mt = null;
    +
    +    ErrorCollector collector = new ErrorCollectorImpl();
    +    GregorianCalendar calendar = new GregorianCalendar(TimeZone.getTimeZone("UTC"));
    +
    +    calendar.setTimeInMillis(System.currentTimeMillis());
    +    createKeyColumn("schema", ValueExpressions.getBigInt(schema++), copySrcVecMap);
    +    createKeyColumn("computed", ValueExpressions.getDate(calendar), copySrcVecMap);
    +
    +    for (VectorWrapper<?> vw : incoming) {
    +      addVectorToOutgoingContainer(vw.getField().getLastName(), vw, collector);
    +      /*MaterializedField ds = vw.getField();
    +      String field = vw.getField().getLastName();
    +      // Input map vector
    +      MapVector mapVector = (MapVector) vw.getValueVector();
    +      assert mapVector.getPrimitiveVectors().size() > 0;
    +      // Proceed to create output map vector with same name e.g. statcount etc.
    +      mt = mapVector.getField().getType();
    +      MaterializedField mf = MaterializedField.create(functions.get(field), mt);
    +      assert !dataSrcVecMap.containsKey(mf);
    +      ValueVector vector = TypeHelper.getNewVector(mf, oContext.getAllocator());
    +      container.add(vector);
    +      MapVector outputMapVector = (MapVector) vector;
    +
    +      for (ValueVector vv : mapVector) {
    +        String fieldName = vv.getField().getLastName();
    +        if (!keyList.contains(fieldName)) {
    +          throw new UnsupportedOperationException("Unpivot data vector " +
    +              ds + " contains key " + fieldName + " not contained in key source!");
    +        }
    +        if (vv.getField().getType().getMinorType() == TypeProtos.MinorType.MAP) {
    +          throw new UnsupportedOperationException("Unpivot of nested map is not supported!");
    +        }
    +        if (field.equals("column")) {
    +          outputMapVector.addOrGet(fieldName, vv.getField().getType(), vv.getClass());
    +        } else {
    +          List<LogicalExpression> args = Lists.newArrayList();
    +          //TODO: Something else to access value of col such as emp_id?
    +          args.add(SchemaPath.getSimplePath(vv.getField().getPath()));
    +          //TODO: Put in the mapVector
    +          LogicalExpression call = FunctionCallFactory.createExpression(functions.get(field), args);
    +          //TODO: Is this sufficient to add to new Map?
    +          ValueVector vector1 = addMapVector(fieldName, outputMapVector, call);
    +          if (collector.hasErrors()) {
    +            throw new SchemaChangeException("Failure while materializing expression. "
    +                + collector.toErrorString());
    +          }
    +        }
    +      }
    +      dataSrcVecMap.put(ds, outputMapVector);*/
    +    }
    +    //Now create NDV in the outgoing container which was not avaliable in the incoming
    +    for (VectorWrapper<?> vw : incoming) {
    +      if (vw.getField().getLastName().equalsIgnoreCase("sum_width")) {//NullableFloat8 type
    +        addVectorToOutgoingContainer("ndv", vw, collector);
    +        break;
    +      }
    +    }
    +    container.setRecordCount(0);
    +    recordCount = 0;
    +    container.buildSchema(incoming.getSchema().getSelectionVectorMode());
    +  }
    +
    +  private void addVectorToOutgoingContainer(String field, VectorWrapper vw, ErrorCollector collector)
    +     throws SchemaChangeException {
    +    // Input map vector
    +    MapVector mapVector = (MapVector) vw.getValueVector();
    +    MaterializedField mf;
    +    assert mapVector.getPrimitiveVectors().size() > 0;
    +    // Proceed to create output map vector with same name e.g. statcount etc.
    +    MajorType mt = mapVector.getField().getType();
    +    if (functions.get(field) != null) {
    +      mf = MaterializedField.create(functions.get(field), mt);
    +    } else {
    +      mf = MaterializedField.create(field, mt);
    +    }
    +    assert !dataSrcVecMap.containsKey(mf);
    +    ValueVector vector = TypeHelper.getNewVector(mf, oContext.getAllocator());
    +    container.add(vector);
    +    MapVector outputMapVector = (MapVector) vector;
    +
    +    for (ValueVector vv : mapVector) {
    +      String fieldName = vv.getField().getLastName();
    +      if (!keyList.contains(fieldName)) {
    +        throw new UnsupportedOperationException("Unpivot data vector " +
    +                field + " contains key " + fieldName + " not contained in key source!");
    +      }
    +      if (vv.getField().getType().getMinorType() == TypeProtos.MinorType.MAP) {
    +        throw new UnsupportedOperationException("Unpivot of nested map is not supported!");
    +      }
    +      if (field.equals("column")) {
    +        outputMapVector.addOrGet(fieldName, vv.getField().getType(), vv.getClass());
    +      } else {
    +        List<LogicalExpression> args = Lists.newArrayList();
    +        LogicalExpression call;
    +        //TODO: Something else to access value of col such as emp_id?
    +        args.add(SchemaPath.getSimplePath(vv.getField().getPath()));
    +        //TODO: Put in the mapVector
    +        if (functions.get(field) != null) {
    +          call = FunctionCallFactory.createExpression(functions.get(field), args);
    +        } else {
    +          call = FunctionCallFactory.createExpression(field, args);
    +        }
    +        //TODO: Is this sufficient to add to new Map?
    +        ValueVector vector1 = addMapVector(fieldName, outputMapVector, call);
    +        if (collector.hasErrors()) {
    +          throw new SchemaChangeException("Failure while materializing expression. "
    +                  + collector.toErrorString());
    +        }
    +      }
    +    }
    +  }
    +
    +  @Override
    +  protected boolean setupNewSchema() throws SchemaChangeException {
    +    container.clear();
    +    buildKeyList();
    +    buildOutputContainer();
    +    return true;
    +  }
    +
    +  @Override
    +  protected IterOutcome doWork() {
    +    int outRecordCount = incoming.getRecordCount();
    +    HashMap<String, Long> nonNullRows = new HashMap<>();
    +
    +    for (VectorWrapper<?> vw : incoming) {
    +      ValueVector vv = vw.getValueVector();
    +      if (vv.getField().getType().getMinorType() != TypeProtos.MinorType.MAP) {
    +        //We only expect Maps in the incoming. STOP, if this is not the case.
    +        return IterOutcome.STOP;
    +      }
    +      MapVector mapVec = (MapVector) vv;
    +      Map<String, ValueHolder> statMap = aggregationMap.get(vv.getField().getLastName());
    +
    --- End diff --
    
    Boy, some explanation would sure be helpful while reviewing this 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 #729: Drill 1328: Support table statistics for Parquet

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/729#discussion_r102875132
  
    --- Diff: exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/statistics/MergedStatisticFactory.java ---
    @@ -0,0 +1,46 @@
    +/*
    + * 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.impl.statistics;
    +
    +public class MergedStatisticFactory {
    +  /*
    +   * Creates the appropriate statistics object given the name of the statistics and the input statistic
    +   */
    +  public static MergedStatistic getMergedStatistic(String outputStatName, String inputStatName) {
    +    if (outputStatName == null || inputStatName == null) {
    +      return null;
    +    } else if (outputStatName.equals(Statistic.COLNAME)) {
    +      return new ColumnMergedStatistic(outputStatName, inputStatName);
    +    } else if (outputStatName.equals(Statistic.COLTYPE)) {
    +      return new ColTypeMergedStatistic(outputStatName, inputStatName);
    +    } else if (outputStatName.equals(Statistic.STATCOUNT)) {
    +      return new StatCountMergedStatistic(outputStatName, inputStatName);
    +    } else if (outputStatName.equals(Statistic.NNSTATCOUNT)) {
    +      return new NNStatCountMergedStatistic(outputStatName, inputStatName);
    +    } else if (outputStatName.equals(Statistic.AVG_WIDTH)) {
    +      return new AvgWidthMergedStatistic(outputStatName, inputStatName);
    +    } else if (outputStatName.equals(Statistic.HLL_MERGE)) {
    +      return new HLLMergedStatistic(outputStatName, inputStatName);
    --- End diff --
    
    If the outputStatName determines the class, then we don't have to tell the class the name of its statistic, do we? The stat name can be hard-coded for each class to the matching constant...
    
    Otherwise, it raises the question whether a single stats implementation can be known by multiple names...
    
    Is there a 1:1 mapping from input stat to output stat? Then the input stat name can be hard-coded in each implementation also, right?
    ```
    public class HLLMergedStatistic extends AbstractMergedStatistic {
      ...
      @Override
      public String getName() { return Statistic.HLL_MERGE; }
    }


---
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 #729: Drill 1328: Support table statistics for Parquet

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/729#discussion_r100676949
  
    --- Diff: exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/statistics/StatisticsAggBatch.java ---
    @@ -0,0 +1,256 @@
    +/**
    + * 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.impl.statistics;
    +
    +import com.google.common.collect.Lists;
    +import com.sun.codemodel.JExpr;
    +import org.apache.drill.common.expression.ErrorCollector;
    +import org.apache.drill.common.expression.ErrorCollectorImpl;
    +import org.apache.drill.common.expression.FunctionCallFactory;
    +import org.apache.drill.common.expression.LogicalExpression;
    +import org.apache.drill.common.expression.SchemaPath;
    +import org.apache.drill.common.expression.ValueExpressions;
    +import org.apache.drill.exec.exception.ClassTransformationException;
    +import org.apache.drill.exec.exception.OutOfMemoryException;
    +import org.apache.drill.exec.exception.SchemaChangeException;
    +import org.apache.drill.exec.expr.ClassGenerator;
    +import org.apache.drill.exec.expr.CodeGenerator;
    +import org.apache.drill.exec.expr.ExpressionTreeMaterializer;
    +import org.apache.drill.exec.expr.TypeHelper;
    +import org.apache.drill.exec.expr.ValueVectorWriteExpression;
    +import org.apache.drill.exec.ops.FragmentContext;
    +import org.apache.drill.exec.physical.config.StatisticsAggregate;
    +import org.apache.drill.exec.physical.impl.aggregate.StreamingAggBatch;
    +import org.apache.drill.exec.physical.impl.aggregate.StreamingAggTemplate;
    +import org.apache.drill.exec.physical.impl.aggregate.StreamingAggregator;
    +import org.apache.drill.exec.planner.physical.StatsAggPrel.OperatorPhase;
    +import org.apache.drill.exec.record.BatchSchema.SelectionVectorMode;
    +import org.apache.drill.exec.record.MaterializedField;
    +import org.apache.drill.exec.record.RecordBatch;
    +import org.apache.drill.exec.record.TypedFieldId;
    +import org.apache.drill.exec.store.ImplicitColumnExplorer;
    +import org.apache.drill.exec.vector.ValueVector;
    +import org.apache.drill.exec.vector.complex.FieldIdUtil;
    +import org.apache.drill.exec.vector.complex.MapVector;
    +
    +import java.io.IOException;
    +import java.util.GregorianCalendar;
    +import java.util.List;
    +import java.util.TimeZone;
    +
    +/**
    + * TODO: This needs cleanup. Currently the key values are constants and we compare the constants
    + * for every record. Seems unnecessary.
    + *
    + * Example input and output:
    + * Schema of incoming batch: region_id (VARCHAR), sales_city (VARCHAR), cnt (BIGINT)
    + * Schema of output:
    + *    "schema" : BIGINT - Schema number. For each schema change this number is incremented.
    + *    "computed" : BIGINT - What time is it computed?
    --- End diff --
    
    Time format? Unix timetamp?


---
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 #729: Drill 1328: Support table statistics for Parquet

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/729#discussion_r100681044
  
    --- Diff: exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/statistics/StatisticsMergeBatch.java ---
    @@ -0,0 +1,532 @@
    +/**
    + * 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.impl.statistics;
    +
    +import com.google.common.collect.Lists;
    +import com.google.common.collect.Maps;
    +import org.apache.drill.common.expression.LogicalExpression;
    +import org.apache.drill.common.expression.ErrorCollector;
    +import org.apache.drill.common.expression.ErrorCollectorImpl;
    +import org.apache.drill.common.expression.FunctionCallFactory;
    +import org.apache.drill.common.expression.SchemaPath;
    +import org.apache.drill.common.expression.ValueExpressions;
    +import org.apache.drill.common.types.TypeProtos;
    +import org.apache.drill.common.types.TypeProtos.MajorType;
    +import org.apache.drill.exec.exception.OutOfMemoryException;
    +import org.apache.drill.exec.exception.SchemaChangeException;
    +import org.apache.drill.exec.expr.ExpressionTreeMaterializer;
    +import org.apache.drill.exec.expr.TypeHelper;
    +import org.apache.drill.exec.expr.holders.BigIntHolder;
    +import org.apache.drill.exec.expr.holders.NullableFloat8Holder;
    +import org.apache.drill.exec.expr.holders.ObjectHolder;
    +import org.apache.drill.exec.expr.holders.ValueHolder;
    +import org.apache.drill.exec.ops.FragmentContext;
    +import org.apache.drill.exec.physical.config.StatisticsMerge;
    +import org.apache.drill.exec.record.AbstractSingleRecordBatch;
    +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.VectorContainer;
    +import org.apache.drill.exec.record.VectorWrapper;
    +import org.apache.drill.exec.vector.ValueVector;
    +import org.apache.drill.exec.vector.DateVector;
    +import org.apache.drill.exec.vector.BigIntVector;
    +import org.apache.drill.exec.vector.NullableBigIntVector;
    +import org.apache.drill.exec.vector.NullableFloat8Vector;
    +import org.apache.drill.exec.vector.NullableVarBinaryVector;
    +import org.apache.drill.exec.vector.VarCharVector;
    +import org.apache.drill.exec.vector.complex.MapVector;
    +
    +import java.io.ByteArrayInputStream;
    +import java.io.DataInputStream;
    +import java.io.IOException;
    +import java.util.List;
    +import java.util.GregorianCalendar;
    +import java.util.HashMap;
    +import java.util.Map;
    +import java.util.TimeZone;
    +
    +import com.clearspring.analytics.stream.cardinality.HyperLogLog;
    +
    +public class StatisticsMergeBatch extends AbstractSingleRecordBatch<StatisticsMerge> {
    +  private static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(StatisticsMergeBatch.class);
    +  private Map<String, String> functions;
    +  private boolean first = true;
    +  private boolean finished = false;
    +  private int schema = 0;
    +  private int recordCount = 0;
    +  private List<String> keyList = null;
    +  private Map<MaterializedField, ValueVector> dataSrcVecMap = null;
    +  // Map of non-map fields to VV in the incoming schema
    +  private Map<MaterializedField, ValueVector> copySrcVecMap = null;
    +  private Map<String, Map<String, ValueHolder>> aggregationMap = null;
    +  public StatisticsMergeBatch(StatisticsMerge popConfig, RecordBatch incoming,
    +                              FragmentContext context) throws OutOfMemoryException {
    +    super(popConfig, context, incoming);
    +    this.functions = new HashMap<>();
    +    this.aggregationMap = new HashMap<>();
    +
    +    /*for (String key : popConfig.getFunctions()) {
    +      aggregationMap.put(key, new HashMap<String, ValueHolder>());
    +      if (key.equalsIgnoreCase("statcount") || key.equalsIgnoreCase("nonnullstatcount")) {
    +        functions.put(key, "sum");
    +      } else if (key.equalsIgnoreCase("hll")) {
    +        functions.put(key, "hll_merge");
    +      } else if (key.equalsIgnoreCase("sum_width")) {
    +        functions.put(key, "avg_width");
    +      }
    +    }*/
    +    for (String key : popConfig.getFunctions()) {
    +      if (key.equalsIgnoreCase("sum_width")) {
    +        functions.put(key, "avg_width");
    +      } else if (key.equalsIgnoreCase("hll")) {
    +        functions.put(key, "hll_merge");
    +      } else {
    +        functions.put(key, key);
    +      }
    +      aggregationMap.put(functions.get(key), new HashMap<String, ValueHolder>());
    +    }
    +  }
    +
    +  private void createKeyColumn(String name, LogicalExpression expr, Map<MaterializedField, ValueVector> parentMap)
    +      throws SchemaChangeException {
    +    ErrorCollector collector = new ErrorCollectorImpl();
    +
    +    LogicalExpression mle = ExpressionTreeMaterializer.materialize(expr, incoming, collector,
    +        context.getFunctionRegistry());
    +
    +    MaterializedField outputField = MaterializedField.create(name, mle.getMajorType());
    +    ValueVector vector = TypeHelper.getNewVector(outputField, oContext.getAllocator());
    +    container.add(vector);
    +
    +    if (collector.hasErrors()) {
    +      throw new SchemaChangeException("Failure while materializing expression. "
    +          + collector.toErrorString());
    +    }
    +    parentMap.put(outputField, vector);
    +  }
    +
    +  private ValueVector addMapVector(String name, MapVector parent, LogicalExpression expr) throws SchemaChangeException {
    +    ErrorCollector collector = new ErrorCollectorImpl();
    +
    +    LogicalExpression mle = ExpressionTreeMaterializer.materialize(expr, incoming, collector,
    +        context.getFunctionRegistry());
    +
    +    Class<? extends ValueVector> vvc =
    +        TypeHelper.getValueVectorClass(mle.getMajorType().getMinorType(), mle.getMajorType().getMode());
    +    ValueVector vector = parent.addOrGet(name, mle.getMajorType(), vvc);
    +
    +    if (collector.hasErrors()) {
    +      throw new SchemaChangeException("Failure while materializing expression. "
    +          + collector.toErrorString());
    +    }
    +    return vector;
    +  }
    +
    +  /**
    +   * Identify the list of fields within a map which are unpivoted as columns in output
    +   */
    +  private void buildKeyList() {
    +    List<String> lastMapKeyList = null;
    +    for (VectorWrapper<?> vw : incoming) {
    +      if (vw.getField().getType().getMinorType() != TypeProtos.MinorType.MAP) {
    +        continue;
    +      }
    +
    +      keyList = Lists.newArrayList();
    +
    +      for (ValueVector vv : vw.getValueVector()) {
    +        keyList.add(vv.getField().getLastName());
    +      }
    +
    +      if (lastMapKeyList == null) {
    +        lastMapKeyList = keyList;
    +      } else {
    +        if (keyList.size() != lastMapKeyList.size() || !lastMapKeyList.containsAll(keyList)) {
    +          throw new UnsupportedOperationException("Maps have different fields");
    +        }
    +      }
    +    }
    +  }
    +
    +  private void buildOutputContainer() throws SchemaChangeException {
    +    dataSrcVecMap = Maps.newHashMap();
    +    copySrcVecMap = Maps.newHashMap();
    +    MajorType mt = null;
    +
    +    ErrorCollector collector = new ErrorCollectorImpl();
    +    GregorianCalendar calendar = new GregorianCalendar(TimeZone.getTimeZone("UTC"));
    +
    +    calendar.setTimeInMillis(System.currentTimeMillis());
    +    createKeyColumn("schema", ValueExpressions.getBigInt(schema++), copySrcVecMap);
    +    createKeyColumn("computed", ValueExpressions.getDate(calendar), copySrcVecMap);
    +
    +    for (VectorWrapper<?> vw : incoming) {
    +      addVectorToOutgoingContainer(vw.getField().getLastName(), vw, collector);
    +      /*MaterializedField ds = vw.getField();
    +      String field = vw.getField().getLastName();
    +      // Input map vector
    +      MapVector mapVector = (MapVector) vw.getValueVector();
    +      assert mapVector.getPrimitiveVectors().size() > 0;
    +      // Proceed to create output map vector with same name e.g. statcount etc.
    +      mt = mapVector.getField().getType();
    +      MaterializedField mf = MaterializedField.create(functions.get(field), mt);
    +      assert !dataSrcVecMap.containsKey(mf);
    +      ValueVector vector = TypeHelper.getNewVector(mf, oContext.getAllocator());
    +      container.add(vector);
    +      MapVector outputMapVector = (MapVector) vector;
    +
    +      for (ValueVector vv : mapVector) {
    +        String fieldName = vv.getField().getLastName();
    +        if (!keyList.contains(fieldName)) {
    +          throw new UnsupportedOperationException("Unpivot data vector " +
    +              ds + " contains key " + fieldName + " not contained in key source!");
    +        }
    +        if (vv.getField().getType().getMinorType() == TypeProtos.MinorType.MAP) {
    +          throw new UnsupportedOperationException("Unpivot of nested map is not supported!");
    +        }
    +        if (field.equals("column")) {
    +          outputMapVector.addOrGet(fieldName, vv.getField().getType(), vv.getClass());
    +        } else {
    +          List<LogicalExpression> args = Lists.newArrayList();
    +          //TODO: Something else to access value of col such as emp_id?
    +          args.add(SchemaPath.getSimplePath(vv.getField().getPath()));
    +          //TODO: Put in the mapVector
    +          LogicalExpression call = FunctionCallFactory.createExpression(functions.get(field), args);
    +          //TODO: Is this sufficient to add to new Map?
    +          ValueVector vector1 = addMapVector(fieldName, outputMapVector, call);
    +          if (collector.hasErrors()) {
    +            throw new SchemaChangeException("Failure while materializing expression. "
    +                + collector.toErrorString());
    +          }
    +        }
    +      }
    +      dataSrcVecMap.put(ds, outputMapVector);*/
    +    }
    +    //Now create NDV in the outgoing container which was not avaliable in the incoming
    +    for (VectorWrapper<?> vw : incoming) {
    +      if (vw.getField().getLastName().equalsIgnoreCase("sum_width")) {//NullableFloat8 type
    +        addVectorToOutgoingContainer("ndv", vw, collector);
    +        break;
    +      }
    +    }
    +    container.setRecordCount(0);
    +    recordCount = 0;
    +    container.buildSchema(incoming.getSchema().getSelectionVectorMode());
    +  }
    +
    +  private void addVectorToOutgoingContainer(String field, VectorWrapper vw, ErrorCollector collector)
    +     throws SchemaChangeException {
    +    // Input map vector
    +    MapVector mapVector = (MapVector) vw.getValueVector();
    +    MaterializedField mf;
    +    assert mapVector.getPrimitiveVectors().size() > 0;
    +    // Proceed to create output map vector with same name e.g. statcount etc.
    +    MajorType mt = mapVector.getField().getType();
    +    if (functions.get(field) != null) {
    +      mf = MaterializedField.create(functions.get(field), mt);
    +    } else {
    +      mf = MaterializedField.create(field, mt);
    +    }
    +    assert !dataSrcVecMap.containsKey(mf);
    +    ValueVector vector = TypeHelper.getNewVector(mf, oContext.getAllocator());
    +    container.add(vector);
    +    MapVector outputMapVector = (MapVector) vector;
    +
    +    for (ValueVector vv : mapVector) {
    +      String fieldName = vv.getField().getLastName();
    +      if (!keyList.contains(fieldName)) {
    +        throw new UnsupportedOperationException("Unpivot data vector " +
    +                field + " contains key " + fieldName + " not contained in key source!");
    +      }
    +      if (vv.getField().getType().getMinorType() == TypeProtos.MinorType.MAP) {
    +        throw new UnsupportedOperationException("Unpivot of nested map is not supported!");
    +      }
    +      if (field.equals("column")) {
    +        outputMapVector.addOrGet(fieldName, vv.getField().getType(), vv.getClass());
    +      } else {
    +        List<LogicalExpression> args = Lists.newArrayList();
    +        LogicalExpression call;
    +        //TODO: Something else to access value of col such as emp_id?
    +        args.add(SchemaPath.getSimplePath(vv.getField().getPath()));
    +        //TODO: Put in the mapVector
    +        if (functions.get(field) != null) {
    +          call = FunctionCallFactory.createExpression(functions.get(field), args);
    +        } else {
    +          call = FunctionCallFactory.createExpression(field, args);
    +        }
    +        //TODO: Is this sufficient to add to new Map?
    +        ValueVector vector1 = addMapVector(fieldName, outputMapVector, call);
    +        if (collector.hasErrors()) {
    +          throw new SchemaChangeException("Failure while materializing expression. "
    +                  + collector.toErrorString());
    +        }
    +      }
    +    }
    +  }
    +
    +  @Override
    +  protected boolean setupNewSchema() throws SchemaChangeException {
    +    container.clear();
    +    buildKeyList();
    +    buildOutputContainer();
    +    return true;
    +  }
    +
    +  @Override
    +  protected IterOutcome doWork() {
    +    int outRecordCount = incoming.getRecordCount();
    +    HashMap<String, Long> nonNullRows = new HashMap<>();
    +
    +    for (VectorWrapper<?> vw : incoming) {
    +      ValueVector vv = vw.getValueVector();
    +      if (vv.getField().getType().getMinorType() != TypeProtos.MinorType.MAP) {
    +        //We only expect Maps in the incoming. STOP, if this is not the case.
    +        return IterOutcome.STOP;
    +      }
    +      MapVector mapVec = (MapVector) vv;
    +      Map<String, ValueHolder> statMap = aggregationMap.get(vv.getField().getLastName());
    +
    +      for (ValueVector mapElt : mapVec) {
    +        if (vv.getField().getLastName().equalsIgnoreCase("column")) {
    +          BigIntHolder nameHolder;
    +          if (statMap.get(mapElt.getField().getLastName()) != null) {
    +            nameHolder = (BigIntHolder) statMap.get(mapElt.getField().getLastName());
    +          } else {
    +            nameHolder = new BigIntHolder();
    +            statMap.put(mapElt.getField().getLastName(), nameHolder);
    +          }
    +          nameHolder.value = 1;
    +        } else if (vv.getField().getLastName().equalsIgnoreCase("statcount")
    +            || vv.getField().getLastName().equalsIgnoreCase("nonnullstatcount")) {
    +          BigIntHolder sumHolder;
    +          String colName = mapElt.getField().getLastName();
    +          if (statMap.get(colName) != null) {
    +            sumHolder = (BigIntHolder) statMap.get(colName);
    +          } else {
    +            sumHolder = new BigIntHolder();
    +            statMap.put(colName, sumHolder);
    +          }
    +          //TODO: assert size = 1
    +          //TODO: logger
    +          Object val = mapElt.getAccessor().getObject(0);
    +          if (val != null) {
    +            sumHolder.value += (long)val;
    +          }
    +        } else if (vv.getField().getLastName().equalsIgnoreCase("sum_width")) {
    +          NullableFloat8Holder sumHolder;
    +          String colName = mapElt.getField().getLastName();
    +          if (statMap == null) {
    +            statMap = aggregationMap.get(functions.get(vv.getField().getLastName()));
    +          }
    +          if (statMap.get(colName) != null) {
    +            sumHolder = (NullableFloat8Holder) statMap.get(colName);
    +          } else {
    +            sumHolder = new NullableFloat8Holder();
    +            statMap.put(colName, sumHolder);
    +          }
    +          //TODO: assert size = 1
    +          //TODO: logger
    +          Object val = mapElt.getAccessor().getObject(0);
    +          if (val != null) {
    +            sumHolder.value += (double) val;
    +            sumHolder.isSet = 1;
    +          }
    +        } else if (vv.getField().getLastName().equalsIgnoreCase("hll")) {
    +          ObjectHolder hllHolder;
    +          String colName = mapElt.getField().getLastName();
    +          if (statMap == null) {
    +            statMap = aggregationMap.get(functions.get(vv.getField().getLastName()));
    +          }
    +          if (statMap.get(colName) != null) {
    +            hllHolder = (ObjectHolder) statMap.get(colName);
    +          } else {
    +            hllHolder = new ObjectHolder();
    +            hllHolder.obj = new HyperLogLog(context.getContextInformation().getHllAccuracy());
    +            statMap.put(colName, hllHolder);
    +          }
    +          NullableVarBinaryVector hllVector = (NullableVarBinaryVector) mapElt;
    +          try {
    +            if (hllVector.getAccessor().isSet(0) == 1) {
    +              ByteArrayInputStream bais = new ByteArrayInputStream(hllVector.getAccessor().getObject(0), 0,
    +                  mapElt.getBufferSize());
    +              HyperLogLog other = HyperLogLog.Builder.build(new DataInputStream(bais));
    +              ((HyperLogLog) hllHolder.obj).addAll(other);
    +            }
    +          } catch (Exception ex) {
    +            //TODO: Catch IOException/CardinalityMergeException
    +            //TODO: logger
    +            return IterOutcome.STOP;
    +          }
    +        }
    +      }
    +      // Add NDV value vector map using HLL map (since the NDV map is directly generated from HLL and not produced by the underlying
    +      // Statistics Agg)
    +      Map<String, ValueHolder> hllMap = aggregationMap.get("hll");
    +      if (hllMap != null) {
    +        aggregationMap.put("ndv", hllMap);
    +      }
    +    }
    +    return IterOutcome.OK;
    +  }
    +
    +  public VectorContainer getOutgoingContainer() {
    +    return this.container;
    +  }
    +
    +  @Override
    +  public IterOutcome innerNext() {
    +    IterOutcome outcome;
    +    boolean didSomeWork = false;
    +    if (finished) {
    +      return IterOutcome.NONE;
    +    }
    +    try {
    +      outer: while (true) {
    +        outcome = next(incoming);
    +        switch (outcome) {
    +          case NONE:
    +            break outer;
    +          case OUT_OF_MEMORY:
    +          case NOT_YET:
    +          case STOP:
    +            return outcome;
    +          case OK_NEW_SCHEMA:
    +            if (first) {
    +              first =false;
    +              if (!setupNewSchema()) {
    +                outcome = IterOutcome.OK;
    +              }
    +              return outcome;
    +            }
    +            //fall through
    +          case OK:
    +            assert first == false : "First batch should be OK_NEW_SCHEMA";
    +            IterOutcome out = doWork();
    +            didSomeWork = true;
    +            if (out != IterOutcome.OK) {
    +              return out;
    +            }
    +            break;
    +          default:
    +            throw new UnsupportedOperationException("Unsupported upstream state " + outcome);
    +        }
    +      }
    +    } catch (SchemaChangeException ex) {
    +      kill(false);
    +      logger.error("Failure during query", ex);
    +      context.fail(ex);
    --- End diff --
    
    Throw a `UserException`.


---
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 #729: Drill 1328: Support table statistics for Parquet

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

    https://github.com/apache/drill/pull/729#discussion_r102295106
  
    --- Diff: exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/statistics/StatisticsMergeBatch.java ---
    @@ -0,0 +1,532 @@
    +/**
    + * 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.impl.statistics;
    +
    +import com.google.common.collect.Lists;
    +import com.google.common.collect.Maps;
    +import org.apache.drill.common.expression.LogicalExpression;
    +import org.apache.drill.common.expression.ErrorCollector;
    +import org.apache.drill.common.expression.ErrorCollectorImpl;
    +import org.apache.drill.common.expression.FunctionCallFactory;
    +import org.apache.drill.common.expression.SchemaPath;
    +import org.apache.drill.common.expression.ValueExpressions;
    +import org.apache.drill.common.types.TypeProtos;
    +import org.apache.drill.common.types.TypeProtos.MajorType;
    +import org.apache.drill.exec.exception.OutOfMemoryException;
    +import org.apache.drill.exec.exception.SchemaChangeException;
    +import org.apache.drill.exec.expr.ExpressionTreeMaterializer;
    +import org.apache.drill.exec.expr.TypeHelper;
    +import org.apache.drill.exec.expr.holders.BigIntHolder;
    +import org.apache.drill.exec.expr.holders.NullableFloat8Holder;
    +import org.apache.drill.exec.expr.holders.ObjectHolder;
    +import org.apache.drill.exec.expr.holders.ValueHolder;
    +import org.apache.drill.exec.ops.FragmentContext;
    +import org.apache.drill.exec.physical.config.StatisticsMerge;
    +import org.apache.drill.exec.record.AbstractSingleRecordBatch;
    +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.VectorContainer;
    +import org.apache.drill.exec.record.VectorWrapper;
    +import org.apache.drill.exec.vector.ValueVector;
    +import org.apache.drill.exec.vector.DateVector;
    +import org.apache.drill.exec.vector.BigIntVector;
    +import org.apache.drill.exec.vector.NullableBigIntVector;
    +import org.apache.drill.exec.vector.NullableFloat8Vector;
    +import org.apache.drill.exec.vector.NullableVarBinaryVector;
    +import org.apache.drill.exec.vector.VarCharVector;
    +import org.apache.drill.exec.vector.complex.MapVector;
    +
    +import java.io.ByteArrayInputStream;
    +import java.io.DataInputStream;
    +import java.io.IOException;
    +import java.util.List;
    +import java.util.GregorianCalendar;
    +import java.util.HashMap;
    +import java.util.Map;
    +import java.util.TimeZone;
    +
    +import com.clearspring.analytics.stream.cardinality.HyperLogLog;
    +
    +public class StatisticsMergeBatch extends AbstractSingleRecordBatch<StatisticsMerge> {
    +  private static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(StatisticsMergeBatch.class);
    +  private Map<String, String> functions;
    +  private boolean first = true;
    +  private boolean finished = false;
    +  private int schema = 0;
    +  private int recordCount = 0;
    +  private List<String> keyList = null;
    +  private Map<MaterializedField, ValueVector> dataSrcVecMap = null;
    +  // Map of non-map fields to VV in the incoming schema
    +  private Map<MaterializedField, ValueVector> copySrcVecMap = null;
    +  private Map<String, Map<String, ValueHolder>> aggregationMap = null;
    +  public StatisticsMergeBatch(StatisticsMerge popConfig, RecordBatch incoming,
    +                              FragmentContext context) throws OutOfMemoryException {
    +    super(popConfig, context, incoming);
    +    this.functions = new HashMap<>();
    +    this.aggregationMap = new HashMap<>();
    +
    +    /*for (String key : popConfig.getFunctions()) {
    +      aggregationMap.put(key, new HashMap<String, ValueHolder>());
    +      if (key.equalsIgnoreCase("statcount") || key.equalsIgnoreCase("nonnullstatcount")) {
    +        functions.put(key, "sum");
    +      } else if (key.equalsIgnoreCase("hll")) {
    +        functions.put(key, "hll_merge");
    +      } else if (key.equalsIgnoreCase("sum_width")) {
    +        functions.put(key, "avg_width");
    +      }
    +    }*/
    +    for (String key : popConfig.getFunctions()) {
    +      if (key.equalsIgnoreCase("sum_width")) {
    +        functions.put(key, "avg_width");
    +      } else if (key.equalsIgnoreCase("hll")) {
    +        functions.put(key, "hll_merge");
    +      } else {
    +        functions.put(key, key);
    +      }
    +      aggregationMap.put(functions.get(key), new HashMap<String, ValueHolder>());
    +    }
    +  }
    +
    +  private void createKeyColumn(String name, LogicalExpression expr, Map<MaterializedField, ValueVector> parentMap)
    +      throws SchemaChangeException {
    +    ErrorCollector collector = new ErrorCollectorImpl();
    +
    +    LogicalExpression mle = ExpressionTreeMaterializer.materialize(expr, incoming, collector,
    +        context.getFunctionRegistry());
    +
    +    MaterializedField outputField = MaterializedField.create(name, mle.getMajorType());
    +    ValueVector vector = TypeHelper.getNewVector(outputField, oContext.getAllocator());
    +    container.add(vector);
    +
    +    if (collector.hasErrors()) {
    +      throw new SchemaChangeException("Failure while materializing expression. "
    +          + collector.toErrorString());
    +    }
    +    parentMap.put(outputField, vector);
    +  }
    +
    +  private ValueVector addMapVector(String name, MapVector parent, LogicalExpression expr) throws SchemaChangeException {
    +    ErrorCollector collector = new ErrorCollectorImpl();
    +
    +    LogicalExpression mle = ExpressionTreeMaterializer.materialize(expr, incoming, collector,
    +        context.getFunctionRegistry());
    +
    +    Class<? extends ValueVector> vvc =
    +        TypeHelper.getValueVectorClass(mle.getMajorType().getMinorType(), mle.getMajorType().getMode());
    +    ValueVector vector = parent.addOrGet(name, mle.getMajorType(), vvc);
    +
    +    if (collector.hasErrors()) {
    +      throw new SchemaChangeException("Failure while materializing expression. "
    +          + collector.toErrorString());
    +    }
    +    return vector;
    +  }
    +
    +  /**
    +   * Identify the list of fields within a map which are unpivoted as columns in output
    +   */
    +  private void buildKeyList() {
    +    List<String> lastMapKeyList = null;
    +    for (VectorWrapper<?> vw : incoming) {
    +      if (vw.getField().getType().getMinorType() != TypeProtos.MinorType.MAP) {
    +        continue;
    +      }
    +
    +      keyList = Lists.newArrayList();
    +
    +      for (ValueVector vv : vw.getValueVector()) {
    +        keyList.add(vv.getField().getLastName());
    +      }
    +
    +      if (lastMapKeyList == null) {
    +        lastMapKeyList = keyList;
    +      } else {
    +        if (keyList.size() != lastMapKeyList.size() || !lastMapKeyList.containsAll(keyList)) {
    +          throw new UnsupportedOperationException("Maps have different fields");
    +        }
    +      }
    +    }
    +  }
    +
    +  private void buildOutputContainer() throws SchemaChangeException {
    +    dataSrcVecMap = Maps.newHashMap();
    +    copySrcVecMap = Maps.newHashMap();
    +    MajorType mt = null;
    +
    +    ErrorCollector collector = new ErrorCollectorImpl();
    +    GregorianCalendar calendar = new GregorianCalendar(TimeZone.getTimeZone("UTC"));
    +
    +    calendar.setTimeInMillis(System.currentTimeMillis());
    +    createKeyColumn("schema", ValueExpressions.getBigInt(schema++), copySrcVecMap);
    +    createKeyColumn("computed", ValueExpressions.getDate(calendar), copySrcVecMap);
    +
    +    for (VectorWrapper<?> vw : incoming) {
    +      addVectorToOutgoingContainer(vw.getField().getLastName(), vw, collector);
    +      /*MaterializedField ds = vw.getField();
    +      String field = vw.getField().getLastName();
    +      // Input map vector
    +      MapVector mapVector = (MapVector) vw.getValueVector();
    +      assert mapVector.getPrimitiveVectors().size() > 0;
    +      // Proceed to create output map vector with same name e.g. statcount etc.
    +      mt = mapVector.getField().getType();
    +      MaterializedField mf = MaterializedField.create(functions.get(field), mt);
    +      assert !dataSrcVecMap.containsKey(mf);
    +      ValueVector vector = TypeHelper.getNewVector(mf, oContext.getAllocator());
    +      container.add(vector);
    +      MapVector outputMapVector = (MapVector) vector;
    +
    +      for (ValueVector vv : mapVector) {
    +        String fieldName = vv.getField().getLastName();
    +        if (!keyList.contains(fieldName)) {
    +          throw new UnsupportedOperationException("Unpivot data vector " +
    +              ds + " contains key " + fieldName + " not contained in key source!");
    +        }
    +        if (vv.getField().getType().getMinorType() == TypeProtos.MinorType.MAP) {
    +          throw new UnsupportedOperationException("Unpivot of nested map is not supported!");
    +        }
    +        if (field.equals("column")) {
    +          outputMapVector.addOrGet(fieldName, vv.getField().getType(), vv.getClass());
    +        } else {
    +          List<LogicalExpression> args = Lists.newArrayList();
    +          //TODO: Something else to access value of col such as emp_id?
    +          args.add(SchemaPath.getSimplePath(vv.getField().getPath()));
    +          //TODO: Put in the mapVector
    +          LogicalExpression call = FunctionCallFactory.createExpression(functions.get(field), args);
    +          //TODO: Is this sufficient to add to new Map?
    +          ValueVector vector1 = addMapVector(fieldName, outputMapVector, call);
    +          if (collector.hasErrors()) {
    +            throw new SchemaChangeException("Failure while materializing expression. "
    +                + collector.toErrorString());
    +          }
    +        }
    +      }
    +      dataSrcVecMap.put(ds, outputMapVector);*/
    +    }
    +    //Now create NDV in the outgoing container which was not avaliable in the incoming
    +    for (VectorWrapper<?> vw : incoming) {
    +      if (vw.getField().getLastName().equalsIgnoreCase("sum_width")) {//NullableFloat8 type
    +        addVectorToOutgoingContainer("ndv", vw, collector);
    +        break;
    +      }
    +    }
    +    container.setRecordCount(0);
    +    recordCount = 0;
    +    container.buildSchema(incoming.getSchema().getSelectionVectorMode());
    +  }
    +
    +  private void addVectorToOutgoingContainer(String field, VectorWrapper vw, ErrorCollector collector)
    +     throws SchemaChangeException {
    +    // Input map vector
    +    MapVector mapVector = (MapVector) vw.getValueVector();
    +    MaterializedField mf;
    +    assert mapVector.getPrimitiveVectors().size() > 0;
    +    // Proceed to create output map vector with same name e.g. statcount etc.
    +    MajorType mt = mapVector.getField().getType();
    +    if (functions.get(field) != null) {
    +      mf = MaterializedField.create(functions.get(field), mt);
    +    } else {
    +      mf = MaterializedField.create(field, mt);
    +    }
    +    assert !dataSrcVecMap.containsKey(mf);
    +    ValueVector vector = TypeHelper.getNewVector(mf, oContext.getAllocator());
    +    container.add(vector);
    +    MapVector outputMapVector = (MapVector) vector;
    +
    +    for (ValueVector vv : mapVector) {
    +      String fieldName = vv.getField().getLastName();
    +      if (!keyList.contains(fieldName)) {
    +        throw new UnsupportedOperationException("Unpivot data vector " +
    +                field + " contains key " + fieldName + " not contained in key source!");
    +      }
    +      if (vv.getField().getType().getMinorType() == TypeProtos.MinorType.MAP) {
    +        throw new UnsupportedOperationException("Unpivot of nested map is not supported!");
    +      }
    +      if (field.equals("column")) {
    +        outputMapVector.addOrGet(fieldName, vv.getField().getType(), vv.getClass());
    +      } else {
    +        List<LogicalExpression> args = Lists.newArrayList();
    +        LogicalExpression call;
    +        //TODO: Something else to access value of col such as emp_id?
    +        args.add(SchemaPath.getSimplePath(vv.getField().getPath()));
    +        //TODO: Put in the mapVector
    +        if (functions.get(field) != null) {
    +          call = FunctionCallFactory.createExpression(functions.get(field), args);
    +        } else {
    +          call = FunctionCallFactory.createExpression(field, args);
    +        }
    +        //TODO: Is this sufficient to add to new Map?
    +        ValueVector vector1 = addMapVector(fieldName, outputMapVector, call);
    +        if (collector.hasErrors()) {
    +          throw new SchemaChangeException("Failure while materializing expression. "
    +                  + collector.toErrorString());
    +        }
    +      }
    +    }
    +  }
    +
    +  @Override
    +  protected boolean setupNewSchema() throws SchemaChangeException {
    +    container.clear();
    +    buildKeyList();
    +    buildOutputContainer();
    +    return true;
    +  }
    +
    +  @Override
    +  protected IterOutcome doWork() {
    +    int outRecordCount = incoming.getRecordCount();
    +    HashMap<String, Long> nonNullRows = new HashMap<>();
    +
    +    for (VectorWrapper<?> vw : incoming) {
    +      ValueVector vv = vw.getValueVector();
    +      if (vv.getField().getType().getMinorType() != TypeProtos.MinorType.MAP) {
    +        //We only expect Maps in the incoming. STOP, if this is not the case.
    +        return IterOutcome.STOP;
    +      }
    +      MapVector mapVec = (MapVector) vv;
    +      Map<String, ValueHolder> statMap = aggregationMap.get(vv.getField().getLastName());
    +
    --- End diff --
    
    Sorry, the refactored code has comments to explain what we are doing.


---
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 #729: Drill 1328: Support table statistics for Parquet

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

    https://github.com/apache/drill/pull/729#discussion_r103410529
  
    --- Diff: exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/statistics/ColTypeMergedStatistic.java ---
    @@ -0,0 +1,93 @@
    +/**
    + * 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.impl.statistics;
    +
    +import org.apache.drill.common.types.TypeProtos;
    +import org.apache.drill.exec.expr.holders.IntHolder;
    +import org.apache.drill.exec.expr.holders.ValueHolder;
    +import org.apache.drill.exec.vector.IntVector;
    +import org.apache.drill.exec.vector.ValueVector;
    +import org.apache.drill.exec.vector.complex.MapVector;
    +
    +import java.util.HashMap;
    +import java.util.Map;
    +
    +public class ColTypeMergedStatistic extends AbstractMergedStatistic {
    +  private String name;
    +  private String inputName;
    +  private boolean mergeComplete = false;
    +  private Map<String, ValueHolder> typeHolder;
    +
    +
    +  public ColTypeMergedStatistic (String name, String inputName) {
    +    this.name = name;
    +    this.inputName = inputName;
    +    typeHolder = new HashMap<>();
    +  }
    +
    +  @Override
    +  public String getName() {
    +    return name;
    +  }
    +
    +  @Override
    +  public String getInput() {
    +    return inputName;
    +  }
    +
    +  @Override
    +  public void merge(ValueVector input) {
    +    // Check the input is a Map Vector
    +    assert (input.getField().getType().getMinorType() == TypeProtos.MinorType.MAP);
    +    MapVector inputMap = (MapVector) input;
    +    for (ValueVector vv : inputMap) {
    +      String colName = vv.getField().getLastName();
    +      if (typeHolder.get(colName) == null) {
    +        IntHolder colType = new IntHolder();
    +        ((IntVector) vv).getAccessor().get(0, colType);
    +        typeHolder.put(colName, colType);
    +      }
    +    }
    +  }
    +
    +  @Override
    +  public Object getStat(String colName) {
    --- 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 #729: Drill 1328: Support table statistics for Parquet

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/729#discussion_r102867208
  
    --- Diff: exec/java-exec/src/main/java/org/apache/drill/exec/ops/QueryContext.java ---
    @@ -283,4 +288,22 @@ public void close() throws Exception {
           closed = true;
         }
       }
    +
    +  /**
    +  * @param stmtType : Sets the type {@link SqlStatementType} of the statement e.g. CTAS, ANALYZE
    +  */
    +  public void setSQLStatementType(SqlStatementType stmtType) {
    +    if (this.stmtType == null) {
    +      this.stmtType = stmtType;
    +    } else {
    +      throw new UnsupportedOperationException("SQL Statement type is already set");
    --- End diff --
    
    `UnsupportedOperationException` is when the user requests an operation that Drill does not support. Here, you mean either `IllegalArgumentException` or `IllegalStateException` - both indicate a programming error.


---
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 #729: Drill 1328 r4

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/729#discussion_r99392086
  
    --- Diff: exec/java-exec/src/main/java/org/apache/drill/exec/ops/FragmentContext.java ---
    @@ -245,6 +245,16 @@ public SchemaPlus getRootSchema() {
       }
     
       /**
    +   * Returns the statement type (e.g. SELECT, CTAS, ANALYZE) from the query context.
    +   * @return query statement type {@link QueryContext.StatementType}, if known.
    +   */
    +  public QueryContext.StatementType getStatementType() {
    +    if (queryContext != null) {
    --- End diff --
    
    Please explain under what conditions the query context will be null. Is this just being paranoid? Or, are there statement types where there is no context? And, if there are such statements, aren't those statements known, not "UNKNOWN"?


---
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 #729: Drill 1328: Support table statistics for Parquet

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

    https://github.com/apache/drill/pull/729#discussion_r101176671
  
    --- Diff: exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/statistics/StatisticsMergeBatch.java ---
    @@ -0,0 +1,532 @@
    +/**
    + * 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.impl.statistics;
    +
    +import com.google.common.collect.Lists;
    +import com.google.common.collect.Maps;
    +import org.apache.drill.common.expression.LogicalExpression;
    +import org.apache.drill.common.expression.ErrorCollector;
    +import org.apache.drill.common.expression.ErrorCollectorImpl;
    +import org.apache.drill.common.expression.FunctionCallFactory;
    +import org.apache.drill.common.expression.SchemaPath;
    +import org.apache.drill.common.expression.ValueExpressions;
    +import org.apache.drill.common.types.TypeProtos;
    +import org.apache.drill.common.types.TypeProtos.MajorType;
    +import org.apache.drill.exec.exception.OutOfMemoryException;
    +import org.apache.drill.exec.exception.SchemaChangeException;
    +import org.apache.drill.exec.expr.ExpressionTreeMaterializer;
    +import org.apache.drill.exec.expr.TypeHelper;
    +import org.apache.drill.exec.expr.holders.BigIntHolder;
    +import org.apache.drill.exec.expr.holders.NullableFloat8Holder;
    +import org.apache.drill.exec.expr.holders.ObjectHolder;
    +import org.apache.drill.exec.expr.holders.ValueHolder;
    +import org.apache.drill.exec.ops.FragmentContext;
    +import org.apache.drill.exec.physical.config.StatisticsMerge;
    +import org.apache.drill.exec.record.AbstractSingleRecordBatch;
    +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.VectorContainer;
    +import org.apache.drill.exec.record.VectorWrapper;
    +import org.apache.drill.exec.vector.ValueVector;
    +import org.apache.drill.exec.vector.DateVector;
    +import org.apache.drill.exec.vector.BigIntVector;
    +import org.apache.drill.exec.vector.NullableBigIntVector;
    +import org.apache.drill.exec.vector.NullableFloat8Vector;
    +import org.apache.drill.exec.vector.NullableVarBinaryVector;
    +import org.apache.drill.exec.vector.VarCharVector;
    +import org.apache.drill.exec.vector.complex.MapVector;
    +
    +import java.io.ByteArrayInputStream;
    +import java.io.DataInputStream;
    +import java.io.IOException;
    +import java.util.List;
    +import java.util.GregorianCalendar;
    +import java.util.HashMap;
    +import java.util.Map;
    +import java.util.TimeZone;
    +
    +import com.clearspring.analytics.stream.cardinality.HyperLogLog;
    +
    +public class StatisticsMergeBatch extends AbstractSingleRecordBatch<StatisticsMerge> {
    +  private static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(StatisticsMergeBatch.class);
    +  private Map<String, String> functions;
    +  private boolean first = true;
    +  private boolean finished = false;
    +  private int schema = 0;
    +  private int recordCount = 0;
    +  private List<String> keyList = null;
    +  private Map<MaterializedField, ValueVector> dataSrcVecMap = null;
    +  // Map of non-map fields to VV in the incoming schema
    +  private Map<MaterializedField, ValueVector> copySrcVecMap = null;
    +  private Map<String, Map<String, ValueHolder>> aggregationMap = null;
    +  public StatisticsMergeBatch(StatisticsMerge popConfig, RecordBatch incoming,
    +                              FragmentContext context) throws OutOfMemoryException {
    +    super(popConfig, context, incoming);
    +    this.functions = new HashMap<>();
    +    this.aggregationMap = new HashMap<>();
    +
    +    /*for (String key : popConfig.getFunctions()) {
    +      aggregationMap.put(key, new HashMap<String, ValueHolder>());
    +      if (key.equalsIgnoreCase("statcount") || key.equalsIgnoreCase("nonnullstatcount")) {
    +        functions.put(key, "sum");
    +      } else if (key.equalsIgnoreCase("hll")) {
    +        functions.put(key, "hll_merge");
    +      } else if (key.equalsIgnoreCase("sum_width")) {
    +        functions.put(key, "avg_width");
    +      }
    +    }*/
    +    for (String key : popConfig.getFunctions()) {
    +      if (key.equalsIgnoreCase("sum_width")) {
    +        functions.put(key, "avg_width");
    +      } else if (key.equalsIgnoreCase("hll")) {
    +        functions.put(key, "hll_merge");
    +      } else {
    +        functions.put(key, key);
    +      }
    +      aggregationMap.put(functions.get(key), new HashMap<String, ValueHolder>());
    +    }
    +  }
    +
    +  private void createKeyColumn(String name, LogicalExpression expr, Map<MaterializedField, ValueVector> parentMap)
    +      throws SchemaChangeException {
    +    ErrorCollector collector = new ErrorCollectorImpl();
    +
    +    LogicalExpression mle = ExpressionTreeMaterializer.materialize(expr, incoming, collector,
    +        context.getFunctionRegistry());
    +
    +    MaterializedField outputField = MaterializedField.create(name, mle.getMajorType());
    +    ValueVector vector = TypeHelper.getNewVector(outputField, oContext.getAllocator());
    +    container.add(vector);
    +
    +    if (collector.hasErrors()) {
    +      throw new SchemaChangeException("Failure while materializing expression. "
    +          + collector.toErrorString());
    +    }
    +    parentMap.put(outputField, vector);
    +  }
    +
    +  private ValueVector addMapVector(String name, MapVector parent, LogicalExpression expr) throws SchemaChangeException {
    +    ErrorCollector collector = new ErrorCollectorImpl();
    +
    +    LogicalExpression mle = ExpressionTreeMaterializer.materialize(expr, incoming, collector,
    +        context.getFunctionRegistry());
    +
    +    Class<? extends ValueVector> vvc =
    +        TypeHelper.getValueVectorClass(mle.getMajorType().getMinorType(), mle.getMajorType().getMode());
    +    ValueVector vector = parent.addOrGet(name, mle.getMajorType(), vvc);
    +
    +    if (collector.hasErrors()) {
    +      throw new SchemaChangeException("Failure while materializing expression. "
    +          + collector.toErrorString());
    +    }
    +    return vector;
    +  }
    +
    +  /**
    +   * Identify the list of fields within a map which are unpivoted as columns in output
    +   */
    +  private void buildKeyList() {
    +    List<String> lastMapKeyList = null;
    +    for (VectorWrapper<?> vw : incoming) {
    +      if (vw.getField().getType().getMinorType() != TypeProtos.MinorType.MAP) {
    +        continue;
    +      }
    +
    +      keyList = Lists.newArrayList();
    +
    +      for (ValueVector vv : vw.getValueVector()) {
    +        keyList.add(vv.getField().getLastName());
    +      }
    +
    +      if (lastMapKeyList == null) {
    +        lastMapKeyList = keyList;
    +      } else {
    +        if (keyList.size() != lastMapKeyList.size() || !lastMapKeyList.containsAll(keyList)) {
    +          throw new UnsupportedOperationException("Maps have different fields");
    --- 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 #729: Drill 1328: Support table statistics for Parquet

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

    https://github.com/apache/drill/pull/729#discussion_r102292075
  
    --- Diff: exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/statistics/StatisticsMergeBatch.java ---
    @@ -0,0 +1,532 @@
    +/**
    + * 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.impl.statistics;
    +
    +import com.google.common.collect.Lists;
    +import com.google.common.collect.Maps;
    +import org.apache.drill.common.expression.LogicalExpression;
    +import org.apache.drill.common.expression.ErrorCollector;
    +import org.apache.drill.common.expression.ErrorCollectorImpl;
    +import org.apache.drill.common.expression.FunctionCallFactory;
    +import org.apache.drill.common.expression.SchemaPath;
    +import org.apache.drill.common.expression.ValueExpressions;
    +import org.apache.drill.common.types.TypeProtos;
    +import org.apache.drill.common.types.TypeProtos.MajorType;
    +import org.apache.drill.exec.exception.OutOfMemoryException;
    +import org.apache.drill.exec.exception.SchemaChangeException;
    +import org.apache.drill.exec.expr.ExpressionTreeMaterializer;
    +import org.apache.drill.exec.expr.TypeHelper;
    +import org.apache.drill.exec.expr.holders.BigIntHolder;
    +import org.apache.drill.exec.expr.holders.NullableFloat8Holder;
    +import org.apache.drill.exec.expr.holders.ObjectHolder;
    +import org.apache.drill.exec.expr.holders.ValueHolder;
    +import org.apache.drill.exec.ops.FragmentContext;
    +import org.apache.drill.exec.physical.config.StatisticsMerge;
    +import org.apache.drill.exec.record.AbstractSingleRecordBatch;
    +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.VectorContainer;
    +import org.apache.drill.exec.record.VectorWrapper;
    +import org.apache.drill.exec.vector.ValueVector;
    +import org.apache.drill.exec.vector.DateVector;
    +import org.apache.drill.exec.vector.BigIntVector;
    +import org.apache.drill.exec.vector.NullableBigIntVector;
    +import org.apache.drill.exec.vector.NullableFloat8Vector;
    +import org.apache.drill.exec.vector.NullableVarBinaryVector;
    +import org.apache.drill.exec.vector.VarCharVector;
    +import org.apache.drill.exec.vector.complex.MapVector;
    +
    +import java.io.ByteArrayInputStream;
    +import java.io.DataInputStream;
    +import java.io.IOException;
    +import java.util.List;
    +import java.util.GregorianCalendar;
    +import java.util.HashMap;
    +import java.util.Map;
    +import java.util.TimeZone;
    +
    +import com.clearspring.analytics.stream.cardinality.HyperLogLog;
    +
    +public class StatisticsMergeBatch extends AbstractSingleRecordBatch<StatisticsMerge> {
    +  private static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(StatisticsMergeBatch.class);
    +  private Map<String, String> functions;
    +  private boolean first = true;
    +  private boolean finished = false;
    +  private int schema = 0;
    +  private int recordCount = 0;
    +  private List<String> keyList = null;
    +  private Map<MaterializedField, ValueVector> dataSrcVecMap = null;
    +  // Map of non-map fields to VV in the incoming schema
    +  private Map<MaterializedField, ValueVector> copySrcVecMap = null;
    +  private Map<String, Map<String, ValueHolder>> aggregationMap = null;
    +  public StatisticsMergeBatch(StatisticsMerge popConfig, RecordBatch incoming,
    +                              FragmentContext context) throws OutOfMemoryException {
    +    super(popConfig, context, incoming);
    +    this.functions = new HashMap<>();
    +    this.aggregationMap = new HashMap<>();
    +
    +    /*for (String key : popConfig.getFunctions()) {
    +      aggregationMap.put(key, new HashMap<String, ValueHolder>());
    +      if (key.equalsIgnoreCase("statcount") || key.equalsIgnoreCase("nonnullstatcount")) {
    +        functions.put(key, "sum");
    +      } else if (key.equalsIgnoreCase("hll")) {
    +        functions.put(key, "hll_merge");
    +      } else if (key.equalsIgnoreCase("sum_width")) {
    +        functions.put(key, "avg_width");
    +      }
    +    }*/
    +    for (String key : popConfig.getFunctions()) {
    +      if (key.equalsIgnoreCase("sum_width")) {
    +        functions.put(key, "avg_width");
    +      } else if (key.equalsIgnoreCase("hll")) {
    +        functions.put(key, "hll_merge");
    +      } else {
    +        functions.put(key, key);
    +      }
    +      aggregationMap.put(functions.get(key), new HashMap<String, ValueHolder>());
    +    }
    +  }
    +
    +  private void createKeyColumn(String name, LogicalExpression expr, Map<MaterializedField, ValueVector> parentMap)
    +      throws SchemaChangeException {
    +    ErrorCollector collector = new ErrorCollectorImpl();
    +
    +    LogicalExpression mle = ExpressionTreeMaterializer.materialize(expr, incoming, collector,
    +        context.getFunctionRegistry());
    +
    +    MaterializedField outputField = MaterializedField.create(name, mle.getMajorType());
    +    ValueVector vector = TypeHelper.getNewVector(outputField, oContext.getAllocator());
    +    container.add(vector);
    +
    +    if (collector.hasErrors()) {
    +      throw new SchemaChangeException("Failure while materializing expression. "
    +          + collector.toErrorString());
    +    }
    +    parentMap.put(outputField, vector);
    +  }
    +
    +  private ValueVector addMapVector(String name, MapVector parent, LogicalExpression expr) throws SchemaChangeException {
    +    ErrorCollector collector = new ErrorCollectorImpl();
    +
    +    LogicalExpression mle = ExpressionTreeMaterializer.materialize(expr, incoming, collector,
    +        context.getFunctionRegistry());
    +
    +    Class<? extends ValueVector> vvc =
    +        TypeHelper.getValueVectorClass(mle.getMajorType().getMinorType(), mle.getMajorType().getMode());
    +    ValueVector vector = parent.addOrGet(name, mle.getMajorType(), vvc);
    +
    +    if (collector.hasErrors()) {
    +      throw new SchemaChangeException("Failure while materializing expression. "
    +          + collector.toErrorString());
    +    }
    +    return vector;
    +  }
    +
    +  /**
    +   * Identify the list of fields within a map which are unpivoted as columns in output
    +   */
    +  private void buildKeyList() {
    +    List<String> lastMapKeyList = null;
    +    for (VectorWrapper<?> vw : incoming) {
    +      if (vw.getField().getType().getMinorType() != TypeProtos.MinorType.MAP) {
    +        continue;
    +      }
    +
    +      keyList = Lists.newArrayList();
    +
    +      for (ValueVector vv : vw.getValueVector()) {
    +        keyList.add(vv.getField().getLastName());
    +      }
    +
    +      if (lastMapKeyList == null) {
    +        lastMapKeyList = keyList;
    +      } else {
    +        if (keyList.size() != lastMapKeyList.size() || !lastMapKeyList.containsAll(keyList)) {
    +          throw new UnsupportedOperationException("Maps have different fields");
    +        }
    +      }
    +    }
    +  }
    +
    +  private void buildOutputContainer() throws SchemaChangeException {
    +    dataSrcVecMap = Maps.newHashMap();
    +    copySrcVecMap = Maps.newHashMap();
    +    MajorType mt = null;
    +
    +    ErrorCollector collector = new ErrorCollectorImpl();
    +    GregorianCalendar calendar = new GregorianCalendar(TimeZone.getTimeZone("UTC"));
    +
    +    calendar.setTimeInMillis(System.currentTimeMillis());
    +    createKeyColumn("schema", ValueExpressions.getBigInt(schema++), copySrcVecMap);
    +    createKeyColumn("computed", ValueExpressions.getDate(calendar), copySrcVecMap);
    +
    +    for (VectorWrapper<?> vw : incoming) {
    +      addVectorToOutgoingContainer(vw.getField().getLastName(), vw, collector);
    +      /*MaterializedField ds = vw.getField();
    +      String field = vw.getField().getLastName();
    +      // Input map vector
    +      MapVector mapVector = (MapVector) vw.getValueVector();
    +      assert mapVector.getPrimitiveVectors().size() > 0;
    +      // Proceed to create output map vector with same name e.g. statcount etc.
    +      mt = mapVector.getField().getType();
    +      MaterializedField mf = MaterializedField.create(functions.get(field), mt);
    +      assert !dataSrcVecMap.containsKey(mf);
    +      ValueVector vector = TypeHelper.getNewVector(mf, oContext.getAllocator());
    +      container.add(vector);
    +      MapVector outputMapVector = (MapVector) vector;
    +
    +      for (ValueVector vv : mapVector) {
    +        String fieldName = vv.getField().getLastName();
    +        if (!keyList.contains(fieldName)) {
    +          throw new UnsupportedOperationException("Unpivot data vector " +
    +              ds + " contains key " + fieldName + " not contained in key source!");
    +        }
    +        if (vv.getField().getType().getMinorType() == TypeProtos.MinorType.MAP) {
    +          throw new UnsupportedOperationException("Unpivot of nested map is not supported!");
    +        }
    +        if (field.equals("column")) {
    +          outputMapVector.addOrGet(fieldName, vv.getField().getType(), vv.getClass());
    +        } else {
    +          List<LogicalExpression> args = Lists.newArrayList();
    +          //TODO: Something else to access value of col such as emp_id?
    +          args.add(SchemaPath.getSimplePath(vv.getField().getPath()));
    +          //TODO: Put in the mapVector
    +          LogicalExpression call = FunctionCallFactory.createExpression(functions.get(field), args);
    +          //TODO: Is this sufficient to add to new Map?
    +          ValueVector vector1 = addMapVector(fieldName, outputMapVector, call);
    +          if (collector.hasErrors()) {
    +            throw new SchemaChangeException("Failure while materializing expression. "
    +                + collector.toErrorString());
    +          }
    +        }
    +      }
    +      dataSrcVecMap.put(ds, outputMapVector);*/
    +    }
    +    //Now create NDV in the outgoing container which was not avaliable in the incoming
    +    for (VectorWrapper<?> vw : incoming) {
    +      if (vw.getField().getLastName().equalsIgnoreCase("sum_width")) {//NullableFloat8 type
    +        addVectorToOutgoingContainer("ndv", vw, collector);
    +        break;
    +      }
    +    }
    +    container.setRecordCount(0);
    +    recordCount = 0;
    +    container.buildSchema(incoming.getSchema().getSelectionVectorMode());
    +  }
    +
    +  private void addVectorToOutgoingContainer(String field, VectorWrapper vw, ErrorCollector collector)
    +     throws SchemaChangeException {
    +    // Input map vector
    +    MapVector mapVector = (MapVector) vw.getValueVector();
    +    MaterializedField mf;
    +    assert mapVector.getPrimitiveVectors().size() > 0;
    +    // Proceed to create output map vector with same name e.g. statcount etc.
    +    MajorType mt = mapVector.getField().getType();
    +    if (functions.get(field) != null) {
    +      mf = MaterializedField.create(functions.get(field), mt);
    +    } else {
    +      mf = MaterializedField.create(field, mt);
    +    }
    +    assert !dataSrcVecMap.containsKey(mf);
    +    ValueVector vector = TypeHelper.getNewVector(mf, oContext.getAllocator());
    +    container.add(vector);
    +    MapVector outputMapVector = (MapVector) vector;
    +
    +    for (ValueVector vv : mapVector) {
    +      String fieldName = vv.getField().getLastName();
    +      if (!keyList.contains(fieldName)) {
    +        throw new UnsupportedOperationException("Unpivot data vector " +
    +                field + " contains key " + fieldName + " not contained in key source!");
    +      }
    +      if (vv.getField().getType().getMinorType() == TypeProtos.MinorType.MAP) {
    +        throw new UnsupportedOperationException("Unpivot of nested map is not supported!");
    +      }
    +      if (field.equals("column")) {
    +        outputMapVector.addOrGet(fieldName, vv.getField().getType(), vv.getClass());
    +      } else {
    +        List<LogicalExpression> args = Lists.newArrayList();
    +        LogicalExpression call;
    +        //TODO: Something else to access value of col such as emp_id?
    +        args.add(SchemaPath.getSimplePath(vv.getField().getPath()));
    +        //TODO: Put in the mapVector
    +        if (functions.get(field) != null) {
    +          call = FunctionCallFactory.createExpression(functions.get(field), args);
    +        } else {
    +          call = FunctionCallFactory.createExpression(field, args);
    +        }
    +        //TODO: Is this sufficient to add to new Map?
    +        ValueVector vector1 = addMapVector(fieldName, outputMapVector, call);
    +        if (collector.hasErrors()) {
    +          throw new SchemaChangeException("Failure while materializing expression. "
    +                  + collector.toErrorString());
    +        }
    +      }
    +    }
    +  }
    +
    +  @Override
    +  protected boolean setupNewSchema() throws SchemaChangeException {
    +    container.clear();
    +    buildKeyList();
    +    buildOutputContainer();
    +    return true;
    +  }
    +
    +  @Override
    +  protected IterOutcome doWork() {
    +    int outRecordCount = incoming.getRecordCount();
    +    HashMap<String, Long> nonNullRows = new HashMap<>();
    +
    +    for (VectorWrapper<?> vw : incoming) {
    +      ValueVector vv = vw.getValueVector();
    +      if (vv.getField().getType().getMinorType() != TypeProtos.MinorType.MAP) {
    +        //We only expect Maps in the incoming. STOP, if this is not the case.
    +        return IterOutcome.STOP;
    +      }
    +      MapVector mapVec = (MapVector) vv;
    +      Map<String, ValueHolder> statMap = aggregationMap.get(vv.getField().getLastName());
    +
    +      for (ValueVector mapElt : mapVec) {
    +        if (vv.getField().getLastName().equalsIgnoreCase("column")) {
    +          BigIntHolder nameHolder;
    +          if (statMap.get(mapElt.getField().getLastName()) != null) {
    +            nameHolder = (BigIntHolder) statMap.get(mapElt.getField().getLastName());
    +          } else {
    +            nameHolder = new BigIntHolder();
    +            statMap.put(mapElt.getField().getLastName(), nameHolder);
    +          }
    +          nameHolder.value = 1;
    +        } else if (vv.getField().getLastName().equalsIgnoreCase("statcount")
    +            || vv.getField().getLastName().equalsIgnoreCase("nonnullstatcount")) {
    +          BigIntHolder sumHolder;
    +          String colName = mapElt.getField().getLastName();
    +          if (statMap.get(colName) != null) {
    +            sumHolder = (BigIntHolder) statMap.get(colName);
    +          } else {
    +            sumHolder = new BigIntHolder();
    +            statMap.put(colName, sumHolder);
    +          }
    +          //TODO: assert size = 1
    +          //TODO: logger
    +          Object val = mapElt.getAccessor().getObject(0);
    +          if (val != null) {
    +            sumHolder.value += (long)val;
    +          }
    +        } else if (vv.getField().getLastName().equalsIgnoreCase("sum_width")) {
    +          NullableFloat8Holder sumHolder;
    +          String colName = mapElt.getField().getLastName();
    +          if (statMap == null) {
    +            statMap = aggregationMap.get(functions.get(vv.getField().getLastName()));
    +          }
    +          if (statMap.get(colName) != null) {
    +            sumHolder = (NullableFloat8Holder) statMap.get(colName);
    +          } else {
    +            sumHolder = new NullableFloat8Holder();
    +            statMap.put(colName, sumHolder);
    +          }
    +          //TODO: assert size = 1
    +          //TODO: logger
    +          Object val = mapElt.getAccessor().getObject(0);
    +          if (val != null) {
    +            sumHolder.value += (double) val;
    +            sumHolder.isSet = 1;
    +          }
    +        } else if (vv.getField().getLastName().equalsIgnoreCase("hll")) {
    --- End diff --
    
    Refactored the code here. Please take a look.


---
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 #729: Drill 1328: Support table statistics for Parquet

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

    https://github.com/apache/drill/pull/729#discussion_r103608069
  
    --- Diff: exec/java-exec/src/main/java/org/apache/drill/exec/planner/common/DrillRelOptUtil.java ---
    @@ -169,4 +175,43 @@ private static boolean containIdentity(List<? extends RexNode> exps,
         }
         return true;
       }
    +
    +  /**
    +   * Returns whether statistics-based estimates or guesses are used by the optimizer
    +   * */
    +  public static boolean guessRows(RelNode rel) {
    +    final PlannerSettings settings =
    +            rel.getCluster().getPlanner().getContext().unwrap(PlannerSettings.class);
    +    if (!settings.useStatistics()) {
    +      return true;
    +    }
    +    if (rel instanceof RelSubset) {
    --- End diff --
    
    Added comment explaining the special treatment.


---
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 #729: Drill 1328: Support table statistics for Parquet

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

    https://github.com/apache/drill/pull/729#discussion_r100709630
  
    --- Diff: exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/statistics/StatisticsAggBatch.java ---
    @@ -0,0 +1,256 @@
    +/**
    + * 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.impl.statistics;
    +
    +import com.google.common.collect.Lists;
    +import com.sun.codemodel.JExpr;
    +import org.apache.drill.common.expression.ErrorCollector;
    +import org.apache.drill.common.expression.ErrorCollectorImpl;
    +import org.apache.drill.common.expression.FunctionCallFactory;
    +import org.apache.drill.common.expression.LogicalExpression;
    +import org.apache.drill.common.expression.SchemaPath;
    +import org.apache.drill.common.expression.ValueExpressions;
    +import org.apache.drill.exec.exception.ClassTransformationException;
    +import org.apache.drill.exec.exception.OutOfMemoryException;
    +import org.apache.drill.exec.exception.SchemaChangeException;
    +import org.apache.drill.exec.expr.ClassGenerator;
    +import org.apache.drill.exec.expr.CodeGenerator;
    +import org.apache.drill.exec.expr.ExpressionTreeMaterializer;
    +import org.apache.drill.exec.expr.TypeHelper;
    +import org.apache.drill.exec.expr.ValueVectorWriteExpression;
    +import org.apache.drill.exec.ops.FragmentContext;
    +import org.apache.drill.exec.physical.config.StatisticsAggregate;
    +import org.apache.drill.exec.physical.impl.aggregate.StreamingAggBatch;
    +import org.apache.drill.exec.physical.impl.aggregate.StreamingAggTemplate;
    +import org.apache.drill.exec.physical.impl.aggregate.StreamingAggregator;
    +import org.apache.drill.exec.planner.physical.StatsAggPrel.OperatorPhase;
    +import org.apache.drill.exec.record.BatchSchema.SelectionVectorMode;
    +import org.apache.drill.exec.record.MaterializedField;
    +import org.apache.drill.exec.record.RecordBatch;
    +import org.apache.drill.exec.record.TypedFieldId;
    +import org.apache.drill.exec.store.ImplicitColumnExplorer;
    +import org.apache.drill.exec.vector.ValueVector;
    +import org.apache.drill.exec.vector.complex.FieldIdUtil;
    +import org.apache.drill.exec.vector.complex.MapVector;
    +
    +import java.io.IOException;
    +import java.util.GregorianCalendar;
    +import java.util.List;
    +import java.util.TimeZone;
    +
    +/**
    + * TODO: This needs cleanup. Currently the key values are constants and we compare the constants
    + * for every record. Seems unnecessary.
    + *
    + * Example input and output:
    + * Schema of incoming batch: region_id (VARCHAR), sales_city (VARCHAR), cnt (BIGINT)
    + * Schema of output:
    + *    "schema" : BIGINT - Schema number. For each schema change this number is incremented.
    + *    "computed" : BIGINT - What time is it computed?
    + *    "columns"       : MAP - Column names
    + *       "region_id"  : VARCHAR
    + *       "sales_city" : VARCHAR
    + *       "cnt"        : VARCHAR
    + *    "statscount" : MAP
    + *       "region_id"  : BIGINT - statscount(region_id) - aggregation over all values of region_id
    + *                      in incoming batch
    + *       "sales_city" : BIGINT - statscount(sales_city)
    + *       "cnt"        : BIGINT - statscount(cnt)
    + *    "nonnullstatcount" : MAP
    + *       "region_id"  : BIGINT - nonnullstatcount(region_id)
    + *       "sales_city" : BIGINT - nonnullstatcount(sales_city)
    + *       "cnt"        : BIGINT - nonnullstatcount(cnt)
    + *   .... another map for next stats function ....
    + */
    +public class StatisticsAggBatch extends StreamingAggBatch {
    +  private List<String> functions;
    +  private int schema = 0;
    +
    +  public StatisticsAggBatch(StatisticsAggregate popConfig, RecordBatch incoming,
    +      FragmentContext context) throws OutOfMemoryException {
    +    super(popConfig, incoming, context);
    +    this.functions = popConfig.getFunctions();
    +  }
    +
    +  private void createKeyColumn(String name, LogicalExpression expr, List<LogicalExpression> keyExprs,
    +      List<TypedFieldId> keyOutputIds) throws SchemaChangeException {
    +    ErrorCollector collector = new ErrorCollectorImpl();
    +
    +    LogicalExpression mle = ExpressionTreeMaterializer.materialize(expr, incoming, collector,
    +        context.getFunctionRegistry());
    +
    +    MaterializedField outputField = MaterializedField.create(name, mle.getMajorType());
    +    ValueVector vector = TypeHelper.getNewVector(outputField, oContext.getAllocator());
    +
    +    keyExprs.add(mle);
    +    keyOutputIds.add(container.add(vector));
    +
    +    if (collector.hasErrors()) {
    +      throw new SchemaChangeException("Failure while materializing expression. "
    +          + collector.toErrorString());
    +    }
    +  }
    +
    +  private void createNestedKeyColumn(MapVector parent, String name, LogicalExpression expr,
    +      List<LogicalExpression> keyExprs, List<TypedFieldId> keyOutputIds)
    +          throws SchemaChangeException {
    +    ErrorCollector collector = new ErrorCollectorImpl();
    +
    +    LogicalExpression mle = ExpressionTreeMaterializer.materialize(expr, incoming, collector,
    +        context.getFunctionRegistry());
    +
    +    Class<? extends ValueVector> vvc =
    +        TypeHelper.getValueVectorClass(mle.getMajorType().getMinorType(),
    +            mle.getMajorType().getMode());
    +
    +    ValueVector vv = parent.addOrGet(name, mle.getMajorType(), vvc);
    +
    +    TypedFieldId pfid = container.getValueVectorId(SchemaPath.getSimplePath(parent.getField().getPath()));
    +    assert pfid.getFieldIds().length == 1;
    +    TypedFieldId.Builder builder = TypedFieldId.newBuilder();
    +    builder.addId(pfid.getFieldIds()[0]);
    +    TypedFieldId id =
    +        FieldIdUtil.getFieldIdIfMatches(parent, builder, true,
    +            SchemaPath.getSimplePath(vv.getField().getPath()).getRootSegment());
    +
    +    keyExprs.add(mle);
    +    keyOutputIds.add(id);
    +
    +    if (collector.hasErrors()) {
    +      throw new SchemaChangeException("Failure while materializing expression. "
    +          + collector.toErrorString());
    +    }
    +  }
    +
    +  private void addMapVector(String name, MapVector parent, LogicalExpression expr,
    +      List<LogicalExpression> valueExprs) throws SchemaChangeException {
    +    ErrorCollector collector = new ErrorCollectorImpl();
    +
    +    LogicalExpression mle = ExpressionTreeMaterializer.materialize(expr, incoming, collector,
    +        context.getFunctionRegistry());
    +
    +    Class<? extends ValueVector> vvc =
    +        TypeHelper.getValueVectorClass(mle.getMajorType().getMinorType(), mle.getMajorType().getMode());
    +    ValueVector vv = parent.addOrGet(name, mle.getMajorType(), vvc);
    +
    +    TypedFieldId pfid = container.getValueVectorId(SchemaPath.getSimplePath(parent.getField().getPath()));
    +    assert pfid.getFieldIds().length == 1;
    +    TypedFieldId.Builder builder = TypedFieldId.newBuilder();
    +    builder.addId(pfid.getFieldIds()[0]);
    +    TypedFieldId id = FieldIdUtil.getFieldIdIfMatches(parent, builder, true,
    +        SchemaPath.getSimplePath(vv.getField().getPath()).getRootSegment());
    +
    +    valueExprs.add(new ValueVectorWriteExpression(id, mle, true));
    +
    +    if (collector.hasErrors()) {
    +      throw new SchemaChangeException("Failure while materializing expression. "
    +          + collector.toErrorString());
    +    }
    +  }
    +
    +  private StreamingAggregator codegenAggregator(List<LogicalExpression> keyExprs,
    +      List<LogicalExpression> valueExprs, List<TypedFieldId> keyOutputIds)
    +          throws SchemaChangeException, ClassTransformationException, IOException {
    +    ClassGenerator<StreamingAggregator> cg =
    +        CodeGenerator.getRoot(StreamingAggTemplate.TEMPLATE_DEFINITION, context.getFunctionRegistry(),
    +            context.getOptions());
    +
    +    LogicalExpression[] keyExprsArray = new LogicalExpression[keyExprs.size()];
    +    LogicalExpression[] valueExprsArray = new LogicalExpression[valueExprs.size()];
    +    TypedFieldId[] keyOutputIdsArray = new TypedFieldId[keyOutputIds.size()];
    +
    +    keyExprs.toArray(keyExprsArray);
    +    valueExprs.toArray(valueExprsArray);
    +    keyOutputIds.toArray(keyOutputIdsArray);
    +
    +    setupIsSame(cg, keyExprsArray);
    +    setupIsSameApart(cg, keyExprsArray);
    +    addRecordValues(cg, valueExprsArray);
    +    outputRecordKeys(cg, keyOutputIdsArray, keyExprsArray);
    +    outputRecordKeysPrev(cg, keyOutputIdsArray, keyExprsArray);
    +
    +    cg.getBlock("resetValues")._return(JExpr.TRUE);
    +    getIndex(cg);
    +
    +    container.buildSchema(SelectionVectorMode.NONE);
    +    StreamingAggregator agg = context.getImplementationClass(cg);
    +    agg.setup(oContext, incoming, this);
    +    return agg;
    +  }
    +
    +  private boolean isImplicitFileColumn(MaterializedField mf) {
    +    return ImplicitColumnExplorer.initImplicitFileColumns(context.getOptions()).get(mf.getName()) != null;
    +  }
    +
    +  protected StreamingAggregator createAggregatorInternal()
    +      throws SchemaChangeException, ClassTransformationException, IOException {
    +    container.clear();
    +
    +    List<LogicalExpression> keyExprs = Lists.newArrayList();
    +    List<LogicalExpression> valueExprs = Lists.newArrayList();
    +    List<TypedFieldId> keyOutputIds = Lists.newArrayList();
    +    GregorianCalendar calendar = new GregorianCalendar(TimeZone.getTimeZone("UTC"));
    +    calendar.setTimeInMillis(System.currentTimeMillis());
    +
    +    if (this.getPopConfig() instanceof StatisticsAggregate
    +        && (((StatisticsAggregate) this.getPopConfig()).getPhase() == OperatorPhase.PHASE_1of1
    +            || ((StatisticsAggregate) this.getPopConfig()).getPhase() == OperatorPhase.PHASE_2of2)) {
    +      createKeyColumn("schema",
    +          ValueExpressions.getBigInt(schema++),
    --- End diff --
    
    This code was refactored out (no longer present). This is an artifact from the previous code. We currently do not handle schema changes.


---
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 #729: Drill 1328: Support table statistics for Parquet

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/729#discussion_r100680958
  
    --- Diff: exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/statistics/StatisticsMergeBatch.java ---
    @@ -0,0 +1,532 @@
    +/**
    + * 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.impl.statistics;
    +
    +import com.google.common.collect.Lists;
    +import com.google.common.collect.Maps;
    +import org.apache.drill.common.expression.LogicalExpression;
    +import org.apache.drill.common.expression.ErrorCollector;
    +import org.apache.drill.common.expression.ErrorCollectorImpl;
    +import org.apache.drill.common.expression.FunctionCallFactory;
    +import org.apache.drill.common.expression.SchemaPath;
    +import org.apache.drill.common.expression.ValueExpressions;
    +import org.apache.drill.common.types.TypeProtos;
    +import org.apache.drill.common.types.TypeProtos.MajorType;
    +import org.apache.drill.exec.exception.OutOfMemoryException;
    +import org.apache.drill.exec.exception.SchemaChangeException;
    +import org.apache.drill.exec.expr.ExpressionTreeMaterializer;
    +import org.apache.drill.exec.expr.TypeHelper;
    +import org.apache.drill.exec.expr.holders.BigIntHolder;
    +import org.apache.drill.exec.expr.holders.NullableFloat8Holder;
    +import org.apache.drill.exec.expr.holders.ObjectHolder;
    +import org.apache.drill.exec.expr.holders.ValueHolder;
    +import org.apache.drill.exec.ops.FragmentContext;
    +import org.apache.drill.exec.physical.config.StatisticsMerge;
    +import org.apache.drill.exec.record.AbstractSingleRecordBatch;
    +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.VectorContainer;
    +import org.apache.drill.exec.record.VectorWrapper;
    +import org.apache.drill.exec.vector.ValueVector;
    +import org.apache.drill.exec.vector.DateVector;
    +import org.apache.drill.exec.vector.BigIntVector;
    +import org.apache.drill.exec.vector.NullableBigIntVector;
    +import org.apache.drill.exec.vector.NullableFloat8Vector;
    +import org.apache.drill.exec.vector.NullableVarBinaryVector;
    +import org.apache.drill.exec.vector.VarCharVector;
    +import org.apache.drill.exec.vector.complex.MapVector;
    +
    +import java.io.ByteArrayInputStream;
    +import java.io.DataInputStream;
    +import java.io.IOException;
    +import java.util.List;
    +import java.util.GregorianCalendar;
    +import java.util.HashMap;
    +import java.util.Map;
    +import java.util.TimeZone;
    +
    +import com.clearspring.analytics.stream.cardinality.HyperLogLog;
    +
    +public class StatisticsMergeBatch extends AbstractSingleRecordBatch<StatisticsMerge> {
    +  private static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(StatisticsMergeBatch.class);
    +  private Map<String, String> functions;
    +  private boolean first = true;
    +  private boolean finished = false;
    +  private int schema = 0;
    +  private int recordCount = 0;
    +  private List<String> keyList = null;
    +  private Map<MaterializedField, ValueVector> dataSrcVecMap = null;
    +  // Map of non-map fields to VV in the incoming schema
    +  private Map<MaterializedField, ValueVector> copySrcVecMap = null;
    +  private Map<String, Map<String, ValueHolder>> aggregationMap = null;
    +  public StatisticsMergeBatch(StatisticsMerge popConfig, RecordBatch incoming,
    +                              FragmentContext context) throws OutOfMemoryException {
    +    super(popConfig, context, incoming);
    +    this.functions = new HashMap<>();
    +    this.aggregationMap = new HashMap<>();
    +
    +    /*for (String key : popConfig.getFunctions()) {
    +      aggregationMap.put(key, new HashMap<String, ValueHolder>());
    +      if (key.equalsIgnoreCase("statcount") || key.equalsIgnoreCase("nonnullstatcount")) {
    +        functions.put(key, "sum");
    +      } else if (key.equalsIgnoreCase("hll")) {
    +        functions.put(key, "hll_merge");
    +      } else if (key.equalsIgnoreCase("sum_width")) {
    +        functions.put(key, "avg_width");
    +      }
    +    }*/
    +    for (String key : popConfig.getFunctions()) {
    +      if (key.equalsIgnoreCase("sum_width")) {
    +        functions.put(key, "avg_width");
    +      } else if (key.equalsIgnoreCase("hll")) {
    +        functions.put(key, "hll_merge");
    +      } else {
    +        functions.put(key, key);
    +      }
    +      aggregationMap.put(functions.get(key), new HashMap<String, ValueHolder>());
    +    }
    +  }
    +
    +  private void createKeyColumn(String name, LogicalExpression expr, Map<MaterializedField, ValueVector> parentMap)
    +      throws SchemaChangeException {
    +    ErrorCollector collector = new ErrorCollectorImpl();
    +
    +    LogicalExpression mle = ExpressionTreeMaterializer.materialize(expr, incoming, collector,
    +        context.getFunctionRegistry());
    +
    +    MaterializedField outputField = MaterializedField.create(name, mle.getMajorType());
    +    ValueVector vector = TypeHelper.getNewVector(outputField, oContext.getAllocator());
    +    container.add(vector);
    +
    +    if (collector.hasErrors()) {
    +      throw new SchemaChangeException("Failure while materializing expression. "
    +          + collector.toErrorString());
    +    }
    +    parentMap.put(outputField, vector);
    +  }
    +
    +  private ValueVector addMapVector(String name, MapVector parent, LogicalExpression expr) throws SchemaChangeException {
    +    ErrorCollector collector = new ErrorCollectorImpl();
    +
    +    LogicalExpression mle = ExpressionTreeMaterializer.materialize(expr, incoming, collector,
    +        context.getFunctionRegistry());
    +
    +    Class<? extends ValueVector> vvc =
    +        TypeHelper.getValueVectorClass(mle.getMajorType().getMinorType(), mle.getMajorType().getMode());
    +    ValueVector vector = parent.addOrGet(name, mle.getMajorType(), vvc);
    +
    +    if (collector.hasErrors()) {
    +      throw new SchemaChangeException("Failure while materializing expression. "
    +          + collector.toErrorString());
    +    }
    +    return vector;
    +  }
    +
    +  /**
    +   * Identify the list of fields within a map which are unpivoted as columns in output
    +   */
    +  private void buildKeyList() {
    +    List<String> lastMapKeyList = null;
    +    for (VectorWrapper<?> vw : incoming) {
    +      if (vw.getField().getType().getMinorType() != TypeProtos.MinorType.MAP) {
    +        continue;
    +      }
    +
    +      keyList = Lists.newArrayList();
    +
    +      for (ValueVector vv : vw.getValueVector()) {
    +        keyList.add(vv.getField().getLastName());
    +      }
    +
    +      if (lastMapKeyList == null) {
    +        lastMapKeyList = keyList;
    +      } else {
    +        if (keyList.size() != lastMapKeyList.size() || !lastMapKeyList.containsAll(keyList)) {
    +          throw new UnsupportedOperationException("Maps have different fields");
    +        }
    +      }
    +    }
    +  }
    +
    +  private void buildOutputContainer() throws SchemaChangeException {
    +    dataSrcVecMap = Maps.newHashMap();
    +    copySrcVecMap = Maps.newHashMap();
    +    MajorType mt = null;
    +
    +    ErrorCollector collector = new ErrorCollectorImpl();
    +    GregorianCalendar calendar = new GregorianCalendar(TimeZone.getTimeZone("UTC"));
    +
    +    calendar.setTimeInMillis(System.currentTimeMillis());
    +    createKeyColumn("schema", ValueExpressions.getBigInt(schema++), copySrcVecMap);
    +    createKeyColumn("computed", ValueExpressions.getDate(calendar), copySrcVecMap);
    +
    +    for (VectorWrapper<?> vw : incoming) {
    +      addVectorToOutgoingContainer(vw.getField().getLastName(), vw, collector);
    +      /*MaterializedField ds = vw.getField();
    +      String field = vw.getField().getLastName();
    +      // Input map vector
    +      MapVector mapVector = (MapVector) vw.getValueVector();
    +      assert mapVector.getPrimitiveVectors().size() > 0;
    +      // Proceed to create output map vector with same name e.g. statcount etc.
    +      mt = mapVector.getField().getType();
    +      MaterializedField mf = MaterializedField.create(functions.get(field), mt);
    +      assert !dataSrcVecMap.containsKey(mf);
    +      ValueVector vector = TypeHelper.getNewVector(mf, oContext.getAllocator());
    +      container.add(vector);
    +      MapVector outputMapVector = (MapVector) vector;
    +
    +      for (ValueVector vv : mapVector) {
    +        String fieldName = vv.getField().getLastName();
    +        if (!keyList.contains(fieldName)) {
    +          throw new UnsupportedOperationException("Unpivot data vector " +
    +              ds + " contains key " + fieldName + " not contained in key source!");
    +        }
    +        if (vv.getField().getType().getMinorType() == TypeProtos.MinorType.MAP) {
    +          throw new UnsupportedOperationException("Unpivot of nested map is not supported!");
    +        }
    +        if (field.equals("column")) {
    +          outputMapVector.addOrGet(fieldName, vv.getField().getType(), vv.getClass());
    +        } else {
    +          List<LogicalExpression> args = Lists.newArrayList();
    +          //TODO: Something else to access value of col such as emp_id?
    +          args.add(SchemaPath.getSimplePath(vv.getField().getPath()));
    +          //TODO: Put in the mapVector
    +          LogicalExpression call = FunctionCallFactory.createExpression(functions.get(field), args);
    +          //TODO: Is this sufficient to add to new Map?
    +          ValueVector vector1 = addMapVector(fieldName, outputMapVector, call);
    +          if (collector.hasErrors()) {
    +            throw new SchemaChangeException("Failure while materializing expression. "
    +                + collector.toErrorString());
    +          }
    +        }
    +      }
    +      dataSrcVecMap.put(ds, outputMapVector);*/
    +    }
    +    //Now create NDV in the outgoing container which was not avaliable in the incoming
    +    for (VectorWrapper<?> vw : incoming) {
    +      if (vw.getField().getLastName().equalsIgnoreCase("sum_width")) {//NullableFloat8 type
    +        addVectorToOutgoingContainer("ndv", vw, collector);
    +        break;
    +      }
    +    }
    +    container.setRecordCount(0);
    +    recordCount = 0;
    +    container.buildSchema(incoming.getSchema().getSelectionVectorMode());
    +  }
    +
    +  private void addVectorToOutgoingContainer(String field, VectorWrapper vw, ErrorCollector collector)
    +     throws SchemaChangeException {
    +    // Input map vector
    +    MapVector mapVector = (MapVector) vw.getValueVector();
    +    MaterializedField mf;
    +    assert mapVector.getPrimitiveVectors().size() > 0;
    +    // Proceed to create output map vector with same name e.g. statcount etc.
    +    MajorType mt = mapVector.getField().getType();
    +    if (functions.get(field) != null) {
    +      mf = MaterializedField.create(functions.get(field), mt);
    +    } else {
    +      mf = MaterializedField.create(field, mt);
    +    }
    +    assert !dataSrcVecMap.containsKey(mf);
    +    ValueVector vector = TypeHelper.getNewVector(mf, oContext.getAllocator());
    +    container.add(vector);
    +    MapVector outputMapVector = (MapVector) vector;
    +
    +    for (ValueVector vv : mapVector) {
    +      String fieldName = vv.getField().getLastName();
    +      if (!keyList.contains(fieldName)) {
    +        throw new UnsupportedOperationException("Unpivot data vector " +
    +                field + " contains key " + fieldName + " not contained in key source!");
    +      }
    +      if (vv.getField().getType().getMinorType() == TypeProtos.MinorType.MAP) {
    +        throw new UnsupportedOperationException("Unpivot of nested map is not supported!");
    +      }
    +      if (field.equals("column")) {
    +        outputMapVector.addOrGet(fieldName, vv.getField().getType(), vv.getClass());
    +      } else {
    +        List<LogicalExpression> args = Lists.newArrayList();
    +        LogicalExpression call;
    +        //TODO: Something else to access value of col such as emp_id?
    +        args.add(SchemaPath.getSimplePath(vv.getField().getPath()));
    +        //TODO: Put in the mapVector
    +        if (functions.get(field) != null) {
    +          call = FunctionCallFactory.createExpression(functions.get(field), args);
    +        } else {
    +          call = FunctionCallFactory.createExpression(field, args);
    +        }
    +        //TODO: Is this sufficient to add to new Map?
    +        ValueVector vector1 = addMapVector(fieldName, outputMapVector, call);
    +        if (collector.hasErrors()) {
    +          throw new SchemaChangeException("Failure while materializing expression. "
    +                  + collector.toErrorString());
    +        }
    +      }
    +    }
    +  }
    +
    +  @Override
    +  protected boolean setupNewSchema() throws SchemaChangeException {
    +    container.clear();
    +    buildKeyList();
    +    buildOutputContainer();
    +    return true;
    +  }
    +
    +  @Override
    +  protected IterOutcome doWork() {
    +    int outRecordCount = incoming.getRecordCount();
    +    HashMap<String, Long> nonNullRows = new HashMap<>();
    +
    +    for (VectorWrapper<?> vw : incoming) {
    +      ValueVector vv = vw.getValueVector();
    +      if (vv.getField().getType().getMinorType() != TypeProtos.MinorType.MAP) {
    +        //We only expect Maps in the incoming. STOP, if this is not the case.
    +        return IterOutcome.STOP;
    +      }
    +      MapVector mapVec = (MapVector) vv;
    +      Map<String, ValueHolder> statMap = aggregationMap.get(vv.getField().getLastName());
    +
    +      for (ValueVector mapElt : mapVec) {
    +        if (vv.getField().getLastName().equalsIgnoreCase("column")) {
    +          BigIntHolder nameHolder;
    +          if (statMap.get(mapElt.getField().getLastName()) != null) {
    +            nameHolder = (BigIntHolder) statMap.get(mapElt.getField().getLastName());
    +          } else {
    +            nameHolder = new BigIntHolder();
    +            statMap.put(mapElt.getField().getLastName(), nameHolder);
    +          }
    +          nameHolder.value = 1;
    +        } else if (vv.getField().getLastName().equalsIgnoreCase("statcount")
    +            || vv.getField().getLastName().equalsIgnoreCase("nonnullstatcount")) {
    +          BigIntHolder sumHolder;
    +          String colName = mapElt.getField().getLastName();
    +          if (statMap.get(colName) != null) {
    +            sumHolder = (BigIntHolder) statMap.get(colName);
    +          } else {
    +            sumHolder = new BigIntHolder();
    +            statMap.put(colName, sumHolder);
    +          }
    +          //TODO: assert size = 1
    +          //TODO: logger
    +          Object val = mapElt.getAccessor().getObject(0);
    +          if (val != null) {
    +            sumHolder.value += (long)val;
    +          }
    +        } else if (vv.getField().getLastName().equalsIgnoreCase("sum_width")) {
    +          NullableFloat8Holder sumHolder;
    +          String colName = mapElt.getField().getLastName();
    +          if (statMap == null) {
    +            statMap = aggregationMap.get(functions.get(vv.getField().getLastName()));
    +          }
    +          if (statMap.get(colName) != null) {
    +            sumHolder = (NullableFloat8Holder) statMap.get(colName);
    +          } else {
    +            sumHolder = new NullableFloat8Holder();
    +            statMap.put(colName, sumHolder);
    +          }
    +          //TODO: assert size = 1
    +          //TODO: logger
    +          Object val = mapElt.getAccessor().getObject(0);
    +          if (val != null) {
    +            sumHolder.value += (double) val;
    +            sumHolder.isSet = 1;
    +          }
    +        } else if (vv.getField().getLastName().equalsIgnoreCase("hll")) {
    --- End diff --
    
    We are hard coding the behavior for each function. (Also the name.) We are doing this in a big if/then statement. Each block has lots of repeated code. Can we make this more modular?
    
    * At the very least, the body of each if statement should be a method so it is easier to follow.
    * Those methods should call other methods to do shared tasks. (Please, no copy & paste!)
    * Perhaps the behavior should be factored out into a function definition class to achieve the extensibility that appears to be built into other parts of the code.
    
    Overall, this frankly seems far too complex for what it is doing. Something seems to need a good rethink 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 #729: Drill 1328: Support table statistics for Parquet

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

    https://github.com/apache/drill/pull/729#discussion_r100703479
  
    --- Diff: exec/java-exec/src/main/java/org/apache/drill/exec/physical/config/StatisticsAggregate.java ---
    @@ -0,0 +1,73 @@
    +/**
    + * 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.config;
    +
    +import org.apache.drill.exec.physical.base.PhysicalOperator;
    +import org.apache.drill.exec.physical.base.PhysicalVisitor;
    +import org.apache.drill.exec.planner.physical.StatsAggPrel.OperatorPhase;
    +import org.apache.drill.exec.proto.UserBitShared.CoreOperatorType;
    +
    +import com.fasterxml.jackson.annotation.JsonCreator;
    +import com.fasterxml.jackson.annotation.JsonProperty;
    +import com.fasterxml.jackson.annotation.JsonTypeName;
    +
    +import com.google.common.collect.ImmutableList;
    +
    +import java.util.List;
    +
    +@JsonTypeName("statistics-aggregate")
    +public class StatisticsAggregate extends StreamingAggregate {
    +  // private static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(StatisticsAggregate.class);
    +  protected OperatorPhase phase = OperatorPhase.PHASE_1of1;  // default phase
    --- End diff --
    
    The operator phase is not required. Removed it.


---
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 #729: Drill 1328: Support table statistics for Parquet

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

    https://github.com/apache/drill/pull/729#discussion_r100707129
  
    --- Diff: exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/statistics/StatisticsAggBatch.java ---
    @@ -0,0 +1,256 @@
    +/**
    + * 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.impl.statistics;
    +
    +import com.google.common.collect.Lists;
    +import com.sun.codemodel.JExpr;
    +import org.apache.drill.common.expression.ErrorCollector;
    +import org.apache.drill.common.expression.ErrorCollectorImpl;
    +import org.apache.drill.common.expression.FunctionCallFactory;
    +import org.apache.drill.common.expression.LogicalExpression;
    +import org.apache.drill.common.expression.SchemaPath;
    +import org.apache.drill.common.expression.ValueExpressions;
    +import org.apache.drill.exec.exception.ClassTransformationException;
    +import org.apache.drill.exec.exception.OutOfMemoryException;
    +import org.apache.drill.exec.exception.SchemaChangeException;
    +import org.apache.drill.exec.expr.ClassGenerator;
    +import org.apache.drill.exec.expr.CodeGenerator;
    +import org.apache.drill.exec.expr.ExpressionTreeMaterializer;
    +import org.apache.drill.exec.expr.TypeHelper;
    +import org.apache.drill.exec.expr.ValueVectorWriteExpression;
    +import org.apache.drill.exec.ops.FragmentContext;
    +import org.apache.drill.exec.physical.config.StatisticsAggregate;
    +import org.apache.drill.exec.physical.impl.aggregate.StreamingAggBatch;
    +import org.apache.drill.exec.physical.impl.aggregate.StreamingAggTemplate;
    +import org.apache.drill.exec.physical.impl.aggregate.StreamingAggregator;
    +import org.apache.drill.exec.planner.physical.StatsAggPrel.OperatorPhase;
    +import org.apache.drill.exec.record.BatchSchema.SelectionVectorMode;
    +import org.apache.drill.exec.record.MaterializedField;
    +import org.apache.drill.exec.record.RecordBatch;
    +import org.apache.drill.exec.record.TypedFieldId;
    +import org.apache.drill.exec.store.ImplicitColumnExplorer;
    +import org.apache.drill.exec.vector.ValueVector;
    +import org.apache.drill.exec.vector.complex.FieldIdUtil;
    +import org.apache.drill.exec.vector.complex.MapVector;
    +
    +import java.io.IOException;
    +import java.util.GregorianCalendar;
    +import java.util.List;
    +import java.util.TimeZone;
    +
    +/**
    + * TODO: This needs cleanup. Currently the key values are constants and we compare the constants
    + * for every record. Seems unnecessary.
    + *
    + * Example input and output:
    + * Schema of incoming batch: region_id (VARCHAR), sales_city (VARCHAR), cnt (BIGINT)
    + * Schema of output:
    + *    "schema" : BIGINT - Schema number. For each schema change this number is incremented.
    + *    "computed" : BIGINT - What time is it computed?
    + *    "columns"       : MAP - Column names
    + *       "region_id"  : VARCHAR
    + *       "sales_city" : VARCHAR
    + *       "cnt"        : VARCHAR
    + *    "statscount" : MAP
    + *       "region_id"  : BIGINT - statscount(region_id) - aggregation over all values of region_id
    + *                      in incoming batch
    + *       "sales_city" : BIGINT - statscount(sales_city)
    + *       "cnt"        : BIGINT - statscount(cnt)
    + *    "nonnullstatcount" : MAP
    --- End diff --
    
    Yes. No, the value would depend on the output type of the function. e.g. for NDV we use the HLL algorithm in which case the library generates a compact data structure representing the distinct values.


---
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 #729: Drill 1328: Support table statistics for Parquet

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

    https://github.com/apache/drill/pull/729#discussion_r103606988
  
    --- Diff: exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/statistics/MergedStatisticFactory.java ---
    @@ -0,0 +1,46 @@
    +/*
    + * 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.impl.statistics;
    +
    +public class MergedStatisticFactory {
    +  /*
    +   * Creates the appropriate statistics object given the name of the statistics and the input statistic
    +   */
    +  public static MergedStatistic getMergedStatistic(String outputStatName, String inputStatName) {
    +    if (outputStatName == null || inputStatName == null) {
    +      return null;
    +    } else if (outputStatName.equals(Statistic.COLNAME)) {
    +      return new ColumnMergedStatistic(outputStatName, inputStatName);
    +    } else if (outputStatName.equals(Statistic.COLTYPE)) {
    +      return new ColTypeMergedStatistic(outputStatName, inputStatName);
    +    } else if (outputStatName.equals(Statistic.STATCOUNT)) {
    +      return new StatCountMergedStatistic(outputStatName, inputStatName);
    +    } else if (outputStatName.equals(Statistic.NNSTATCOUNT)) {
    +      return new NNStatCountMergedStatistic(outputStatName, inputStatName);
    +    } else if (outputStatName.equals(Statistic.AVG_WIDTH)) {
    +      return new AvgWidthMergedStatistic(outputStatName, inputStatName);
    +    } else if (outputStatName.equals(Statistic.HLL_MERGE)) {
    +      return new HLLMergedStatistic(outputStatName, inputStatName);
    +    } else if (outputStatName.equals(Statistic.NDV)) {
    +      return new NDVMergedStatistic(outputStatName, inputStatName);
    --- End diff --
    
    Done using the factory approach you described above. Thanks so much for the suggestion.


---
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 #729: Drill 1328: Support table statistics for Parquet

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/729#discussion_r102873542
  
    --- Diff: exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/statistics/AvgWidthMergedStatistic.java ---
    @@ -0,0 +1,135 @@
    +/*
    + * 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.impl.statistics;
    +
    +import org.apache.drill.common.types.TypeProtos;
    +import org.apache.drill.exec.expr.holders.NullableFloat8Holder;
    +import org.apache.drill.exec.expr.holders.ValueHolder;
    +import org.apache.drill.exec.vector.NullableFloat8Vector;
    +import org.apache.drill.exec.vector.ValueVector;
    +import org.apache.drill.exec.vector.complex.MapVector;
    +
    +import java.util.HashMap;
    +import java.util.List;
    +import java.util.Map;
    +
    +public class AvgWidthMergedStatistic extends AbstractMergedStatistic {
    +
    +  private String name;
    +  private String inputName;
    --- End diff --
    
    Since multiple classes have member variables for name, inputName, and so on, go ahead and move them into the common base class to avoid duplication.


---
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 #729: Drill 1328: Support table statistics for Parquet

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/729#discussion_r102875234
  
    --- Diff: exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/statistics/Statistic.java ---
    @@ -0,0 +1,38 @@
    +/**
    --- End diff --
    
    /** --> /*
    The copyright notice need not be Javadoc.


---
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 #729: Drill 1328: Support table statistics for Parquet

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

    https://github.com/apache/drill/pull/729#discussion_r103371017
  
    --- Diff: exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/statistics/AvgWidthMergedStatistic.java ---
    @@ -0,0 +1,135 @@
    +/*
    + * 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.impl.statistics;
    +
    +import org.apache.drill.common.types.TypeProtos;
    +import org.apache.drill.exec.expr.holders.NullableFloat8Holder;
    +import org.apache.drill.exec.expr.holders.ValueHolder;
    +import org.apache.drill.exec.vector.NullableFloat8Vector;
    +import org.apache.drill.exec.vector.ValueVector;
    +import org.apache.drill.exec.vector.complex.MapVector;
    +
    +import java.util.HashMap;
    +import java.util.List;
    +import java.util.Map;
    +
    +public class AvgWidthMergedStatistic extends AbstractMergedStatistic {
    +
    +  private String name;
    +  private String inputName;
    --- 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 #729: Drill 1328: Support table statistics for Parquet

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/729#discussion_r100677415
  
    --- Diff: exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/statistics/StatisticsAggBatch.java ---
    @@ -0,0 +1,256 @@
    +/**
    + * 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.impl.statistics;
    +
    +import com.google.common.collect.Lists;
    +import com.sun.codemodel.JExpr;
    +import org.apache.drill.common.expression.ErrorCollector;
    +import org.apache.drill.common.expression.ErrorCollectorImpl;
    +import org.apache.drill.common.expression.FunctionCallFactory;
    +import org.apache.drill.common.expression.LogicalExpression;
    +import org.apache.drill.common.expression.SchemaPath;
    +import org.apache.drill.common.expression.ValueExpressions;
    +import org.apache.drill.exec.exception.ClassTransformationException;
    +import org.apache.drill.exec.exception.OutOfMemoryException;
    +import org.apache.drill.exec.exception.SchemaChangeException;
    +import org.apache.drill.exec.expr.ClassGenerator;
    +import org.apache.drill.exec.expr.CodeGenerator;
    +import org.apache.drill.exec.expr.ExpressionTreeMaterializer;
    +import org.apache.drill.exec.expr.TypeHelper;
    +import org.apache.drill.exec.expr.ValueVectorWriteExpression;
    +import org.apache.drill.exec.ops.FragmentContext;
    +import org.apache.drill.exec.physical.config.StatisticsAggregate;
    +import org.apache.drill.exec.physical.impl.aggregate.StreamingAggBatch;
    +import org.apache.drill.exec.physical.impl.aggregate.StreamingAggTemplate;
    +import org.apache.drill.exec.physical.impl.aggregate.StreamingAggregator;
    +import org.apache.drill.exec.planner.physical.StatsAggPrel.OperatorPhase;
    +import org.apache.drill.exec.record.BatchSchema.SelectionVectorMode;
    +import org.apache.drill.exec.record.MaterializedField;
    +import org.apache.drill.exec.record.RecordBatch;
    +import org.apache.drill.exec.record.TypedFieldId;
    +import org.apache.drill.exec.store.ImplicitColumnExplorer;
    +import org.apache.drill.exec.vector.ValueVector;
    +import org.apache.drill.exec.vector.complex.FieldIdUtil;
    +import org.apache.drill.exec.vector.complex.MapVector;
    +
    +import java.io.IOException;
    +import java.util.GregorianCalendar;
    +import java.util.List;
    +import java.util.TimeZone;
    +
    +/**
    + * TODO: This needs cleanup. Currently the key values are constants and we compare the constants
    + * for every record. Seems unnecessary.
    + *
    + * Example input and output:
    + * Schema of incoming batch: region_id (VARCHAR), sales_city (VARCHAR), cnt (BIGINT)
    + * Schema of output:
    + *    "schema" : BIGINT - Schema number. For each schema change this number is incremented.
    + *    "computed" : BIGINT - What time is it computed?
    + *    "columns"       : MAP - Column names
    + *       "region_id"  : VARCHAR
    + *       "sales_city" : VARCHAR
    + *       "cnt"        : VARCHAR
    + *    "statscount" : MAP
    + *       "region_id"  : BIGINT - statscount(region_id) - aggregation over all values of region_id
    + *                      in incoming batch
    + *       "sales_city" : BIGINT - statscount(sales_city)
    + *       "cnt"        : BIGINT - statscount(cnt)
    + *    "nonnullstatcount" : MAP
    + *       "region_id"  : BIGINT - nonnullstatcount(region_id)
    + *       "sales_city" : BIGINT - nonnullstatcount(sales_city)
    + *       "cnt"        : BIGINT - nonnullstatcount(cnt)
    + *   .... another map for next stats function ....
    + */
    +public class StatisticsAggBatch extends StreamingAggBatch {
    +  private List<String> functions;
    +  private int schema = 0;
    +
    +  public StatisticsAggBatch(StatisticsAggregate popConfig, RecordBatch incoming,
    +      FragmentContext context) throws OutOfMemoryException {
    +    super(popConfig, incoming, context);
    +    this.functions = popConfig.getFunctions();
    +  }
    +
    +  private void createKeyColumn(String name, LogicalExpression expr, List<LogicalExpression> keyExprs,
    +      List<TypedFieldId> keyOutputIds) throws SchemaChangeException {
    +    ErrorCollector collector = new ErrorCollectorImpl();
    +
    +    LogicalExpression mle = ExpressionTreeMaterializer.materialize(expr, incoming, collector,
    +        context.getFunctionRegistry());
    +
    +    MaterializedField outputField = MaterializedField.create(name, mle.getMajorType());
    +    ValueVector vector = TypeHelper.getNewVector(outputField, oContext.getAllocator());
    +
    +    keyExprs.add(mle);
    +    keyOutputIds.add(container.add(vector));
    +
    +    if (collector.hasErrors()) {
    +      throw new SchemaChangeException("Failure while materializing expression. "
    +          + collector.toErrorString());
    +    }
    +  }
    +
    +  private void createNestedKeyColumn(MapVector parent, String name, LogicalExpression expr,
    +      List<LogicalExpression> keyExprs, List<TypedFieldId> keyOutputIds)
    +          throws SchemaChangeException {
    +    ErrorCollector collector = new ErrorCollectorImpl();
    +
    +    LogicalExpression mle = ExpressionTreeMaterializer.materialize(expr, incoming, collector,
    +        context.getFunctionRegistry());
    +
    +    Class<? extends ValueVector> vvc =
    +        TypeHelper.getValueVectorClass(mle.getMajorType().getMinorType(),
    +            mle.getMajorType().getMode());
    +
    +    ValueVector vv = parent.addOrGet(name, mle.getMajorType(), vvc);
    +
    +    TypedFieldId pfid = container.getValueVectorId(SchemaPath.getSimplePath(parent.getField().getPath()));
    +    assert pfid.getFieldIds().length == 1;
    +    TypedFieldId.Builder builder = TypedFieldId.newBuilder();
    +    builder.addId(pfid.getFieldIds()[0]);
    +    TypedFieldId id =
    +        FieldIdUtil.getFieldIdIfMatches(parent, builder, true,
    +            SchemaPath.getSimplePath(vv.getField().getPath()).getRootSegment());
    +
    +    keyExprs.add(mle);
    +    keyOutputIds.add(id);
    +
    +    if (collector.hasErrors()) {
    +      throw new SchemaChangeException("Failure while materializing expression. "
    +          + collector.toErrorString());
    +    }
    +  }
    +
    +  private void addMapVector(String name, MapVector parent, LogicalExpression expr,
    +      List<LogicalExpression> valueExprs) throws SchemaChangeException {
    +    ErrorCollector collector = new ErrorCollectorImpl();
    +
    +    LogicalExpression mle = ExpressionTreeMaterializer.materialize(expr, incoming, collector,
    +        context.getFunctionRegistry());
    +
    +    Class<? extends ValueVector> vvc =
    +        TypeHelper.getValueVectorClass(mle.getMajorType().getMinorType(), mle.getMajorType().getMode());
    +    ValueVector vv = parent.addOrGet(name, mle.getMajorType(), vvc);
    +
    +    TypedFieldId pfid = container.getValueVectorId(SchemaPath.getSimplePath(parent.getField().getPath()));
    +    assert pfid.getFieldIds().length == 1;
    +    TypedFieldId.Builder builder = TypedFieldId.newBuilder();
    +    builder.addId(pfid.getFieldIds()[0]);
    +    TypedFieldId id = FieldIdUtil.getFieldIdIfMatches(parent, builder, true,
    +        SchemaPath.getSimplePath(vv.getField().getPath()).getRootSegment());
    +
    +    valueExprs.add(new ValueVectorWriteExpression(id, mle, true));
    +
    +    if (collector.hasErrors()) {
    +      throw new SchemaChangeException("Failure while materializing expression. "
    +          + collector.toErrorString());
    +    }
    +  }
    +
    +  private StreamingAggregator codegenAggregator(List<LogicalExpression> keyExprs,
    +      List<LogicalExpression> valueExprs, List<TypedFieldId> keyOutputIds)
    +          throws SchemaChangeException, ClassTransformationException, IOException {
    +    ClassGenerator<StreamingAggregator> cg =
    +        CodeGenerator.getRoot(StreamingAggTemplate.TEMPLATE_DEFINITION, context.getFunctionRegistry(),
    +            context.getOptions());
    +
    +    LogicalExpression[] keyExprsArray = new LogicalExpression[keyExprs.size()];
    +    LogicalExpression[] valueExprsArray = new LogicalExpression[valueExprs.size()];
    +    TypedFieldId[] keyOutputIdsArray = new TypedFieldId[keyOutputIds.size()];
    +
    +    keyExprs.toArray(keyExprsArray);
    +    valueExprs.toArray(valueExprsArray);
    +    keyOutputIds.toArray(keyOutputIdsArray);
    +
    +    setupIsSame(cg, keyExprsArray);
    +    setupIsSameApart(cg, keyExprsArray);
    +    addRecordValues(cg, valueExprsArray);
    +    outputRecordKeys(cg, keyOutputIdsArray, keyExprsArray);
    +    outputRecordKeysPrev(cg, keyOutputIdsArray, keyExprsArray);
    +
    +    cg.getBlock("resetValues")._return(JExpr.TRUE);
    +    getIndex(cg);
    +
    +    container.buildSchema(SelectionVectorMode.NONE);
    +    StreamingAggregator agg = context.getImplementationClass(cg);
    +    agg.setup(oContext, incoming, this);
    +    return agg;
    +  }
    +
    +  private boolean isImplicitFileColumn(MaterializedField mf) {
    +    return ImplicitColumnExplorer.initImplicitFileColumns(context.getOptions()).get(mf.getName()) != null;
    +  }
    +
    +  protected StreamingAggregator createAggregatorInternal()
    +      throws SchemaChangeException, ClassTransformationException, IOException {
    +    container.clear();
    +
    +    List<LogicalExpression> keyExprs = Lists.newArrayList();
    +    List<LogicalExpression> valueExprs = Lists.newArrayList();
    +    List<TypedFieldId> keyOutputIds = Lists.newArrayList();
    +    GregorianCalendar calendar = new GregorianCalendar(TimeZone.getTimeZone("UTC"));
    +    calendar.setTimeInMillis(System.currentTimeMillis());
    +
    +    if (this.getPopConfig() instanceof StatisticsAggregate
    +        && (((StatisticsAggregate) this.getPopConfig()).getPhase() == OperatorPhase.PHASE_1of1
    +            || ((StatisticsAggregate) this.getPopConfig()).getPhase() == OperatorPhase.PHASE_2of2)) {
    +      createKeyColumn("schema",
    +          ValueExpressions.getBigInt(schema++),
    +          keyExprs,
    +          keyOutputIds
    +      );
    +      createKeyColumn("computed",
    +          ValueExpressions.getDate(calendar),
    --- End diff --
    
    This seems a very convoluted way to get a timestamp.
    
    This also introduces a race condition. We set the date in each operator. When parallelized, each operator will have a time skew. Further, each node may have a small clock skew. Shouldn't the time be the start of the overall operation? Else, how do we deal with a variety of times?


---
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 #729: Drill 1328: Support table statistics for Parquet

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/729#discussion_r100681133
  
    --- Diff: exec/java-exec/src/main/java/org/apache/drill/exec/planner/common/DrillFilterRelBase.java ---
    @@ -113,6 +114,6 @@ public double getRows() {
             selectivity = filterMaxSelectivityEstimateFactor;
           }
         }
    -    return selectivity*RelMetadataQuery.getRowCount(getInput());
    +    return NumberUtil.multiply(selectivity, RelMetadataQuery.getRowCount(getInput()));
    --- End diff --
    
    Perhaps explain why we need a number utility to do a multiplication?


---
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 #729: Drill 1328: Support table statistics for Parquet

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/729#discussion_r100676876
  
    --- Diff: exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/aggregate/StreamingAggBatch.java ---
    @@ -82,11 +82,18 @@
       private boolean specialBatchSent = false;
       private static final int SPECIAL_BATCH_COUNT = 1;
     
    -  public StreamingAggBatch(StreamingAggregate popConfig, RecordBatch incoming, FragmentContext context) throws OutOfMemoryException {
    +  public StreamingAggBatch(StreamingAggregate popConfig, RecordBatch incoming, FragmentContext context)
    +      throws OutOfMemoryException {
         super(popConfig, context);
         this.incoming = incoming;
       }
     
    +  public StreamingAggBatch(StreamingAggregate popConfig, RecordBatch incoming, FragmentContext context,
    +                           final boolean buildSchema) throws OutOfMemoryException {
    --- End diff --
    
    Javadoc or plain comments are wonderful things. Perhaps we can use one here to explain the purpose of this constructor.
    
    A Record Batch, in Drill, is really an operator that happens (bizarrely) to also hold onto a record batch. As an operator, the streaming agg must build its schema from the incoming. Like all operators, it does so on the first batch and (perhaps) on each schema change event.
    
    Given that, why do we need a flag to tell this operator to build its schema? Are we trying to use the operator for something for which it was not intended?


---
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 #729: Drill 1328 r4

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

    https://github.com/apache/drill/pull/729#discussion_r98101328
  
    --- Diff: exec/java-exec/src/main/java/org/apache/drill/exec/planner/common/DrillRelOptUtil.java ---
    @@ -169,4 +175,43 @@ private static boolean containIdentity(List<? extends RexNode> exps,
         }
         return true;
       }
    +
    +  /**
    +   * Returns whether statistics-based estimates or guesses are used by the optimizer
    +   * */
    +  public static boolean guessRows(RelNode rel) {
    +    final PlannerSettings settings =
    +            rel.getCluster().getPlanner().getContext().unwrap(PlannerSettings.class);
    +    if (!settings.useStatistics()) {
    +      return true;
    +    }
    +    if (rel instanceof RelSubset) {
    --- End diff --
    
    It is unclear why RelSubset and HepRelVertex are treated in a special 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 #729: Drill 1328: Support table statistics for Parquet

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

    https://github.com/apache/drill/pull/729#discussion_r103636828
  
    --- Diff: exec/java-exec/src/main/java/org/apache/drill/exec/ops/ContextInformation.java ---
    @@ -29,13 +29,15 @@
       private final long queryStartTime;
       private final int rootFragmentTimeZone;
       private final String sessionId;
    +  private final int hllAccuracy;
     
       public ContextInformation(final UserCredentials userCredentials, final QueryContextInformation queryContextInfo) {
         this.queryUser = userCredentials.getUserName();
         this.currentDefaultSchema = queryContextInfo.getDefaultSchemaName();
         this.queryStartTime = queryContextInfo.getQueryStartTime();
         this.rootFragmentTimeZone = queryContextInfo.getTimeZone();
         this.sessionId = queryContextInfo.getSessionId();
    +    this.hllAccuracy = queryContextInfo.getHllAccuracy();
    --- End diff --
    
    This is not required for use within the `RecordBatch` but within the UDFs. The `ContextInformation` seems to be the existing mechanism to pass information to the UDFs.


---
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 #729: Drill 1328: Support table statistics for Parquet

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

    https://github.com/apache/drill/pull/729#discussion_r103636512
  
    --- Diff: protocol/src/main/java/org/apache/drill/exec/proto/beans/QueryContextInformation.java ---
    @@ -51,6 +51,7 @@ public static QueryContextInformation getDefaultInstance()
         private int timeZone;
         private String defaultSchemaName;
         private String sessionId;
    +    private int hllAccuracy;
    --- End diff --
    
    hllAccuracy is required within the HLL UDFs. The only way I found to pass information inside the UDFs is by injecting the `@Inject ContextInformation` available in the `UDFUtilities.java`.
    Hence, I define it in the QueryContext eventually passing it to the `ContextInformation`. Please suggest if there is a better way to do the same.hllAccuracy is required within the HLL UDFs. The only way I found to pass information inside the UDFs is by injecting the `@Inject ContextInformation` available in the `UDFUtilities.java`.
    Hence, I define it in the QueryContext eventually passing it to the `ContextInformation`. Please suggest if there is a better way to do the same.


---
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 #729: Drill 1328: Support table statistics for Parquet

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/729#discussion_r102867347
  
    --- Diff: exec/java-exec/src/main/java/org/apache/drill/exec/ops/QueryContext.java ---
    @@ -283,4 +288,22 @@ public void close() throws Exception {
           closed = true;
         }
       }
    +
    +  /**
    +  * @param stmtType : Sets the type {@link SqlStatementType} of the statement e.g. CTAS, ANALYZE
    +  */
    +  public void setSQLStatementType(SqlStatementType stmtType) {
    +    if (this.stmtType == null) {
    +      this.stmtType = stmtType;
    +    } else {
    +      throw new UnsupportedOperationException("SQL Statement type is already set");
    +    }
    +  }
    +
    +  /**
    +   * @return Get the type {@link SqlStatementType} of the statement e.g. CTAS, ANALYZE
    +   */
    +  public SqlStatementType getSQLStatementType() {
    +    return this.stmtType;
    --- End diff --
    
    No need for "this."


---
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 #729: Drill 1328: Support table statistics for Parquet

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

    https://github.com/apache/drill/pull/729#discussion_r101155927
  
    --- Diff: exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/statistics/StatisticsAggBatch.java ---
    @@ -0,0 +1,256 @@
    +/**
    + * 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.impl.statistics;
    +
    +import com.google.common.collect.Lists;
    +import com.sun.codemodel.JExpr;
    +import org.apache.drill.common.expression.ErrorCollector;
    +import org.apache.drill.common.expression.ErrorCollectorImpl;
    +import org.apache.drill.common.expression.FunctionCallFactory;
    +import org.apache.drill.common.expression.LogicalExpression;
    +import org.apache.drill.common.expression.SchemaPath;
    +import org.apache.drill.common.expression.ValueExpressions;
    +import org.apache.drill.exec.exception.ClassTransformationException;
    +import org.apache.drill.exec.exception.OutOfMemoryException;
    +import org.apache.drill.exec.exception.SchemaChangeException;
    +import org.apache.drill.exec.expr.ClassGenerator;
    +import org.apache.drill.exec.expr.CodeGenerator;
    +import org.apache.drill.exec.expr.ExpressionTreeMaterializer;
    +import org.apache.drill.exec.expr.TypeHelper;
    +import org.apache.drill.exec.expr.ValueVectorWriteExpression;
    +import org.apache.drill.exec.ops.FragmentContext;
    +import org.apache.drill.exec.physical.config.StatisticsAggregate;
    +import org.apache.drill.exec.physical.impl.aggregate.StreamingAggBatch;
    +import org.apache.drill.exec.physical.impl.aggregate.StreamingAggTemplate;
    +import org.apache.drill.exec.physical.impl.aggregate.StreamingAggregator;
    +import org.apache.drill.exec.planner.physical.StatsAggPrel.OperatorPhase;
    +import org.apache.drill.exec.record.BatchSchema.SelectionVectorMode;
    +import org.apache.drill.exec.record.MaterializedField;
    +import org.apache.drill.exec.record.RecordBatch;
    +import org.apache.drill.exec.record.TypedFieldId;
    +import org.apache.drill.exec.store.ImplicitColumnExplorer;
    +import org.apache.drill.exec.vector.ValueVector;
    +import org.apache.drill.exec.vector.complex.FieldIdUtil;
    +import org.apache.drill.exec.vector.complex.MapVector;
    +
    +import java.io.IOException;
    +import java.util.GregorianCalendar;
    +import java.util.List;
    +import java.util.TimeZone;
    +
    +/**
    + * TODO: This needs cleanup. Currently the key values are constants and we compare the constants
    + * for every record. Seems unnecessary.
    + *
    + * Example input and output:
    + * Schema of incoming batch: region_id (VARCHAR), sales_city (VARCHAR), cnt (BIGINT)
    + * Schema of output:
    + *    "schema" : BIGINT - Schema number. For each schema change this number is incremented.
    + *    "computed" : BIGINT - What time is it computed?
    + *    "columns"       : MAP - Column names
    + *       "region_id"  : VARCHAR
    + *       "sales_city" : VARCHAR
    + *       "cnt"        : VARCHAR
    + *    "statscount" : MAP
    + *       "region_id"  : BIGINT - statscount(region_id) - aggregation over all values of region_id
    + *                      in incoming batch
    + *       "sales_city" : BIGINT - statscount(sales_city)
    + *       "cnt"        : BIGINT - statscount(cnt)
    + *    "nonnullstatcount" : MAP
    + *       "region_id"  : BIGINT - nonnullstatcount(region_id)
    + *       "sales_city" : BIGINT - nonnullstatcount(sales_city)
    + *       "cnt"        : BIGINT - nonnullstatcount(cnt)
    + *   .... another map for next stats function ....
    --- End diff --
    
    We do not plan to support nested types in the current version. We can revisit the design in a subsequent version if we plan to support nested types.


---
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 #729: Drill 1328: Support table statistics for Parquet

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

    https://github.com/apache/drill/pull/729
  
    Thanks for the reminder @paul-rogers. Based on the last discussion with the reviewers and Drill community members, we would hold off on the PR because it also causes regressions in queries in TPC-H, TPC-DS benchmarks. We identified that we need histograms and other enhancements to fully address the regressions. I will post a new PR once these issues are addressed.


---
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 #729: Drill 1328: Support table statistics for Parquet

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/729#discussion_r100676304
  
    --- Diff: exec/java-exec/pom.xml ---
    @@ -410,6 +410,11 @@
           <artifactId>joda-time</artifactId>
           <version>2.9</version>
         </dependency>
    +    <dependency>
    +      <groupId>com.clearspring.analytics</groupId>
    +      <artifactId>stream</artifactId>
    +      <version>2.7.0</version>
    +    </dependency>
    --- End diff --
    
    Thanks, if it is already excluded they we are good.


---
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 #729: Drill 1328: Support table statistics for Parquet

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

    https://github.com/apache/drill/pull/729#discussion_r100445708
  
    --- Diff: exec/java-exec/src/main/java/org/apache/drill/exec/ops/FragmentContext.java ---
    @@ -245,6 +245,16 @@ public SchemaPlus getRootSchema() {
       }
     
       /**
    +   * Returns the statement type (e.g. SELECT, CTAS, ANALYZE) from the query context.
    +   * @return query statement type {@link QueryContext.StatementType}, if known.
    +   */
    +  public QueryContext.StatementType getStatementType() {
    +    if (queryContext != null) {
    --- End diff --
    
    From existing code (getRootSchema()), it seems like if we have a non-root fragment the query context maybe null. I am not sure about the the conditions - hence the paranoia check. I can throw an exception just like getRootSchema().


---
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 #729: Drill 1328: Support table statistics for Parquet

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/729#discussion_r102871725
  
    --- Diff: exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/statistics/AvgWidthMergedStatistic.java ---
    @@ -0,0 +1,135 @@
    +/*
    + * 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.impl.statistics;
    +
    +import org.apache.drill.common.types.TypeProtos;
    +import org.apache.drill.exec.expr.holders.NullableFloat8Holder;
    +import org.apache.drill.exec.expr.holders.ValueHolder;
    +import org.apache.drill.exec.vector.NullableFloat8Vector;
    +import org.apache.drill.exec.vector.ValueVector;
    +import org.apache.drill.exec.vector.complex.MapVector;
    +
    +import java.util.HashMap;
    +import java.util.List;
    +import java.util.Map;
    +
    +public class AvgWidthMergedStatistic extends AbstractMergedStatistic {
    +
    +  private String name;
    +  private String inputName;
    +  private boolean configureComplete = false;
    +  private boolean mergeComplete = false;
    +  private Map<String, ValueHolder> sumHolder;
    +  MergedStatistic types, nonNullStatCounts, statCounts;
    +
    +  public AvgWidthMergedStatistic (String name, String inputName) {
    +    this.name = name;
    +    this.inputName = inputName;
    +    this.sumHolder = new HashMap<>();
    +    types = nonNullStatCounts = statCounts = null;
    +  }
    +
    +  @Override
    +  public String getName() {
    +    return name;
    +  }
    +
    +  @Override
    +  public String getInput() {
    +    return inputName;
    +  }
    +
    +  @Override
    +  public void merge(ValueVector input) {
    +    // Check the input is a Map Vector
    +    assert (input.getField().getType().getMinorType() == TypeProtos.MinorType.MAP);
    +    MapVector inputMap = (MapVector) input;
    +    for (ValueVector vv : inputMap) {
    +      String colName = vv.getField().getLastName();
    +      NullableFloat8Holder colSumHolder;
    +      if (sumHolder.get(colName) != null) {
    +        colSumHolder = (NullableFloat8Holder) sumHolder.get(colName);
    +      } else {
    +        colSumHolder = new NullableFloat8Holder();
    +        sumHolder.put(colName, colSumHolder);
    +      }
    +      Object val = vv.getAccessor().getObject(0);
    +      if (val != null) {
    +        colSumHolder.value += (double) val;
    +        colSumHolder.isSet = 1;
    +      }
    +    }
    +  }
    +
    +  @Override
    +  public Object getStat(String colName) {
    +      if (mergeComplete != true) {
    +        throw new IllegalStateException(
    +            String.format("Statistic `%s` has not completed merging statistics", name));
    +      }
    +      NullableFloat8Holder colSumHolder = (NullableFloat8Holder) sumHolder.get(colName);
    +      return (long) (colSumHolder.value/ getRowCount(colName));
    +    }
    +
    +  @Override
    +  public void setOutput(ValueVector output) {
    +    // Check the input is a Map Vector
    +    assert (output.getField().getType().getMinorType() == TypeProtos.MinorType.MAP);
    +    // Dependencies have been configured correctly
    +    assert (configureComplete == true);
    +    MapVector outputMap = (MapVector) output;
    +
    +    for (ValueVector outMapCol : outputMap) {
    +      String colName = outMapCol.getField().getLastName();
    +      NullableFloat8Holder colSumHolder = (NullableFloat8Holder) sumHolder.get(colName);
    +      NullableFloat8Vector vv = (NullableFloat8Vector) outMapCol;
    +      vv.allocateNewSafe();
    +      vv.getMutator().setSafe(0, (colSumHolder.value / getRowCount(colName)));
    +    }
    +    mergeComplete = true;
    +  }
    +
    +  @Override
    +  public void configure(Object configurations) {
    +    List<MergedStatistic> statistics = (List<MergedStatistic>) configurations;
    +    for (MergedStatistic statistic : statistics) {
    +      if (statistic.getName().equals("type")) {
    +        types = statistic;
    +      } else if (statistic.getName().equals("statcount")) {
    +        statCounts = statistic;
    +      } else if (statistic.getName().equals("nonnullstatcount")) {
    +        nonNullStatCounts = statistic;
    +      }
    +    }
    +    assert (types != null && statCounts != null && nonNullStatCounts != null);
    +    configureComplete = true;
    +  }
    +
    +  private long getRowCount(String colName) {
    +    int type = (int) types.getStat(colName);
    --- End diff --
    
    Please use the enum value here: MinorType (or TypeProtos.MinorType, but Java will import the nested type for you so the code is simpler.)
    
    Java allows code of the form:
    
    ```
    if (type == MinorType.VAR16CHAR || type == ... ) {
    ```
    
    or, more simply:
    
    ```
    switch(type) {
    case VAR16CHAR:
    case VARCHAR:
    case VARBINARY:
      return ...
    default:
      return ...
    }
    ```


---
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 #729: Drill 1328: Support table statistics for Parquet

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/729#discussion_r102874413
  
    --- Diff: exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/statistics/MergedStatisticFactory.java ---
    @@ -0,0 +1,46 @@
    +/*
    + * 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.impl.statistics;
    +
    +public class MergedStatisticFactory {
    --- End diff --
    
    If the goal is to use only the static function, then:
    ```
    // Can't instantiate
    private MergedStatisticFactory() { }
    ```


---
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 #729: Drill 1328: Support table statistics for Parquet

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

    https://github.com/apache/drill/pull/729#discussion_r103608235
  
    --- Diff: exec/java-exec/src/main/java/org/apache/drill/exec/ExecConstants.java ---
    @@ -390,4 +391,15 @@
     
       String DYNAMIC_UDF_SUPPORT_ENABLED = "exec.udf.enable_dynamic_support";
       BooleanValidator DYNAMIC_UDF_SUPPORT_ENABLED_VALIDATOR = new BooleanValidator(DYNAMIC_UDF_SUPPORT_ENABLED, true, true);
    +
    +  /**
    +   * Option whose value is a long value representing the number of bits required for computing ndv (using HLL)
    +   */
    +  LongValidator NDV_MEMORY_LIMIT = new PositiveLongValidator("exec.statistics.ndv_memory_limit", 30, 20);
    +
    +  /**
    +   * Option whose value represents the current version of the statistics. Decreasing the value will generate
    +   * the older version of statistics
    +   */
    +  LongValidator STATISTICS_VERSION = new NonNegativeLongValidator("exec.statistics.capability_version", 1, 1);
    --- End diff --
    
    Also, the version in the option will change to the latest for any changes to the statistics. However, this option will give users flexibility for scenarios described in the earlier commit.


---
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 #729: Drill 1328: Support table statistics for Parquet

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

    https://github.com/apache/drill/pull/729#discussion_r102319897
  
    --- Diff: exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/statistics/StatisticsMergeBatch.java ---
    @@ -0,0 +1,532 @@
    +/**
    + * 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.impl.statistics;
    +
    +import com.google.common.collect.Lists;
    +import com.google.common.collect.Maps;
    +import org.apache.drill.common.expression.LogicalExpression;
    +import org.apache.drill.common.expression.ErrorCollector;
    +import org.apache.drill.common.expression.ErrorCollectorImpl;
    +import org.apache.drill.common.expression.FunctionCallFactory;
    +import org.apache.drill.common.expression.SchemaPath;
    +import org.apache.drill.common.expression.ValueExpressions;
    +import org.apache.drill.common.types.TypeProtos;
    +import org.apache.drill.common.types.TypeProtos.MajorType;
    +import org.apache.drill.exec.exception.OutOfMemoryException;
    +import org.apache.drill.exec.exception.SchemaChangeException;
    +import org.apache.drill.exec.expr.ExpressionTreeMaterializer;
    +import org.apache.drill.exec.expr.TypeHelper;
    +import org.apache.drill.exec.expr.holders.BigIntHolder;
    +import org.apache.drill.exec.expr.holders.NullableFloat8Holder;
    +import org.apache.drill.exec.expr.holders.ObjectHolder;
    +import org.apache.drill.exec.expr.holders.ValueHolder;
    +import org.apache.drill.exec.ops.FragmentContext;
    +import org.apache.drill.exec.physical.config.StatisticsMerge;
    +import org.apache.drill.exec.record.AbstractSingleRecordBatch;
    +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.VectorContainer;
    +import org.apache.drill.exec.record.VectorWrapper;
    +import org.apache.drill.exec.vector.ValueVector;
    +import org.apache.drill.exec.vector.DateVector;
    +import org.apache.drill.exec.vector.BigIntVector;
    +import org.apache.drill.exec.vector.NullableBigIntVector;
    +import org.apache.drill.exec.vector.NullableFloat8Vector;
    +import org.apache.drill.exec.vector.NullableVarBinaryVector;
    +import org.apache.drill.exec.vector.VarCharVector;
    +import org.apache.drill.exec.vector.complex.MapVector;
    +
    +import java.io.ByteArrayInputStream;
    +import java.io.DataInputStream;
    +import java.io.IOException;
    +import java.util.List;
    +import java.util.GregorianCalendar;
    +import java.util.HashMap;
    +import java.util.Map;
    +import java.util.TimeZone;
    +
    +import com.clearspring.analytics.stream.cardinality.HyperLogLog;
    +
    +public class StatisticsMergeBatch extends AbstractSingleRecordBatch<StatisticsMerge> {
    +  private static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(StatisticsMergeBatch.class);
    +  private Map<String, String> functions;
    +  private boolean first = true;
    +  private boolean finished = false;
    +  private int schema = 0;
    +  private int recordCount = 0;
    +  private List<String> keyList = null;
    +  private Map<MaterializedField, ValueVector> dataSrcVecMap = null;
    +  // Map of non-map fields to VV in the incoming schema
    +  private Map<MaterializedField, ValueVector> copySrcVecMap = null;
    +  private Map<String, Map<String, ValueHolder>> aggregationMap = null;
    +  public StatisticsMergeBatch(StatisticsMerge popConfig, RecordBatch incoming,
    +                              FragmentContext context) throws OutOfMemoryException {
    +    super(popConfig, context, incoming);
    +    this.functions = new HashMap<>();
    +    this.aggregationMap = new HashMap<>();
    +
    +    /*for (String key : popConfig.getFunctions()) {
    +      aggregationMap.put(key, new HashMap<String, ValueHolder>());
    +      if (key.equalsIgnoreCase("statcount") || key.equalsIgnoreCase("nonnullstatcount")) {
    +        functions.put(key, "sum");
    +      } else if (key.equalsIgnoreCase("hll")) {
    +        functions.put(key, "hll_merge");
    +      } else if (key.equalsIgnoreCase("sum_width")) {
    +        functions.put(key, "avg_width");
    +      }
    +    }*/
    +    for (String key : popConfig.getFunctions()) {
    +      if (key.equalsIgnoreCase("sum_width")) {
    +        functions.put(key, "avg_width");
    +      } else if (key.equalsIgnoreCase("hll")) {
    +        functions.put(key, "hll_merge");
    +      } else {
    +        functions.put(key, key);
    +      }
    +      aggregationMap.put(functions.get(key), new HashMap<String, ValueHolder>());
    +    }
    +  }
    +
    +  private void createKeyColumn(String name, LogicalExpression expr, Map<MaterializedField, ValueVector> parentMap)
    +      throws SchemaChangeException {
    +    ErrorCollector collector = new ErrorCollectorImpl();
    +
    +    LogicalExpression mle = ExpressionTreeMaterializer.materialize(expr, incoming, collector,
    +        context.getFunctionRegistry());
    +
    +    MaterializedField outputField = MaterializedField.create(name, mle.getMajorType());
    +    ValueVector vector = TypeHelper.getNewVector(outputField, oContext.getAllocator());
    +    container.add(vector);
    +
    +    if (collector.hasErrors()) {
    +      throw new SchemaChangeException("Failure while materializing expression. "
    +          + collector.toErrorString());
    +    }
    +    parentMap.put(outputField, vector);
    +  }
    +
    +  private ValueVector addMapVector(String name, MapVector parent, LogicalExpression expr) throws SchemaChangeException {
    +    ErrorCollector collector = new ErrorCollectorImpl();
    +
    +    LogicalExpression mle = ExpressionTreeMaterializer.materialize(expr, incoming, collector,
    +        context.getFunctionRegistry());
    +
    +    Class<? extends ValueVector> vvc =
    +        TypeHelper.getValueVectorClass(mle.getMajorType().getMinorType(), mle.getMajorType().getMode());
    +    ValueVector vector = parent.addOrGet(name, mle.getMajorType(), vvc);
    +
    +    if (collector.hasErrors()) {
    +      throw new SchemaChangeException("Failure while materializing expression. "
    +          + collector.toErrorString());
    +    }
    +    return vector;
    +  }
    +
    +  /**
    +   * Identify the list of fields within a map which are unpivoted as columns in output
    +   */
    +  private void buildKeyList() {
    +    List<String> lastMapKeyList = null;
    +    for (VectorWrapper<?> vw : incoming) {
    +      if (vw.getField().getType().getMinorType() != TypeProtos.MinorType.MAP) {
    +        continue;
    +      }
    +
    +      keyList = Lists.newArrayList();
    +
    +      for (ValueVector vv : vw.getValueVector()) {
    +        keyList.add(vv.getField().getLastName());
    +      }
    +
    +      if (lastMapKeyList == null) {
    +        lastMapKeyList = keyList;
    +      } else {
    +        if (keyList.size() != lastMapKeyList.size() || !lastMapKeyList.containsAll(keyList)) {
    +          throw new UnsupportedOperationException("Maps have different fields");
    +        }
    +      }
    +    }
    +  }
    +
    +  private void buildOutputContainer() throws SchemaChangeException {
    +    dataSrcVecMap = Maps.newHashMap();
    +    copySrcVecMap = Maps.newHashMap();
    +    MajorType mt = null;
    +
    +    ErrorCollector collector = new ErrorCollectorImpl();
    +    GregorianCalendar calendar = new GregorianCalendar(TimeZone.getTimeZone("UTC"));
    +
    +    calendar.setTimeInMillis(System.currentTimeMillis());
    +    createKeyColumn("schema", ValueExpressions.getBigInt(schema++), copySrcVecMap);
    +    createKeyColumn("computed", ValueExpressions.getDate(calendar), copySrcVecMap);
    +
    +    for (VectorWrapper<?> vw : incoming) {
    +      addVectorToOutgoingContainer(vw.getField().getLastName(), vw, collector);
    +      /*MaterializedField ds = vw.getField();
    +      String field = vw.getField().getLastName();
    +      // Input map vector
    +      MapVector mapVector = (MapVector) vw.getValueVector();
    +      assert mapVector.getPrimitiveVectors().size() > 0;
    +      // Proceed to create output map vector with same name e.g. statcount etc.
    +      mt = mapVector.getField().getType();
    +      MaterializedField mf = MaterializedField.create(functions.get(field), mt);
    +      assert !dataSrcVecMap.containsKey(mf);
    +      ValueVector vector = TypeHelper.getNewVector(mf, oContext.getAllocator());
    +      container.add(vector);
    +      MapVector outputMapVector = (MapVector) vector;
    +
    +      for (ValueVector vv : mapVector) {
    +        String fieldName = vv.getField().getLastName();
    +        if (!keyList.contains(fieldName)) {
    +          throw new UnsupportedOperationException("Unpivot data vector " +
    +              ds + " contains key " + fieldName + " not contained in key source!");
    +        }
    +        if (vv.getField().getType().getMinorType() == TypeProtos.MinorType.MAP) {
    +          throw new UnsupportedOperationException("Unpivot of nested map is not supported!");
    +        }
    +        if (field.equals("column")) {
    +          outputMapVector.addOrGet(fieldName, vv.getField().getType(), vv.getClass());
    +        } else {
    +          List<LogicalExpression> args = Lists.newArrayList();
    +          //TODO: Something else to access value of col such as emp_id?
    +          args.add(SchemaPath.getSimplePath(vv.getField().getPath()));
    +          //TODO: Put in the mapVector
    +          LogicalExpression call = FunctionCallFactory.createExpression(functions.get(field), args);
    +          //TODO: Is this sufficient to add to new Map?
    +          ValueVector vector1 = addMapVector(fieldName, outputMapVector, call);
    +          if (collector.hasErrors()) {
    +            throw new SchemaChangeException("Failure while materializing expression. "
    +                + collector.toErrorString());
    +          }
    +        }
    +      }
    +      dataSrcVecMap.put(ds, outputMapVector);*/
    +    }
    +    //Now create NDV in the outgoing container which was not avaliable in the incoming
    +    for (VectorWrapper<?> vw : incoming) {
    +      if (vw.getField().getLastName().equalsIgnoreCase("sum_width")) {//NullableFloat8 type
    +        addVectorToOutgoingContainer("ndv", vw, collector);
    +        break;
    +      }
    +    }
    +    container.setRecordCount(0);
    +    recordCount = 0;
    +    container.buildSchema(incoming.getSchema().getSelectionVectorMode());
    +  }
    +
    +  private void addVectorToOutgoingContainer(String field, VectorWrapper vw, ErrorCollector collector)
    +     throws SchemaChangeException {
    +    // Input map vector
    +    MapVector mapVector = (MapVector) vw.getValueVector();
    +    MaterializedField mf;
    +    assert mapVector.getPrimitiveVectors().size() > 0;
    +    // Proceed to create output map vector with same name e.g. statcount etc.
    +    MajorType mt = mapVector.getField().getType();
    +    if (functions.get(field) != null) {
    +      mf = MaterializedField.create(functions.get(field), mt);
    +    } else {
    +      mf = MaterializedField.create(field, mt);
    +    }
    +    assert !dataSrcVecMap.containsKey(mf);
    +    ValueVector vector = TypeHelper.getNewVector(mf, oContext.getAllocator());
    +    container.add(vector);
    +    MapVector outputMapVector = (MapVector) vector;
    +
    +    for (ValueVector vv : mapVector) {
    +      String fieldName = vv.getField().getLastName();
    +      if (!keyList.contains(fieldName)) {
    +        throw new UnsupportedOperationException("Unpivot data vector " +
    +                field + " contains key " + fieldName + " not contained in key source!");
    +      }
    +      if (vv.getField().getType().getMinorType() == TypeProtos.MinorType.MAP) {
    +        throw new UnsupportedOperationException("Unpivot of nested map is not supported!");
    +      }
    +      if (field.equals("column")) {
    +        outputMapVector.addOrGet(fieldName, vv.getField().getType(), vv.getClass());
    +      } else {
    +        List<LogicalExpression> args = Lists.newArrayList();
    +        LogicalExpression call;
    +        //TODO: Something else to access value of col such as emp_id?
    +        args.add(SchemaPath.getSimplePath(vv.getField().getPath()));
    +        //TODO: Put in the mapVector
    +        if (functions.get(field) != null) {
    +          call = FunctionCallFactory.createExpression(functions.get(field), args);
    +        } else {
    +          call = FunctionCallFactory.createExpression(field, args);
    +        }
    +        //TODO: Is this sufficient to add to new Map?
    +        ValueVector vector1 = addMapVector(fieldName, outputMapVector, call);
    +        if (collector.hasErrors()) {
    +          throw new SchemaChangeException("Failure while materializing expression. "
    +                  + collector.toErrorString());
    +        }
    +      }
    +    }
    +  }
    +
    +  @Override
    +  protected boolean setupNewSchema() throws SchemaChangeException {
    +    container.clear();
    +    buildKeyList();
    +    buildOutputContainer();
    +    return true;
    +  }
    +
    +  @Override
    +  protected IterOutcome doWork() {
    +    int outRecordCount = incoming.getRecordCount();
    +    HashMap<String, Long> nonNullRows = new HashMap<>();
    +
    +    for (VectorWrapper<?> vw : incoming) {
    +      ValueVector vv = vw.getValueVector();
    +      if (vv.getField().getType().getMinorType() != TypeProtos.MinorType.MAP) {
    +        //We only expect Maps in the incoming. STOP, if this is not the case.
    +        return IterOutcome.STOP;
    +      }
    +      MapVector mapVec = (MapVector) vv;
    +      Map<String, ValueHolder> statMap = aggregationMap.get(vv.getField().getLastName());
    +
    +      for (ValueVector mapElt : mapVec) {
    +        if (vv.getField().getLastName().equalsIgnoreCase("column")) {
    +          BigIntHolder nameHolder;
    +          if (statMap.get(mapElt.getField().getLastName()) != null) {
    +            nameHolder = (BigIntHolder) statMap.get(mapElt.getField().getLastName());
    +          } else {
    +            nameHolder = new BigIntHolder();
    +            statMap.put(mapElt.getField().getLastName(), nameHolder);
    +          }
    +          nameHolder.value = 1;
    +        } else if (vv.getField().getLastName().equalsIgnoreCase("statcount")
    +            || vv.getField().getLastName().equalsIgnoreCase("nonnullstatcount")) {
    +          BigIntHolder sumHolder;
    +          String colName = mapElt.getField().getLastName();
    +          if (statMap.get(colName) != null) {
    +            sumHolder = (BigIntHolder) statMap.get(colName);
    +          } else {
    +            sumHolder = new BigIntHolder();
    +            statMap.put(colName, sumHolder);
    +          }
    +          //TODO: assert size = 1
    +          //TODO: logger
    +          Object val = mapElt.getAccessor().getObject(0);
    +          if (val != null) {
    +            sumHolder.value += (long)val;
    +          }
    +        } else if (vv.getField().getLastName().equalsIgnoreCase("sum_width")) {
    +          NullableFloat8Holder sumHolder;
    +          String colName = mapElt.getField().getLastName();
    +          if (statMap == null) {
    +            statMap = aggregationMap.get(functions.get(vv.getField().getLastName()));
    +          }
    +          if (statMap.get(colName) != null) {
    +            sumHolder = (NullableFloat8Holder) statMap.get(colName);
    +          } else {
    +            sumHolder = new NullableFloat8Holder();
    +            statMap.put(colName, sumHolder);
    +          }
    +          //TODO: assert size = 1
    +          //TODO: logger
    +          Object val = mapElt.getAccessor().getObject(0);
    +          if (val != null) {
    +            sumHolder.value += (double) val;
    +            sumHolder.isSet = 1;
    +          }
    +        } else if (vv.getField().getLastName().equalsIgnoreCase("hll")) {
    +          ObjectHolder hllHolder;
    +          String colName = mapElt.getField().getLastName();
    +          if (statMap == null) {
    +            statMap = aggregationMap.get(functions.get(vv.getField().getLastName()));
    +          }
    +          if (statMap.get(colName) != null) {
    +            hllHolder = (ObjectHolder) statMap.get(colName);
    +          } else {
    +            hllHolder = new ObjectHolder();
    +            hllHolder.obj = new HyperLogLog(context.getContextInformation().getHllAccuracy());
    +            statMap.put(colName, hllHolder);
    +          }
    +          NullableVarBinaryVector hllVector = (NullableVarBinaryVector) mapElt;
    +          try {
    +            if (hllVector.getAccessor().isSet(0) == 1) {
    +              ByteArrayInputStream bais = new ByteArrayInputStream(hllVector.getAccessor().getObject(0), 0,
    +                  mapElt.getBufferSize());
    +              HyperLogLog other = HyperLogLog.Builder.build(new DataInputStream(bais));
    +              ((HyperLogLog) hllHolder.obj).addAll(other);
    +            }
    +          } catch (Exception ex) {
    +            //TODO: Catch IOException/CardinalityMergeException
    +            //TODO: logger
    +            return IterOutcome.STOP;
    +          }
    +        }
    +      }
    +      // Add NDV value vector map using HLL map (since the NDV map is directly generated from HLL and not produced by the underlying
    +      // Statistics Agg)
    +      Map<String, ValueHolder> hllMap = aggregationMap.get("hll");
    +      if (hllMap != null) {
    +        aggregationMap.put("ndv", hllMap);
    +      }
    +    }
    +    return IterOutcome.OK;
    +  }
    +
    +  public VectorContainer getOutgoingContainer() {
    +    return this.container;
    +  }
    +
    +  @Override
    +  public IterOutcome innerNext() {
    +    IterOutcome outcome;
    +    boolean didSomeWork = false;
    +    if (finished) {
    +      return IterOutcome.NONE;
    +    }
    +    try {
    +      outer: while (true) {
    +        outcome = next(incoming);
    +        switch (outcome) {
    +          case NONE:
    +            break outer;
    +          case OUT_OF_MEMORY:
    +          case NOT_YET:
    +          case STOP:
    +            return outcome;
    +          case OK_NEW_SCHEMA:
    +            if (first) {
    +              first =false;
    +              if (!setupNewSchema()) {
    +                outcome = IterOutcome.OK;
    +              }
    +              return outcome;
    +            }
    +            //fall through
    +          case OK:
    +            assert first == false : "First batch should be OK_NEW_SCHEMA";
    +            IterOutcome out = doWork();
    +            didSomeWork = true;
    +            if (out != IterOutcome.OK) {
    +              return out;
    +            }
    +            break;
    +          default:
    +            throw new UnsupportedOperationException("Unsupported upstream state " + outcome);
    +        }
    +      }
    +    } catch (SchemaChangeException ex) {
    +      kill(false);
    +      logger.error("Failure during query", ex);
    +      context.fail(ex);
    +      return IterOutcome.STOP;
    +    }
    +
    +    // We can only get here if upstream is NONE i.e. no more batches. If we did some work prior to
    +    // exhausting all upstream, then return OK. Otherwise, return NONE.
    +    if (didSomeWork) {
    +      IterOutcome out = buildOutgoingRecordBatch();
    --- End diff --
    
    For the parallel plan, we would get a record batch from each of the minor fragments. Hence, the multiple batches.


---
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 #729: Drill 1328: Support table statistics for Parquet

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

    https://github.com/apache/drill/pull/729#discussion_r103806768
  
    --- Diff: exec/java-exec/src/main/java/org/apache/drill/exec/ops/ContextInformation.java ---
    @@ -29,13 +29,15 @@
       private final long queryStartTime;
       private final int rootFragmentTimeZone;
       private final String sessionId;
    +  private final int hllAccuracy;
     
       public ContextInformation(final UserCredentials userCredentials, final QueryContextInformation queryContextInfo) {
         this.queryUser = userCredentials.getUserName();
         this.currentDefaultSchema = queryContextInfo.getDefaultSchemaName();
         this.queryStartTime = queryContextInfo.getQueryStartTime();
         this.rootFragmentTimeZone = queryContextInfo.getTimeZone();
         this.sessionId = queryContextInfo.getSessionId();
    +    this.hllAccuracy = queryContextInfo.getHllAccuracy();
    --- End diff --
    
    Perhaps consider doing an @Inject some new context such as StatsContext into the UDF. 


---
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 #729: Drill 1328: Support table statistics for Parquet

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

    https://github.com/apache/drill/pull/729#discussion_r102314383
  
    --- Diff: exec/java-exec/src/main/java/org/apache/drill/exec/planner/common/DrillStatsTable.java ---
    @@ -0,0 +1,347 @@
    +/**
    + * 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.planner.common;
    +
    +import java.io.IOException;
    +import java.util.List;
    +import java.util.Map;
    +import java.util.concurrent.TimeUnit;
    +import com.fasterxml.jackson.annotation.JsonIgnore;
    +import com.fasterxml.jackson.annotation.JsonGetter;
    +import com.fasterxml.jackson.annotation.JsonSetter;
    +import com.fasterxml.jackson.annotation.JsonProperty;
    +import com.fasterxml.jackson.annotation.JsonSubTypes;
    +import com.fasterxml.jackson.annotation.JsonTypeInfo;
    +import com.fasterxml.jackson.annotation.JsonTypeName;
    +import com.fasterxml.jackson.databind.DeserializationFeature;
    +import com.fasterxml.jackson.databind.ObjectMapper;
    +import com.google.common.base.Stopwatch;
    +import com.google.common.collect.Maps;
    +import org.apache.calcite.rel.RelNode;
    +import org.apache.calcite.rel.RelVisitor;
    +import org.apache.calcite.rel.core.TableScan;
    +import org.apache.drill.common.exceptions.DrillRuntimeException;
    +import org.apache.drill.exec.ops.QueryContext;
    +import org.apache.drill.exec.planner.logical.DrillTable;
    +import org.apache.drill.exec.util.ImpersonationUtil;
    +import org.apache.hadoop.fs.FSDataInputStream;
    +import org.apache.hadoop.fs.FileSystem;
    +import org.apache.hadoop.fs.Path;
    +import org.joda.time.DateTime;
    +
    +/**
    + * Wraps the stats table info including schema and tableName. Also materializes stats from storage
    + * and keeps them in memory.
    + */
    +public class DrillStatsTable {
    +  private static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(DrillStatsTable.class);
    +  private final FileSystem fs;
    +  private final Path tablePath;
    +
    +  /**
    +   * List of columns in stats table.
    +   */
    +  public static final String COL_COLUMN = "column";
    +  public static final String COL_COMPUTED = "computed";
    +  public static final String COL_STATCOUNT = "statcount";
    +  public static final String COL_NDV = "ndv";
    +
    +  private final String schemaName;
    +  private final String tableName;
    +
    +  private final Map<String, Long> ndv = Maps.newHashMap();
    +  private double rowCount = -1;
    +
    +  private boolean materialized = false;
    +
    +  private TableStatistics statistics = null;
    +
    +  public DrillStatsTable(String schemaName, String tableName, Path tablePath, FileSystem fs) {
    +    this.schemaName = schemaName;
    +    this.tableName = tableName;
    +    this.tablePath = tablePath;
    +    this.fs = ImpersonationUtil.createFileSystem(ImpersonationUtil.getProcessUserName(), fs.getConf());
    +  }
    +
    +  public String getSchemaName() {
    +    return schemaName;
    +  }
    +
    +  public String getTableName() {
    +    return tableName;
    +  }
    +  /**
    +   * Get number of distinct values of given column. If stats are not present for the given column,
    +   * a null is returned.
    +   *
    +   * Note: returned data may not be accurate. Accuracy depends on whether the table data has changed after the
    +   * stats are computed.
    +   *
    +   * @param col
    +   * @return
    +   */
    +  public Double getNdv(String col) {
    +    // Stats might not have materialized because of errors.
    +    if (!materialized) {
    +      return null;
    +    }
    +    final String upperCol = col.toUpperCase();
    +    final Long ndvCol = ndv.get(upperCol);
    +    // Ndv estimation techniques like HLL may over-estimate, hence cap it at rowCount
    +    if (ndvCol != null) {
    +      return Math.min(ndvCol, rowCount);
    --- End diff --
    
    Histograms would help with the data skew. When we have histograms, the NDV would be obtained from the Histograms. Stats will be off by default (so not as risky?), and the existing defaults also suffer from the same shortcoming.
    
    > Should we be more conservative? Set some minimum value? 
    
    How do we determine the minimum value? We would need to run experiments to determine the value.
    >Take a risk-based approach to deciding which side of hash join to be the build side?
    
    Sorry, I did not understand this. Maybe we can consider it as a followup. 


---
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 #729: Drill 1328: Support table statistics for Parquet

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

    https://github.com/apache/drill/pull/729#discussion_r100708749
  
    --- Diff: exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/statistics/StatisticsAggBatch.java ---
    @@ -0,0 +1,256 @@
    +/**
    + * 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.impl.statistics;
    +
    +import com.google.common.collect.Lists;
    +import com.sun.codemodel.JExpr;
    +import org.apache.drill.common.expression.ErrorCollector;
    +import org.apache.drill.common.expression.ErrorCollectorImpl;
    +import org.apache.drill.common.expression.FunctionCallFactory;
    +import org.apache.drill.common.expression.LogicalExpression;
    +import org.apache.drill.common.expression.SchemaPath;
    +import org.apache.drill.common.expression.ValueExpressions;
    +import org.apache.drill.exec.exception.ClassTransformationException;
    +import org.apache.drill.exec.exception.OutOfMemoryException;
    +import org.apache.drill.exec.exception.SchemaChangeException;
    +import org.apache.drill.exec.expr.ClassGenerator;
    +import org.apache.drill.exec.expr.CodeGenerator;
    +import org.apache.drill.exec.expr.ExpressionTreeMaterializer;
    +import org.apache.drill.exec.expr.TypeHelper;
    +import org.apache.drill.exec.expr.ValueVectorWriteExpression;
    +import org.apache.drill.exec.ops.FragmentContext;
    +import org.apache.drill.exec.physical.config.StatisticsAggregate;
    +import org.apache.drill.exec.physical.impl.aggregate.StreamingAggBatch;
    +import org.apache.drill.exec.physical.impl.aggregate.StreamingAggTemplate;
    +import org.apache.drill.exec.physical.impl.aggregate.StreamingAggregator;
    +import org.apache.drill.exec.planner.physical.StatsAggPrel.OperatorPhase;
    +import org.apache.drill.exec.record.BatchSchema.SelectionVectorMode;
    +import org.apache.drill.exec.record.MaterializedField;
    +import org.apache.drill.exec.record.RecordBatch;
    +import org.apache.drill.exec.record.TypedFieldId;
    +import org.apache.drill.exec.store.ImplicitColumnExplorer;
    +import org.apache.drill.exec.vector.ValueVector;
    +import org.apache.drill.exec.vector.complex.FieldIdUtil;
    +import org.apache.drill.exec.vector.complex.MapVector;
    +
    +import java.io.IOException;
    +import java.util.GregorianCalendar;
    +import java.util.List;
    +import java.util.TimeZone;
    +
    +/**
    + * TODO: This needs cleanup. Currently the key values are constants and we compare the constants
    + * for every record. Seems unnecessary.
    + *
    + * Example input and output:
    + * Schema of incoming batch: region_id (VARCHAR), sales_city (VARCHAR), cnt (BIGINT)
    + * Schema of output:
    + *    "schema" : BIGINT - Schema number. For each schema change this number is incremented.
    + *    "computed" : BIGINT - What time is it computed?
    + *    "columns"       : MAP - Column names
    + *       "region_id"  : VARCHAR
    + *       "sales_city" : VARCHAR
    + *       "cnt"        : VARCHAR
    + *    "statscount" : MAP
    + *       "region_id"  : BIGINT - statscount(region_id) - aggregation over all values of region_id
    + *                      in incoming batch
    + *       "sales_city" : BIGINT - statscount(sales_city)
    + *       "cnt"        : BIGINT - statscount(cnt)
    + *    "nonnullstatcount" : MAP
    + *       "region_id"  : BIGINT - nonnullstatcount(region_id)
    + *       "sales_city" : BIGINT - nonnullstatcount(sales_city)
    + *       "cnt"        : BIGINT - nonnullstatcount(cnt)
    + *   .... another map for next stats function ....
    + */
    +public class StatisticsAggBatch extends StreamingAggBatch {
    +  private List<String> functions;
    +  private int schema = 0;
    +
    +  public StatisticsAggBatch(StatisticsAggregate popConfig, RecordBatch incoming,
    +      FragmentContext context) throws OutOfMemoryException {
    +    super(popConfig, incoming, context);
    +    this.functions = popConfig.getFunctions();
    +  }
    +
    +  private void createKeyColumn(String name, LogicalExpression expr, List<LogicalExpression> keyExprs,
    +      List<TypedFieldId> keyOutputIds) throws SchemaChangeException {
    +    ErrorCollector collector = new ErrorCollectorImpl();
    +
    +    LogicalExpression mle = ExpressionTreeMaterializer.materialize(expr, incoming, collector,
    +        context.getFunctionRegistry());
    +
    +    MaterializedField outputField = MaterializedField.create(name, mle.getMajorType());
    +    ValueVector vector = TypeHelper.getNewVector(outputField, oContext.getAllocator());
    +
    +    keyExprs.add(mle);
    +    keyOutputIds.add(container.add(vector));
    +
    +    if (collector.hasErrors()) {
    +      throw new SchemaChangeException("Failure while materializing expression. "
    +          + collector.toErrorString());
    +    }
    +  }
    +
    +  private void createNestedKeyColumn(MapVector parent, String name, LogicalExpression expr,
    +      List<LogicalExpression> keyExprs, List<TypedFieldId> keyOutputIds)
    +          throws SchemaChangeException {
    +    ErrorCollector collector = new ErrorCollectorImpl();
    +
    +    LogicalExpression mle = ExpressionTreeMaterializer.materialize(expr, incoming, collector,
    +        context.getFunctionRegistry());
    +
    +    Class<? extends ValueVector> vvc =
    +        TypeHelper.getValueVectorClass(mle.getMajorType().getMinorType(),
    +            mle.getMajorType().getMode());
    +
    +    ValueVector vv = parent.addOrGet(name, mle.getMajorType(), vvc);
    +
    +    TypedFieldId pfid = container.getValueVectorId(SchemaPath.getSimplePath(parent.getField().getPath()));
    +    assert pfid.getFieldIds().length == 1;
    +    TypedFieldId.Builder builder = TypedFieldId.newBuilder();
    +    builder.addId(pfid.getFieldIds()[0]);
    +    TypedFieldId id =
    +        FieldIdUtil.getFieldIdIfMatches(parent, builder, true,
    +            SchemaPath.getSimplePath(vv.getField().getPath()).getRootSegment());
    +
    +    keyExprs.add(mle);
    +    keyOutputIds.add(id);
    +
    +    if (collector.hasErrors()) {
    +      throw new SchemaChangeException("Failure while materializing expression. "
    +          + collector.toErrorString());
    +    }
    +  }
    +
    +  private void addMapVector(String name, MapVector parent, LogicalExpression expr,
    +      List<LogicalExpression> valueExprs) throws SchemaChangeException {
    +    ErrorCollector collector = new ErrorCollectorImpl();
    +
    +    LogicalExpression mle = ExpressionTreeMaterializer.materialize(expr, incoming, collector,
    +        context.getFunctionRegistry());
    +
    +    Class<? extends ValueVector> vvc =
    +        TypeHelper.getValueVectorClass(mle.getMajorType().getMinorType(), mle.getMajorType().getMode());
    +    ValueVector vv = parent.addOrGet(name, mle.getMajorType(), vvc);
    +
    +    TypedFieldId pfid = container.getValueVectorId(SchemaPath.getSimplePath(parent.getField().getPath()));
    +    assert pfid.getFieldIds().length == 1;
    +    TypedFieldId.Builder builder = TypedFieldId.newBuilder();
    +    builder.addId(pfid.getFieldIds()[0]);
    +    TypedFieldId id = FieldIdUtil.getFieldIdIfMatches(parent, builder, true,
    +        SchemaPath.getSimplePath(vv.getField().getPath()).getRootSegment());
    +
    +    valueExprs.add(new ValueVectorWriteExpression(id, mle, true));
    +
    +    if (collector.hasErrors()) {
    +      throw new SchemaChangeException("Failure while materializing expression. "
    +          + collector.toErrorString());
    +    }
    +  }
    +
    +  private StreamingAggregator codegenAggregator(List<LogicalExpression> keyExprs,
    +      List<LogicalExpression> valueExprs, List<TypedFieldId> keyOutputIds)
    +          throws SchemaChangeException, ClassTransformationException, IOException {
    +    ClassGenerator<StreamingAggregator> cg =
    +        CodeGenerator.getRoot(StreamingAggTemplate.TEMPLATE_DEFINITION, context.getFunctionRegistry(),
    +            context.getOptions());
    +
    +    LogicalExpression[] keyExprsArray = new LogicalExpression[keyExprs.size()];
    +    LogicalExpression[] valueExprsArray = new LogicalExpression[valueExprs.size()];
    +    TypedFieldId[] keyOutputIdsArray = new TypedFieldId[keyOutputIds.size()];
    +
    +    keyExprs.toArray(keyExprsArray);
    +    valueExprs.toArray(valueExprsArray);
    +    keyOutputIds.toArray(keyOutputIdsArray);
    +
    +    setupIsSame(cg, keyExprsArray);
    +    setupIsSameApart(cg, keyExprsArray);
    +    addRecordValues(cg, valueExprsArray);
    +    outputRecordKeys(cg, keyOutputIdsArray, keyExprsArray);
    +    outputRecordKeysPrev(cg, keyOutputIdsArray, keyExprsArray);
    +
    +    cg.getBlock("resetValues")._return(JExpr.TRUE);
    +    getIndex(cg);
    +
    +    container.buildSchema(SelectionVectorMode.NONE);
    +    StreamingAggregator agg = context.getImplementationClass(cg);
    +    agg.setup(oContext, incoming, this);
    +    return agg;
    +  }
    +
    +  private boolean isImplicitFileColumn(MaterializedField mf) {
    +    return ImplicitColumnExplorer.initImplicitFileColumns(context.getOptions()).get(mf.getName()) != null;
    --- End diff --
    
    Yes, we should init once - done. However, we only have a few implicit columns so minor performance issue, if any?


---
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 #729: Drill 1328: Support table statistics for Parquet

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

    https://github.com/apache/drill/pull/729#discussion_r103407891
  
    --- Diff: exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/statistics/MergedStatisticFactory.java ---
    @@ -0,0 +1,46 @@
    +/*
    + * 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.impl.statistics;
    +
    +public class MergedStatisticFactory {
    +  /*
    +   * Creates the appropriate statistics object given the name of the statistics and the input statistic
    +   */
    +  public static MergedStatistic getMergedStatistic(String outputStatName, String inputStatName) {
    +    if (outputStatName == null || inputStatName == null) {
    +      return null;
    --- 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 #729: Drill 1328: Support table statistics for Parquet

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

    https://github.com/apache/drill/pull/729#discussion_r100414605
  
    --- Diff: exec/java-exec/src/main/java/org/apache/drill/exec/planner/common/DrillJoinRelBase.java ---
    @@ -99,9 +105,40 @@ public RelOptCost computeSelfCost(RelOptPlanner planner) {
       public double getRows() {
         if (this.condition.isAlwaysTrue()) {
           return joinRowFactor * this.getLeft().getRows() * this.getRight().getRows();
    -    } else {
    -      return joinRowFactor * Math.max(this.getLeft().getRows(), this.getRight().getRows());
         }
    +
    +    int[] joinFields = new int[2];
    +
    +    LogicalJoin jr = new LogicalJoin(
    +        this.getCluster(),
    +        this.getTraitSet(),
    +        this.getLeft(),
    +        this.getRight(),
    +        this.getCondition(),
    +        this.getJoinType(),
    +        this.getVariablesStopped(),
    +        false,
    +        ImmutableList.<RelDataTypeField>of());
    +
    +    if (!DrillRelOptUtil.guessRows(this)         //Statistics present for left and right side of the join
    +        && jr.getJoinType() == JoinRelType.INNER
    +        && DrillRelOptUtil.analyzeSimpleEquiJoin((Join)jr, joinFields)) {
    +      ImmutableBitSet leq = ImmutableBitSet.of(joinFields[0]);
    +      ImmutableBitSet req = ImmutableBitSet.of(joinFields[1]);
    +
    +      Double ldrc = RelMetadataQuery.getDistinctRowCount(this.getLeft(), leq, null);
    +      Double rdrc = RelMetadataQuery.getDistinctRowCount(this.getRight(), req, null);
    +
    +      Double lrc = RelMetadataQuery.getRowCount(this.getLeft());
    +      Double rrc = RelMetadataQuery.getRowCount(this.getRight());
    +
    +      if (ldrc != null && rdrc != null && lrc != null && rrc != null) {
    +        return (lrc * rrc) / Math.max(ldrc, rdrc);
    --- 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 #729: Drill 1328: Support table statistics for Parquet

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

    https://github.com/apache/drill/pull/729#discussion_r102326907
  
    --- Diff: exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/statistics/StatisticsMergeBatch.java ---
    @@ -0,0 +1,532 @@
    +/**
    + * 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.impl.statistics;
    +
    +import com.google.common.collect.Lists;
    +import com.google.common.collect.Maps;
    +import org.apache.drill.common.expression.LogicalExpression;
    +import org.apache.drill.common.expression.ErrorCollector;
    +import org.apache.drill.common.expression.ErrorCollectorImpl;
    +import org.apache.drill.common.expression.FunctionCallFactory;
    +import org.apache.drill.common.expression.SchemaPath;
    +import org.apache.drill.common.expression.ValueExpressions;
    +import org.apache.drill.common.types.TypeProtos;
    +import org.apache.drill.common.types.TypeProtos.MajorType;
    +import org.apache.drill.exec.exception.OutOfMemoryException;
    +import org.apache.drill.exec.exception.SchemaChangeException;
    +import org.apache.drill.exec.expr.ExpressionTreeMaterializer;
    +import org.apache.drill.exec.expr.TypeHelper;
    +import org.apache.drill.exec.expr.holders.BigIntHolder;
    +import org.apache.drill.exec.expr.holders.NullableFloat8Holder;
    +import org.apache.drill.exec.expr.holders.ObjectHolder;
    +import org.apache.drill.exec.expr.holders.ValueHolder;
    +import org.apache.drill.exec.ops.FragmentContext;
    +import org.apache.drill.exec.physical.config.StatisticsMerge;
    +import org.apache.drill.exec.record.AbstractSingleRecordBatch;
    +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.VectorContainer;
    +import org.apache.drill.exec.record.VectorWrapper;
    +import org.apache.drill.exec.vector.ValueVector;
    +import org.apache.drill.exec.vector.DateVector;
    +import org.apache.drill.exec.vector.BigIntVector;
    +import org.apache.drill.exec.vector.NullableBigIntVector;
    +import org.apache.drill.exec.vector.NullableFloat8Vector;
    +import org.apache.drill.exec.vector.NullableVarBinaryVector;
    +import org.apache.drill.exec.vector.VarCharVector;
    +import org.apache.drill.exec.vector.complex.MapVector;
    +
    +import java.io.ByteArrayInputStream;
    +import java.io.DataInputStream;
    +import java.io.IOException;
    +import java.util.List;
    +import java.util.GregorianCalendar;
    +import java.util.HashMap;
    +import java.util.Map;
    +import java.util.TimeZone;
    +
    +import com.clearspring.analytics.stream.cardinality.HyperLogLog;
    +
    +public class StatisticsMergeBatch extends AbstractSingleRecordBatch<StatisticsMerge> {
    +  private static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(StatisticsMergeBatch.class);
    +  private Map<String, String> functions;
    +  private boolean first = true;
    +  private boolean finished = false;
    +  private int schema = 0;
    +  private int recordCount = 0;
    +  private List<String> keyList = null;
    +  private Map<MaterializedField, ValueVector> dataSrcVecMap = null;
    +  // Map of non-map fields to VV in the incoming schema
    +  private Map<MaterializedField, ValueVector> copySrcVecMap = null;
    +  private Map<String, Map<String, ValueHolder>> aggregationMap = null;
    +  public StatisticsMergeBatch(StatisticsMerge popConfig, RecordBatch incoming,
    +                              FragmentContext context) throws OutOfMemoryException {
    +    super(popConfig, context, incoming);
    +    this.functions = new HashMap<>();
    +    this.aggregationMap = new HashMap<>();
    +
    +    /*for (String key : popConfig.getFunctions()) {
    +      aggregationMap.put(key, new HashMap<String, ValueHolder>());
    +      if (key.equalsIgnoreCase("statcount") || key.equalsIgnoreCase("nonnullstatcount")) {
    +        functions.put(key, "sum");
    +      } else if (key.equalsIgnoreCase("hll")) {
    +        functions.put(key, "hll_merge");
    +      } else if (key.equalsIgnoreCase("sum_width")) {
    +        functions.put(key, "avg_width");
    +      }
    +    }*/
    +    for (String key : popConfig.getFunctions()) {
    +      if (key.equalsIgnoreCase("sum_width")) {
    +        functions.put(key, "avg_width");
    +      } else if (key.equalsIgnoreCase("hll")) {
    +        functions.put(key, "hll_merge");
    +      } else {
    +        functions.put(key, key);
    +      }
    +      aggregationMap.put(functions.get(key), new HashMap<String, ValueHolder>());
    +    }
    +  }
    +
    +  private void createKeyColumn(String name, LogicalExpression expr, Map<MaterializedField, ValueVector> parentMap)
    +      throws SchemaChangeException {
    +    ErrorCollector collector = new ErrorCollectorImpl();
    +
    +    LogicalExpression mle = ExpressionTreeMaterializer.materialize(expr, incoming, collector,
    +        context.getFunctionRegistry());
    +
    +    MaterializedField outputField = MaterializedField.create(name, mle.getMajorType());
    +    ValueVector vector = TypeHelper.getNewVector(outputField, oContext.getAllocator());
    +    container.add(vector);
    +
    +    if (collector.hasErrors()) {
    +      throw new SchemaChangeException("Failure while materializing expression. "
    +          + collector.toErrorString());
    +    }
    +    parentMap.put(outputField, vector);
    +  }
    +
    +  private ValueVector addMapVector(String name, MapVector parent, LogicalExpression expr) throws SchemaChangeException {
    +    ErrorCollector collector = new ErrorCollectorImpl();
    +
    +    LogicalExpression mle = ExpressionTreeMaterializer.materialize(expr, incoming, collector,
    +        context.getFunctionRegistry());
    +
    +    Class<? extends ValueVector> vvc =
    +        TypeHelper.getValueVectorClass(mle.getMajorType().getMinorType(), mle.getMajorType().getMode());
    +    ValueVector vector = parent.addOrGet(name, mle.getMajorType(), vvc);
    +
    +    if (collector.hasErrors()) {
    +      throw new SchemaChangeException("Failure while materializing expression. "
    +          + collector.toErrorString());
    +    }
    +    return vector;
    +  }
    +
    +  /**
    +   * Identify the list of fields within a map which are unpivoted as columns in output
    +   */
    +  private void buildKeyList() {
    +    List<String> lastMapKeyList = null;
    +    for (VectorWrapper<?> vw : incoming) {
    +      if (vw.getField().getType().getMinorType() != TypeProtos.MinorType.MAP) {
    +        continue;
    +      }
    +
    +      keyList = Lists.newArrayList();
    +
    +      for (ValueVector vv : vw.getValueVector()) {
    +        keyList.add(vv.getField().getLastName());
    +      }
    +
    +      if (lastMapKeyList == null) {
    +        lastMapKeyList = keyList;
    +      } else {
    +        if (keyList.size() != lastMapKeyList.size() || !lastMapKeyList.containsAll(keyList)) {
    +          throw new UnsupportedOperationException("Maps have different fields");
    +        }
    +      }
    +    }
    +  }
    +
    +  private void buildOutputContainer() throws SchemaChangeException {
    +    dataSrcVecMap = Maps.newHashMap();
    +    copySrcVecMap = Maps.newHashMap();
    +    MajorType mt = null;
    +
    +    ErrorCollector collector = new ErrorCollectorImpl();
    +    GregorianCalendar calendar = new GregorianCalendar(TimeZone.getTimeZone("UTC"));
    +
    +    calendar.setTimeInMillis(System.currentTimeMillis());
    +    createKeyColumn("schema", ValueExpressions.getBigInt(schema++), copySrcVecMap);
    +    createKeyColumn("computed", ValueExpressions.getDate(calendar), copySrcVecMap);
    +
    +    for (VectorWrapper<?> vw : incoming) {
    +      addVectorToOutgoingContainer(vw.getField().getLastName(), vw, collector);
    +      /*MaterializedField ds = vw.getField();
    +      String field = vw.getField().getLastName();
    +      // Input map vector
    +      MapVector mapVector = (MapVector) vw.getValueVector();
    +      assert mapVector.getPrimitiveVectors().size() > 0;
    +      // Proceed to create output map vector with same name e.g. statcount etc.
    +      mt = mapVector.getField().getType();
    +      MaterializedField mf = MaterializedField.create(functions.get(field), mt);
    +      assert !dataSrcVecMap.containsKey(mf);
    +      ValueVector vector = TypeHelper.getNewVector(mf, oContext.getAllocator());
    +      container.add(vector);
    +      MapVector outputMapVector = (MapVector) vector;
    +
    +      for (ValueVector vv : mapVector) {
    +        String fieldName = vv.getField().getLastName();
    +        if (!keyList.contains(fieldName)) {
    +          throw new UnsupportedOperationException("Unpivot data vector " +
    +              ds + " contains key " + fieldName + " not contained in key source!");
    +        }
    +        if (vv.getField().getType().getMinorType() == TypeProtos.MinorType.MAP) {
    +          throw new UnsupportedOperationException("Unpivot of nested map is not supported!");
    +        }
    +        if (field.equals("column")) {
    +          outputMapVector.addOrGet(fieldName, vv.getField().getType(), vv.getClass());
    +        } else {
    +          List<LogicalExpression> args = Lists.newArrayList();
    +          //TODO: Something else to access value of col such as emp_id?
    +          args.add(SchemaPath.getSimplePath(vv.getField().getPath()));
    +          //TODO: Put in the mapVector
    +          LogicalExpression call = FunctionCallFactory.createExpression(functions.get(field), args);
    +          //TODO: Is this sufficient to add to new Map?
    +          ValueVector vector1 = addMapVector(fieldName, outputMapVector, call);
    +          if (collector.hasErrors()) {
    +            throw new SchemaChangeException("Failure while materializing expression. "
    +                + collector.toErrorString());
    +          }
    +        }
    +      }
    +      dataSrcVecMap.put(ds, outputMapVector);*/
    +    }
    +    //Now create NDV in the outgoing container which was not avaliable in the incoming
    +    for (VectorWrapper<?> vw : incoming) {
    +      if (vw.getField().getLastName().equalsIgnoreCase("sum_width")) {//NullableFloat8 type
    +        addVectorToOutgoingContainer("ndv", vw, collector);
    +        break;
    +      }
    +    }
    +    container.setRecordCount(0);
    +    recordCount = 0;
    +    container.buildSchema(incoming.getSchema().getSelectionVectorMode());
    +  }
    +
    +  private void addVectorToOutgoingContainer(String field, VectorWrapper vw, ErrorCollector collector)
    +     throws SchemaChangeException {
    +    // Input map vector
    +    MapVector mapVector = (MapVector) vw.getValueVector();
    +    MaterializedField mf;
    +    assert mapVector.getPrimitiveVectors().size() > 0;
    +    // Proceed to create output map vector with same name e.g. statcount etc.
    +    MajorType mt = mapVector.getField().getType();
    +    if (functions.get(field) != null) {
    +      mf = MaterializedField.create(functions.get(field), mt);
    +    } else {
    +      mf = MaterializedField.create(field, mt);
    +    }
    +    assert !dataSrcVecMap.containsKey(mf);
    +    ValueVector vector = TypeHelper.getNewVector(mf, oContext.getAllocator());
    +    container.add(vector);
    +    MapVector outputMapVector = (MapVector) vector;
    +
    +    for (ValueVector vv : mapVector) {
    +      String fieldName = vv.getField().getLastName();
    +      if (!keyList.contains(fieldName)) {
    +        throw new UnsupportedOperationException("Unpivot data vector " +
    +                field + " contains key " + fieldName + " not contained in key source!");
    +      }
    +      if (vv.getField().getType().getMinorType() == TypeProtos.MinorType.MAP) {
    +        throw new UnsupportedOperationException("Unpivot of nested map is not supported!");
    +      }
    +      if (field.equals("column")) {
    +        outputMapVector.addOrGet(fieldName, vv.getField().getType(), vv.getClass());
    +      } else {
    +        List<LogicalExpression> args = Lists.newArrayList();
    +        LogicalExpression call;
    +        //TODO: Something else to access value of col such as emp_id?
    +        args.add(SchemaPath.getSimplePath(vv.getField().getPath()));
    +        //TODO: Put in the mapVector
    +        if (functions.get(field) != null) {
    +          call = FunctionCallFactory.createExpression(functions.get(field), args);
    +        } else {
    +          call = FunctionCallFactory.createExpression(field, args);
    +        }
    +        //TODO: Is this sufficient to add to new Map?
    +        ValueVector vector1 = addMapVector(fieldName, outputMapVector, call);
    +        if (collector.hasErrors()) {
    +          throw new SchemaChangeException("Failure while materializing expression. "
    +                  + collector.toErrorString());
    +        }
    +      }
    +    }
    +  }
    +
    +  @Override
    +  protected boolean setupNewSchema() throws SchemaChangeException {
    +    container.clear();
    +    buildKeyList();
    +    buildOutputContainer();
    +    return true;
    +  }
    +
    +  @Override
    +  protected IterOutcome doWork() {
    +    int outRecordCount = incoming.getRecordCount();
    +    HashMap<String, Long> nonNullRows = new HashMap<>();
    +
    +    for (VectorWrapper<?> vw : incoming) {
    +      ValueVector vv = vw.getValueVector();
    +      if (vv.getField().getType().getMinorType() != TypeProtos.MinorType.MAP) {
    +        //We only expect Maps in the incoming. STOP, if this is not the case.
    +        return IterOutcome.STOP;
    +      }
    +      MapVector mapVec = (MapVector) vv;
    +      Map<String, ValueHolder> statMap = aggregationMap.get(vv.getField().getLastName());
    +
    +      for (ValueVector mapElt : mapVec) {
    +        if (vv.getField().getLastName().equalsIgnoreCase("column")) {
    +          BigIntHolder nameHolder;
    +          if (statMap.get(mapElt.getField().getLastName()) != null) {
    +            nameHolder = (BigIntHolder) statMap.get(mapElt.getField().getLastName());
    +          } else {
    +            nameHolder = new BigIntHolder();
    +            statMap.put(mapElt.getField().getLastName(), nameHolder);
    +          }
    +          nameHolder.value = 1;
    +        } else if (vv.getField().getLastName().equalsIgnoreCase("statcount")
    +            || vv.getField().getLastName().equalsIgnoreCase("nonnullstatcount")) {
    +          BigIntHolder sumHolder;
    +          String colName = mapElt.getField().getLastName();
    +          if (statMap.get(colName) != null) {
    +            sumHolder = (BigIntHolder) statMap.get(colName);
    +          } else {
    +            sumHolder = new BigIntHolder();
    +            statMap.put(colName, sumHolder);
    +          }
    +          //TODO: assert size = 1
    +          //TODO: logger
    +          Object val = mapElt.getAccessor().getObject(0);
    +          if (val != null) {
    +            sumHolder.value += (long)val;
    +          }
    +        } else if (vv.getField().getLastName().equalsIgnoreCase("sum_width")) {
    +          NullableFloat8Holder sumHolder;
    +          String colName = mapElt.getField().getLastName();
    +          if (statMap == null) {
    +            statMap = aggregationMap.get(functions.get(vv.getField().getLastName()));
    +          }
    +          if (statMap.get(colName) != null) {
    +            sumHolder = (NullableFloat8Holder) statMap.get(colName);
    +          } else {
    +            sumHolder = new NullableFloat8Holder();
    +            statMap.put(colName, sumHolder);
    +          }
    +          //TODO: assert size = 1
    +          //TODO: logger
    +          Object val = mapElt.getAccessor().getObject(0);
    +          if (val != null) {
    +            sumHolder.value += (double) val;
    +            sumHolder.isSet = 1;
    +          }
    +        } else if (vv.getField().getLastName().equalsIgnoreCase("hll")) {
    +          ObjectHolder hllHolder;
    +          String colName = mapElt.getField().getLastName();
    +          if (statMap == null) {
    +            statMap = aggregationMap.get(functions.get(vv.getField().getLastName()));
    +          }
    +          if (statMap.get(colName) != null) {
    +            hllHolder = (ObjectHolder) statMap.get(colName);
    +          } else {
    +            hllHolder = new ObjectHolder();
    +            hllHolder.obj = new HyperLogLog(context.getContextInformation().getHllAccuracy());
    +            statMap.put(colName, hllHolder);
    +          }
    +          NullableVarBinaryVector hllVector = (NullableVarBinaryVector) mapElt;
    +          try {
    +            if (hllVector.getAccessor().isSet(0) == 1) {
    +              ByteArrayInputStream bais = new ByteArrayInputStream(hllVector.getAccessor().getObject(0), 0,
    +                  mapElt.getBufferSize());
    +              HyperLogLog other = HyperLogLog.Builder.build(new DataInputStream(bais));
    +              ((HyperLogLog) hllHolder.obj).addAll(other);
    +            }
    +          } catch (Exception ex) {
    +            //TODO: Catch IOException/CardinalityMergeException
    +            //TODO: logger
    +            return IterOutcome.STOP;
    +          }
    +        }
    +      }
    +      // Add NDV value vector map using HLL map (since the NDV map is directly generated from HLL and not produced by the underlying
    +      // Statistics Agg)
    +      Map<String, ValueHolder> hllMap = aggregationMap.get("hll");
    +      if (hllMap != null) {
    +        aggregationMap.put("ndv", hllMap);
    +      }
    +    }
    +    return IterOutcome.OK;
    +  }
    +
    +  public VectorContainer getOutgoingContainer() {
    +    return this.container;
    +  }
    +
    +  @Override
    +  public IterOutcome innerNext() {
    +    IterOutcome outcome;
    +    boolean didSomeWork = false;
    +    if (finished) {
    +      return IterOutcome.NONE;
    +    }
    +    try {
    +      outer: while (true) {
    +        outcome = next(incoming);
    +        switch (outcome) {
    +          case NONE:
    +            break outer;
    +          case OUT_OF_MEMORY:
    +          case NOT_YET:
    +          case STOP:
    +            return outcome;
    +          case OK_NEW_SCHEMA:
    --- End diff --
    
    > The schema of the underlying data source might change. Does that then trigger a schema change for the stats? I suppose it must, if a new field appears in the original data, we add a field to the maps, which triggers a schema change.
    
    Yes, we trigger a schema change for this exact reason
    
    > But, should this occur? Do we need a schema change for stats when the underlying schema changes? (See comment about a list-based schema earlier.)
    
    The downstream operators may maintain state (e.g. list of columns to unpivot). If StatisticsMerge does not relay a schema change downstream operators may fail unexpectedly.


---
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 #729: Drill 1328: Support table statistics for Parquet

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/729#discussion_r102869973
  
    --- Diff: exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/statistics/AvgWidthMergedStatistic.java ---
    @@ -0,0 +1,135 @@
    +/*
    + * 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.impl.statistics;
    +
    +import org.apache.drill.common.types.TypeProtos;
    +import org.apache.drill.exec.expr.holders.NullableFloat8Holder;
    +import org.apache.drill.exec.expr.holders.ValueHolder;
    +import org.apache.drill.exec.vector.NullableFloat8Vector;
    +import org.apache.drill.exec.vector.ValueVector;
    +import org.apache.drill.exec.vector.complex.MapVector;
    +
    +import java.util.HashMap;
    +import java.util.List;
    +import java.util.Map;
    +
    +public class AvgWidthMergedStatistic extends AbstractMergedStatistic {
    +
    +  private String name;
    +  private String inputName;
    +  private boolean configureComplete = false;
    +  private boolean mergeComplete = false;
    +  private Map<String, ValueHolder> sumHolder;
    +  MergedStatistic types, nonNullStatCounts, statCounts;
    +
    +  public AvgWidthMergedStatistic (String name, String inputName) {
    +    this.name = name;
    +    this.inputName = inputName;
    +    this.sumHolder = new HashMap<>();
    +    types = nonNullStatCounts = statCounts = null;
    +  }
    +
    +  @Override
    +  public String getName() {
    +    return name;
    +  }
    +
    +  @Override
    +  public String getInput() {
    +    return inputName;
    +  }
    +
    +  @Override
    +  public void merge(ValueVector input) {
    +    // Check the input is a Map Vector
    +    assert (input.getField().getType().getMinorType() == TypeProtos.MinorType.MAP);
    +    MapVector inputMap = (MapVector) input;
    +    for (ValueVector vv : inputMap) {
    +      String colName = vv.getField().getLastName();
    +      NullableFloat8Holder colSumHolder;
    +      if (sumHolder.get(colName) != null) {
    +        colSumHolder = (NullableFloat8Holder) sumHolder.get(colName);
    +      } else {
    +        colSumHolder = new NullableFloat8Holder();
    +        sumHolder.put(colName, colSumHolder);
    +      }
    +      Object val = vv.getAccessor().getObject(0);
    +      if (val != null) {
    +        colSumHolder.value += (double) val;
    +        colSumHolder.isSet = 1;
    +      }
    +    }
    +  }
    +
    +  @Override
    +  public Object getStat(String colName) {
    --- End diff --
    
    Are all stats doubles? If so, return a double, not an Object.


---
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 #729: Drill 1328: Support table statistics for Parquet

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/729#discussion_r100680677
  
    --- Diff: exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/statistics/StatisticsMergeBatch.java ---
    @@ -0,0 +1,532 @@
    +/**
    + * 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.impl.statistics;
    +
    +import com.google.common.collect.Lists;
    +import com.google.common.collect.Maps;
    +import org.apache.drill.common.expression.LogicalExpression;
    +import org.apache.drill.common.expression.ErrorCollector;
    +import org.apache.drill.common.expression.ErrorCollectorImpl;
    +import org.apache.drill.common.expression.FunctionCallFactory;
    +import org.apache.drill.common.expression.SchemaPath;
    +import org.apache.drill.common.expression.ValueExpressions;
    +import org.apache.drill.common.types.TypeProtos;
    +import org.apache.drill.common.types.TypeProtos.MajorType;
    +import org.apache.drill.exec.exception.OutOfMemoryException;
    +import org.apache.drill.exec.exception.SchemaChangeException;
    +import org.apache.drill.exec.expr.ExpressionTreeMaterializer;
    +import org.apache.drill.exec.expr.TypeHelper;
    +import org.apache.drill.exec.expr.holders.BigIntHolder;
    +import org.apache.drill.exec.expr.holders.NullableFloat8Holder;
    +import org.apache.drill.exec.expr.holders.ObjectHolder;
    +import org.apache.drill.exec.expr.holders.ValueHolder;
    +import org.apache.drill.exec.ops.FragmentContext;
    +import org.apache.drill.exec.physical.config.StatisticsMerge;
    +import org.apache.drill.exec.record.AbstractSingleRecordBatch;
    +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.VectorContainer;
    +import org.apache.drill.exec.record.VectorWrapper;
    +import org.apache.drill.exec.vector.ValueVector;
    +import org.apache.drill.exec.vector.DateVector;
    +import org.apache.drill.exec.vector.BigIntVector;
    +import org.apache.drill.exec.vector.NullableBigIntVector;
    +import org.apache.drill.exec.vector.NullableFloat8Vector;
    +import org.apache.drill.exec.vector.NullableVarBinaryVector;
    +import org.apache.drill.exec.vector.VarCharVector;
    +import org.apache.drill.exec.vector.complex.MapVector;
    +
    +import java.io.ByteArrayInputStream;
    +import java.io.DataInputStream;
    +import java.io.IOException;
    +import java.util.List;
    +import java.util.GregorianCalendar;
    +import java.util.HashMap;
    +import java.util.Map;
    +import java.util.TimeZone;
    +
    +import com.clearspring.analytics.stream.cardinality.HyperLogLog;
    +
    +public class StatisticsMergeBatch extends AbstractSingleRecordBatch<StatisticsMerge> {
    +  private static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(StatisticsMergeBatch.class);
    +  private Map<String, String> functions;
    +  private boolean first = true;
    +  private boolean finished = false;
    +  private int schema = 0;
    +  private int recordCount = 0;
    +  private List<String> keyList = null;
    +  private Map<MaterializedField, ValueVector> dataSrcVecMap = null;
    +  // Map of non-map fields to VV in the incoming schema
    +  private Map<MaterializedField, ValueVector> copySrcVecMap = null;
    +  private Map<String, Map<String, ValueHolder>> aggregationMap = null;
    +  public StatisticsMergeBatch(StatisticsMerge popConfig, RecordBatch incoming,
    +                              FragmentContext context) throws OutOfMemoryException {
    +    super(popConfig, context, incoming);
    +    this.functions = new HashMap<>();
    +    this.aggregationMap = new HashMap<>();
    +
    +    /*for (String key : popConfig.getFunctions()) {
    +      aggregationMap.put(key, new HashMap<String, ValueHolder>());
    +      if (key.equalsIgnoreCase("statcount") || key.equalsIgnoreCase("nonnullstatcount")) {
    +        functions.put(key, "sum");
    +      } else if (key.equalsIgnoreCase("hll")) {
    +        functions.put(key, "hll_merge");
    +      } else if (key.equalsIgnoreCase("sum_width")) {
    +        functions.put(key, "avg_width");
    +      }
    +    }*/
    +    for (String key : popConfig.getFunctions()) {
    +      if (key.equalsIgnoreCase("sum_width")) {
    +        functions.put(key, "avg_width");
    +      } else if (key.equalsIgnoreCase("hll")) {
    +        functions.put(key, "hll_merge");
    +      } else {
    +        functions.put(key, key);
    +      }
    +      aggregationMap.put(functions.get(key), new HashMap<String, ValueHolder>());
    +    }
    +  }
    +
    +  private void createKeyColumn(String name, LogicalExpression expr, Map<MaterializedField, ValueVector> parentMap)
    +      throws SchemaChangeException {
    +    ErrorCollector collector = new ErrorCollectorImpl();
    +
    +    LogicalExpression mle = ExpressionTreeMaterializer.materialize(expr, incoming, collector,
    +        context.getFunctionRegistry());
    +
    +    MaterializedField outputField = MaterializedField.create(name, mle.getMajorType());
    +    ValueVector vector = TypeHelper.getNewVector(outputField, oContext.getAllocator());
    +    container.add(vector);
    +
    +    if (collector.hasErrors()) {
    +      throw new SchemaChangeException("Failure while materializing expression. "
    +          + collector.toErrorString());
    +    }
    +    parentMap.put(outputField, vector);
    +  }
    +
    +  private ValueVector addMapVector(String name, MapVector parent, LogicalExpression expr) throws SchemaChangeException {
    +    ErrorCollector collector = new ErrorCollectorImpl();
    +
    +    LogicalExpression mle = ExpressionTreeMaterializer.materialize(expr, incoming, collector,
    +        context.getFunctionRegistry());
    +
    +    Class<? extends ValueVector> vvc =
    +        TypeHelper.getValueVectorClass(mle.getMajorType().getMinorType(), mle.getMajorType().getMode());
    +    ValueVector vector = parent.addOrGet(name, mle.getMajorType(), vvc);
    +
    +    if (collector.hasErrors()) {
    +      throw new SchemaChangeException("Failure while materializing expression. "
    +          + collector.toErrorString());
    +    }
    +    return vector;
    +  }
    +
    +  /**
    +   * Identify the list of fields within a map which are unpivoted as columns in output
    +   */
    +  private void buildKeyList() {
    +    List<String> lastMapKeyList = null;
    +    for (VectorWrapper<?> vw : incoming) {
    +      if (vw.getField().getType().getMinorType() != TypeProtos.MinorType.MAP) {
    +        continue;
    +      }
    +
    +      keyList = Lists.newArrayList();
    +
    +      for (ValueVector vv : vw.getValueVector()) {
    +        keyList.add(vv.getField().getLastName());
    +      }
    +
    +      if (lastMapKeyList == null) {
    +        lastMapKeyList = keyList;
    +      } else {
    +        if (keyList.size() != lastMapKeyList.size() || !lastMapKeyList.containsAll(keyList)) {
    +          throw new UnsupportedOperationException("Maps have different fields");
    +        }
    +      }
    +    }
    +  }
    +
    +  private void buildOutputContainer() throws SchemaChangeException {
    +    dataSrcVecMap = Maps.newHashMap();
    +    copySrcVecMap = Maps.newHashMap();
    +    MajorType mt = null;
    +
    +    ErrorCollector collector = new ErrorCollectorImpl();
    +    GregorianCalendar calendar = new GregorianCalendar(TimeZone.getTimeZone("UTC"));
    +
    +    calendar.setTimeInMillis(System.currentTimeMillis());
    +    createKeyColumn("schema", ValueExpressions.getBigInt(schema++), copySrcVecMap);
    +    createKeyColumn("computed", ValueExpressions.getDate(calendar), copySrcVecMap);
    +
    +    for (VectorWrapper<?> vw : incoming) {
    +      addVectorToOutgoingContainer(vw.getField().getLastName(), vw, collector);
    +      /*MaterializedField ds = vw.getField();
    +      String field = vw.getField().getLastName();
    +      // Input map vector
    +      MapVector mapVector = (MapVector) vw.getValueVector();
    +      assert mapVector.getPrimitiveVectors().size() > 0;
    +      // Proceed to create output map vector with same name e.g. statcount etc.
    +      mt = mapVector.getField().getType();
    +      MaterializedField mf = MaterializedField.create(functions.get(field), mt);
    +      assert !dataSrcVecMap.containsKey(mf);
    +      ValueVector vector = TypeHelper.getNewVector(mf, oContext.getAllocator());
    +      container.add(vector);
    +      MapVector outputMapVector = (MapVector) vector;
    +
    +      for (ValueVector vv : mapVector) {
    +        String fieldName = vv.getField().getLastName();
    +        if (!keyList.contains(fieldName)) {
    +          throw new UnsupportedOperationException("Unpivot data vector " +
    +              ds + " contains key " + fieldName + " not contained in key source!");
    +        }
    +        if (vv.getField().getType().getMinorType() == TypeProtos.MinorType.MAP) {
    +          throw new UnsupportedOperationException("Unpivot of nested map is not supported!");
    +        }
    +        if (field.equals("column")) {
    +          outputMapVector.addOrGet(fieldName, vv.getField().getType(), vv.getClass());
    +        } else {
    +          List<LogicalExpression> args = Lists.newArrayList();
    +          //TODO: Something else to access value of col such as emp_id?
    +          args.add(SchemaPath.getSimplePath(vv.getField().getPath()));
    +          //TODO: Put in the mapVector
    +          LogicalExpression call = FunctionCallFactory.createExpression(functions.get(field), args);
    +          //TODO: Is this sufficient to add to new Map?
    +          ValueVector vector1 = addMapVector(fieldName, outputMapVector, call);
    +          if (collector.hasErrors()) {
    +            throw new SchemaChangeException("Failure while materializing expression. "
    +                + collector.toErrorString());
    +          }
    +        }
    +      }
    +      dataSrcVecMap.put(ds, outputMapVector);*/
    +    }
    +    //Now create NDV in the outgoing container which was not avaliable in the incoming
    +    for (VectorWrapper<?> vw : incoming) {
    +      if (vw.getField().getLastName().equalsIgnoreCase("sum_width")) {//NullableFloat8 type
    +        addVectorToOutgoingContainer("ndv", vw, collector);
    +        break;
    +      }
    +    }
    +    container.setRecordCount(0);
    +    recordCount = 0;
    +    container.buildSchema(incoming.getSchema().getSelectionVectorMode());
    +  }
    +
    +  private void addVectorToOutgoingContainer(String field, VectorWrapper vw, ErrorCollector collector)
    +     throws SchemaChangeException {
    +    // Input map vector
    +    MapVector mapVector = (MapVector) vw.getValueVector();
    +    MaterializedField mf;
    +    assert mapVector.getPrimitiveVectors().size() > 0;
    +    // Proceed to create output map vector with same name e.g. statcount etc.
    +    MajorType mt = mapVector.getField().getType();
    +    if (functions.get(field) != null) {
    +      mf = MaterializedField.create(functions.get(field), mt);
    +    } else {
    +      mf = MaterializedField.create(field, mt);
    +    }
    +    assert !dataSrcVecMap.containsKey(mf);
    +    ValueVector vector = TypeHelper.getNewVector(mf, oContext.getAllocator());
    +    container.add(vector);
    +    MapVector outputMapVector = (MapVector) vector;
    +
    +    for (ValueVector vv : mapVector) {
    +      String fieldName = vv.getField().getLastName();
    +      if (!keyList.contains(fieldName)) {
    +        throw new UnsupportedOperationException("Unpivot data vector " +
    +                field + " contains key " + fieldName + " not contained in key source!");
    +      }
    +      if (vv.getField().getType().getMinorType() == TypeProtos.MinorType.MAP) {
    +        throw new UnsupportedOperationException("Unpivot of nested map is not supported!");
    +      }
    +      if (field.equals("column")) {
    +        outputMapVector.addOrGet(fieldName, vv.getField().getType(), vv.getClass());
    +      } else {
    +        List<LogicalExpression> args = Lists.newArrayList();
    +        LogicalExpression call;
    +        //TODO: Something else to access value of col such as emp_id?
    +        args.add(SchemaPath.getSimplePath(vv.getField().getPath()));
    +        //TODO: Put in the mapVector
    +        if (functions.get(field) != null) {
    +          call = FunctionCallFactory.createExpression(functions.get(field), args);
    +        } else {
    +          call = FunctionCallFactory.createExpression(field, args);
    +        }
    +        //TODO: Is this sufficient to add to new Map?
    +        ValueVector vector1 = addMapVector(fieldName, outputMapVector, call);
    +        if (collector.hasErrors()) {
    +          throw new SchemaChangeException("Failure while materializing expression. "
    +                  + collector.toErrorString());
    +        }
    +      }
    +    }
    +  }
    +
    +  @Override
    +  protected boolean setupNewSchema() throws SchemaChangeException {
    +    container.clear();
    +    buildKeyList();
    +    buildOutputContainer();
    +    return true;
    +  }
    +
    +  @Override
    +  protected IterOutcome doWork() {
    +    int outRecordCount = incoming.getRecordCount();
    +    HashMap<String, Long> nonNullRows = new HashMap<>();
    +
    +    for (VectorWrapper<?> vw : incoming) {
    +      ValueVector vv = vw.getValueVector();
    +      if (vv.getField().getType().getMinorType() != TypeProtos.MinorType.MAP) {
    +        //We only expect Maps in the incoming. STOP, if this is not the case.
    +        return IterOutcome.STOP;
    +      }
    +      MapVector mapVec = (MapVector) vv;
    +      Map<String, ValueHolder> statMap = aggregationMap.get(vv.getField().getLastName());
    +
    +      for (ValueVector mapElt : mapVec) {
    +        if (vv.getField().getLastName().equalsIgnoreCase("column")) {
    +          BigIntHolder nameHolder;
    +          if (statMap.get(mapElt.getField().getLastName()) != null) {
    +            nameHolder = (BigIntHolder) statMap.get(mapElt.getField().getLastName());
    --- End diff --
    
    How is it that a "name" holder is an int? Is the holder holding some non-name attribute about the name?
    
    In comments earlier, it seemed that the keys were column names and the values were VarChars. But, here the values are BitInts...


---
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 #729: Drill 1328: Support table statistics for Parquet

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/729#discussion_r100680337
  
    --- Diff: exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/statistics/StatisticsAggBatch.java ---
    @@ -0,0 +1,256 @@
    +/**
    + * 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.impl.statistics;
    +
    +import com.google.common.collect.Lists;
    +import com.sun.codemodel.JExpr;
    +import org.apache.drill.common.expression.ErrorCollector;
    +import org.apache.drill.common.expression.ErrorCollectorImpl;
    +import org.apache.drill.common.expression.FunctionCallFactory;
    +import org.apache.drill.common.expression.LogicalExpression;
    +import org.apache.drill.common.expression.SchemaPath;
    +import org.apache.drill.common.expression.ValueExpressions;
    +import org.apache.drill.exec.exception.ClassTransformationException;
    +import org.apache.drill.exec.exception.OutOfMemoryException;
    +import org.apache.drill.exec.exception.SchemaChangeException;
    +import org.apache.drill.exec.expr.ClassGenerator;
    +import org.apache.drill.exec.expr.CodeGenerator;
    +import org.apache.drill.exec.expr.ExpressionTreeMaterializer;
    +import org.apache.drill.exec.expr.TypeHelper;
    +import org.apache.drill.exec.expr.ValueVectorWriteExpression;
    +import org.apache.drill.exec.ops.FragmentContext;
    +import org.apache.drill.exec.physical.config.StatisticsAggregate;
    +import org.apache.drill.exec.physical.impl.aggregate.StreamingAggBatch;
    +import org.apache.drill.exec.physical.impl.aggregate.StreamingAggTemplate;
    +import org.apache.drill.exec.physical.impl.aggregate.StreamingAggregator;
    +import org.apache.drill.exec.planner.physical.StatsAggPrel.OperatorPhase;
    +import org.apache.drill.exec.record.BatchSchema.SelectionVectorMode;
    +import org.apache.drill.exec.record.MaterializedField;
    +import org.apache.drill.exec.record.RecordBatch;
    +import org.apache.drill.exec.record.TypedFieldId;
    +import org.apache.drill.exec.store.ImplicitColumnExplorer;
    +import org.apache.drill.exec.vector.ValueVector;
    +import org.apache.drill.exec.vector.complex.FieldIdUtil;
    +import org.apache.drill.exec.vector.complex.MapVector;
    +
    +import java.io.IOException;
    +import java.util.GregorianCalendar;
    +import java.util.List;
    +import java.util.TimeZone;
    +
    +/**
    + * TODO: This needs cleanup. Currently the key values are constants and we compare the constants
    + * for every record. Seems unnecessary.
    + *
    + * Example input and output:
    + * Schema of incoming batch: region_id (VARCHAR), sales_city (VARCHAR), cnt (BIGINT)
    + * Schema of output:
    + *    "schema" : BIGINT - Schema number. For each schema change this number is incremented.
    + *    "computed" : BIGINT - What time is it computed?
    + *    "columns"       : MAP - Column names
    + *       "region_id"  : VARCHAR
    + *       "sales_city" : VARCHAR
    + *       "cnt"        : VARCHAR
    + *    "statscount" : MAP
    + *       "region_id"  : BIGINT - statscount(region_id) - aggregation over all values of region_id
    + *                      in incoming batch
    + *       "sales_city" : BIGINT - statscount(sales_city)
    + *       "cnt"        : BIGINT - statscount(cnt)
    + *    "nonnullstatcount" : MAP
    + *       "region_id"  : BIGINT - nonnullstatcount(region_id)
    + *       "sales_city" : BIGINT - nonnullstatcount(sales_city)
    + *       "cnt"        : BIGINT - nonnullstatcount(cnt)
    + *   .... another map for next stats function ....
    + */
    +public class StatisticsAggBatch extends StreamingAggBatch {
    +  private List<String> functions;
    +  private int schema = 0;
    +
    +  public StatisticsAggBatch(StatisticsAggregate popConfig, RecordBatch incoming,
    +      FragmentContext context) throws OutOfMemoryException {
    +    super(popConfig, incoming, context);
    +    this.functions = popConfig.getFunctions();
    +  }
    +
    +  private void createKeyColumn(String name, LogicalExpression expr, List<LogicalExpression> keyExprs,
    +      List<TypedFieldId> keyOutputIds) throws SchemaChangeException {
    +    ErrorCollector collector = new ErrorCollectorImpl();
    +
    +    LogicalExpression mle = ExpressionTreeMaterializer.materialize(expr, incoming, collector,
    +        context.getFunctionRegistry());
    +
    +    MaterializedField outputField = MaterializedField.create(name, mle.getMajorType());
    +    ValueVector vector = TypeHelper.getNewVector(outputField, oContext.getAllocator());
    +
    +    keyExprs.add(mle);
    +    keyOutputIds.add(container.add(vector));
    +
    +    if (collector.hasErrors()) {
    +      throw new SchemaChangeException("Failure while materializing expression. "
    +          + collector.toErrorString());
    +    }
    +  }
    +
    +  private void createNestedKeyColumn(MapVector parent, String name, LogicalExpression expr,
    +      List<LogicalExpression> keyExprs, List<TypedFieldId> keyOutputIds)
    +          throws SchemaChangeException {
    +    ErrorCollector collector = new ErrorCollectorImpl();
    +
    +    LogicalExpression mle = ExpressionTreeMaterializer.materialize(expr, incoming, collector,
    +        context.getFunctionRegistry());
    +
    +    Class<? extends ValueVector> vvc =
    +        TypeHelper.getValueVectorClass(mle.getMajorType().getMinorType(),
    +            mle.getMajorType().getMode());
    +
    +    ValueVector vv = parent.addOrGet(name, mle.getMajorType(), vvc);
    +
    +    TypedFieldId pfid = container.getValueVectorId(SchemaPath.getSimplePath(parent.getField().getPath()));
    +    assert pfid.getFieldIds().length == 1;
    +    TypedFieldId.Builder builder = TypedFieldId.newBuilder();
    +    builder.addId(pfid.getFieldIds()[0]);
    +    TypedFieldId id =
    +        FieldIdUtil.getFieldIdIfMatches(parent, builder, true,
    +            SchemaPath.getSimplePath(vv.getField().getPath()).getRootSegment());
    +
    +    keyExprs.add(mle);
    +    keyOutputIds.add(id);
    +
    +    if (collector.hasErrors()) {
    +      throw new SchemaChangeException("Failure while materializing expression. "
    +          + collector.toErrorString());
    +    }
    +  }
    +
    +  private void addMapVector(String name, MapVector parent, LogicalExpression expr,
    +      List<LogicalExpression> valueExprs) throws SchemaChangeException {
    +    ErrorCollector collector = new ErrorCollectorImpl();
    +
    +    LogicalExpression mle = ExpressionTreeMaterializer.materialize(expr, incoming, collector,
    +        context.getFunctionRegistry());
    +
    +    Class<? extends ValueVector> vvc =
    +        TypeHelper.getValueVectorClass(mle.getMajorType().getMinorType(), mle.getMajorType().getMode());
    +    ValueVector vv = parent.addOrGet(name, mle.getMajorType(), vvc);
    +
    +    TypedFieldId pfid = container.getValueVectorId(SchemaPath.getSimplePath(parent.getField().getPath()));
    +    assert pfid.getFieldIds().length == 1;
    +    TypedFieldId.Builder builder = TypedFieldId.newBuilder();
    +    builder.addId(pfid.getFieldIds()[0]);
    +    TypedFieldId id = FieldIdUtil.getFieldIdIfMatches(parent, builder, true,
    +        SchemaPath.getSimplePath(vv.getField().getPath()).getRootSegment());
    +
    +    valueExprs.add(new ValueVectorWriteExpression(id, mle, true));
    +
    +    if (collector.hasErrors()) {
    +      throw new SchemaChangeException("Failure while materializing expression. "
    +          + collector.toErrorString());
    +    }
    +  }
    +
    +  private StreamingAggregator codegenAggregator(List<LogicalExpression> keyExprs,
    +      List<LogicalExpression> valueExprs, List<TypedFieldId> keyOutputIds)
    +          throws SchemaChangeException, ClassTransformationException, IOException {
    +    ClassGenerator<StreamingAggregator> cg =
    +        CodeGenerator.getRoot(StreamingAggTemplate.TEMPLATE_DEFINITION, context.getFunctionRegistry(),
    +            context.getOptions());
    +
    +    LogicalExpression[] keyExprsArray = new LogicalExpression[keyExprs.size()];
    +    LogicalExpression[] valueExprsArray = new LogicalExpression[valueExprs.size()];
    +    TypedFieldId[] keyOutputIdsArray = new TypedFieldId[keyOutputIds.size()];
    +
    +    keyExprs.toArray(keyExprsArray);
    +    valueExprs.toArray(valueExprsArray);
    +    keyOutputIds.toArray(keyOutputIdsArray);
    +
    +    setupIsSame(cg, keyExprsArray);
    +    setupIsSameApart(cg, keyExprsArray);
    +    addRecordValues(cg, valueExprsArray);
    +    outputRecordKeys(cg, keyOutputIdsArray, keyExprsArray);
    +    outputRecordKeysPrev(cg, keyOutputIdsArray, keyExprsArray);
    +
    +    cg.getBlock("resetValues")._return(JExpr.TRUE);
    +    getIndex(cg);
    +
    +    container.buildSchema(SelectionVectorMode.NONE);
    +    StreamingAggregator agg = context.getImplementationClass(cg);
    +    agg.setup(oContext, incoming, this);
    +    return agg;
    +  }
    +
    +  private boolean isImplicitFileColumn(MaterializedField mf) {
    +    return ImplicitColumnExplorer.initImplicitFileColumns(context.getOptions()).get(mf.getName()) != null;
    +  }
    +
    +  protected StreamingAggregator createAggregatorInternal()
    +      throws SchemaChangeException, ClassTransformationException, IOException {
    +    container.clear();
    +
    +    List<LogicalExpression> keyExprs = Lists.newArrayList();
    +    List<LogicalExpression> valueExprs = Lists.newArrayList();
    +    List<TypedFieldId> keyOutputIds = Lists.newArrayList();
    +    GregorianCalendar calendar = new GregorianCalendar(TimeZone.getTimeZone("UTC"));
    +    calendar.setTimeInMillis(System.currentTimeMillis());
    +
    +    if (this.getPopConfig() instanceof StatisticsAggregate
    +        && (((StatisticsAggregate) this.getPopConfig()).getPhase() == OperatorPhase.PHASE_1of1
    +            || ((StatisticsAggregate) this.getPopConfig()).getPhase() == OperatorPhase.PHASE_2of2)) {
    --- End diff --
    
    Of course, the same can be done using the current enum: just add the suggested methods to the enum itself.


---
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 #729: Drill 1328: Support table statistics for Parquet

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

    https://github.com/apache/drill/pull/729#discussion_r102291655
  
    --- Diff: exec/java-exec/src/main/java/org/apache/drill/exec/planner/common/DrillFilterRelBase.java ---
    @@ -113,6 +114,6 @@ public double getRows() {
             selectivity = filterMaxSelectivityEstimateFactor;
           }
         }
    -    return selectivity*RelMetadataQuery.getRowCount(getInput());
    +    return NumberUtil.multiply(selectivity, RelMetadataQuery.getRowCount(getInput()));
    --- 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 #729: Drill 1328: Support table statistics for Parquet

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

    https://github.com/apache/drill/pull/729#discussion_r103368921
  
    --- Diff: exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/statistics/AvgWidthMergedStatistic.java ---
    @@ -0,0 +1,135 @@
    +/*
    + * 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.impl.statistics;
    +
    +import org.apache.drill.common.types.TypeProtos;
    +import org.apache.drill.exec.expr.holders.NullableFloat8Holder;
    +import org.apache.drill.exec.expr.holders.ValueHolder;
    +import org.apache.drill.exec.vector.NullableFloat8Vector;
    +import org.apache.drill.exec.vector.ValueVector;
    +import org.apache.drill.exec.vector.complex.MapVector;
    +
    +import java.util.HashMap;
    +import java.util.List;
    +import java.util.Map;
    +
    +public class AvgWidthMergedStatistic extends AbstractMergedStatistic {
    +
    +  private String name;
    +  private String inputName;
    +  private boolean configureComplete = false;
    +  private boolean mergeComplete = false;
    +  private Map<String, ValueHolder> sumHolder;
    +  MergedStatistic types, nonNullStatCounts, statCounts;
    +
    +  public AvgWidthMergedStatistic (String name, String inputName) {
    +    this.name = name;
    +    this.inputName = inputName;
    +    this.sumHolder = new HashMap<>();
    +    types = nonNullStatCounts = statCounts = null;
    +  }
    +
    +  @Override
    +  public String getName() {
    +    return name;
    +  }
    +
    +  @Override
    +  public String getInput() {
    +    return inputName;
    +  }
    +
    +  @Override
    +  public void merge(ValueVector input) {
    +    // Check the input is a Map Vector
    +    assert (input.getField().getType().getMinorType() == TypeProtos.MinorType.MAP);
    +    MapVector inputMap = (MapVector) input;
    +    for (ValueVector vv : inputMap) {
    +      String colName = vv.getField().getLastName();
    +      NullableFloat8Holder colSumHolder;
    +      if (sumHolder.get(colName) != null) {
    +        colSumHolder = (NullableFloat8Holder) sumHolder.get(colName);
    +      } else {
    +        colSumHolder = new NullableFloat8Holder();
    +        sumHolder.put(colName, colSumHolder);
    +      }
    +      Object val = vv.getAccessor().getObject(0);
    +      if (val != null) {
    +        colSumHolder.value += (double) val;
    +        colSumHolder.isSet = 1;
    +      }
    +    }
    +  }
    +
    +  @Override
    +  public Object getStat(String colName) {
    +      if (mergeComplete != true) {
    +        throw new IllegalStateException(
    +            String.format("Statistic `%s` has not completed merging statistics", name));
    +      }
    +      NullableFloat8Holder colSumHolder = (NullableFloat8Holder) sumHolder.get(colName);
    +      return (long) (colSumHolder.value/ getRowCount(colName));
    +    }
    +
    +  @Override
    +  public void setOutput(ValueVector output) {
    --- 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 #729: Drill 1328 r4

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/729#discussion_r99391067
  
    --- Diff: exec/java-exec/src/main/java/org/apache/drill/exec/ExecConstants.java ---
    @@ -390,4 +391,15 @@
     
       String DYNAMIC_UDF_SUPPORT_ENABLED = "exec.udf.enable_dynamic_support";
       BooleanValidator DYNAMIC_UDF_SUPPORT_ENABLED_VALIDATOR = new BooleanValidator(DYNAMIC_UDF_SUPPORT_ENABLED, true, true);
    +
    +  /**
    +   * Option whose value is a long value representing the number of bits required for computing ndv (using HLL)
    +   */
    +  LongValidator NDV_MEMORY_LIMIT = new PositiveLongValidator("exec.statistics.ndv_memory_limit", 30, 20);
    --- End diff --
    
    These validators are used for user-settable system or session options. The user can choose the NDV bit count? What happens if I do one run with one value, the next run with a different value? Do we handle a mix of lengths? If we don't, should this e a runtime option?


---
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 #729: Drill 1328 r4

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

    https://github.com/apache/drill/pull/729#discussion_r97905513
  
    --- Diff: exec/java-exec/src/main/java/org/apache/drill/exec/planner/cost/DrillRelMdRowCount.java ---
    @@ -17,32 +17,81 @@
      ******************************************************************************/
     package org.apache.drill.exec.planner.cost;
     
    +import java.io.IOException;
    +
    +import org.apache.calcite.rel.RelNode;
     import org.apache.calcite.rel.core.Aggregate;
    -import org.apache.calcite.rel.core.Filter;
    +import org.apache.calcite.rel.core.TableScan;
     import org.apache.calcite.rel.metadata.ReflectiveRelMetadataProvider;
     import org.apache.calcite.rel.metadata.RelMdRowCount;
     import org.apache.calcite.rel.metadata.RelMetadataProvider;
     import org.apache.calcite.util.BuiltInMethod;
     import org.apache.calcite.util.ImmutableBitSet;
    +import org.apache.drill.exec.planner.common.DrillFilterRelBase;
    +import org.apache.drill.exec.planner.common.DrillRelOptUtil;
    +import org.apache.drill.exec.planner.logical.DrillTable;
    +import org.apache.drill.exec.planner.logical.DrillTranslatableTable;
    +import org.apache.drill.exec.store.parquet.ParquetGroupScan;
     
     public class DrillRelMdRowCount extends RelMdRowCount{
       private static final DrillRelMdRowCount INSTANCE = new DrillRelMdRowCount();
     
       public static final RelMetadataProvider SOURCE = ReflectiveRelMetadataProvider.reflectiveSource(BuiltInMethod.ROW_COUNT.method, INSTANCE);
     
       @Override
    -  public Double getRowCount(Aggregate rel) {
    -    ImmutableBitSet groupKey = ImmutableBitSet.range(rel.getGroupCount());
    -
    -    if (groupKey.isEmpty()) {
    -      return 1.0;
    +  public Double getRowCount(RelNode rel) {
    +    if (rel instanceof TableScan) {
    +      return getRowCount((TableScan) rel);
    +    } else if (rel instanceof DrillFilterRelBase) {
    +      return getRowCount((DrillFilterRelBase) rel);
         } else {
           return super.getRowCount(rel);
         }
       }
     
    -  @Override
    -  public Double getRowCount(Filter rel) {
    +  private Double getRowCount(DrillFilterRelBase rel) {
    +    if (DrillRelOptUtil.guessRows(rel)) {
    +      return super.getRowCount(rel);
    +    }
    +    // Need capped selectivity estimates. See the Filter getRows() method
         return rel.getRows();
       }
    +
    +  private Double getRowCount(TableScan rel) {
    +    DrillTable table;
    +    if (DrillRelOptUtil.guessRows(rel)) {
    +      return super.getRowCount(rel);
    +    }
    +    table = rel.getTable().unwrap(DrillTable.class);
    +    if (table == null) {
    +      table = rel.getTable().unwrap(DrillTranslatableTable.class).getDrillTable();
    +    }
    +    // Return rowcount from statistics, if available. Otherwise, delegate to parent.
    +    try {
    +      if (table != null
    +          && table.getStatsTable() != null
    +          /* For ParquetGroupScan rely on accurate count from the scan instead of
    +           * statistics since partition pruning/filter pushdown might have occurred.
    +           * The other way would be to iterate over the rowgroups present in the
    +           * ParquetGroupScan to obtain the rowcount.
    +           */
    +          && !(table.getGroupScan() instanceof ParquetGroupScan)) {
    --- End diff --
    
    You could leverage GroupScanProperty.EXACT_ROW_COUNT/NON_EXACT_ROW_COUNT


---
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 #729: Drill 1328: Support table statistics for Parquet

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/729#discussion_r100681319
  
    --- Diff: exec/java-exec/src/main/java/org/apache/drill/exec/planner/cost/DrillRelMdSelectivity.java ---
    @@ -0,0 +1,219 @@
    +/*******************************************************************************
    + * 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.planner.cost;
    +
    +import org.apache.calcite.plan.RelOptUtil;
    +import org.apache.calcite.plan.volcano.RelSubset;
    +import org.apache.calcite.rel.RelNode;
    +import org.apache.calcite.rel.SingleRel;
    +import org.apache.calcite.rel.core.JoinRelType;
    +import org.apache.calcite.rel.core.TableScan;
    +import org.apache.calcite.rel.metadata.ReflectiveRelMetadataProvider;
    +import org.apache.calcite.rel.metadata.RelMdSelectivity;
    +import org.apache.calcite.rel.metadata.RelMdUtil;
    +import org.apache.calcite.rel.metadata.RelMetadataProvider;
    +import org.apache.calcite.rel.metadata.RelMetadataQuery;
    +import org.apache.calcite.rel.type.RelDataType;
    +import org.apache.calcite.rex.RexBuilder;
    +import org.apache.calcite.rex.RexCall;
    +import org.apache.calcite.rex.RexInputRef;
    +import org.apache.calcite.rex.RexNode;
    +import org.apache.calcite.rex.RexUtil;
    +import org.apache.calcite.rex.RexVisitor;
    +import org.apache.calcite.rex.RexVisitorImpl;
    +import org.apache.calcite.sql.SqlKind;
    +import org.apache.calcite.util.BuiltInMethod;
    +import org.apache.calcite.util.Util;
    +import org.apache.drill.exec.planner.common.DrillJoinRelBase;
    +import org.apache.drill.exec.planner.common.DrillRelOptUtil;
    +import org.apache.drill.exec.planner.logical.DrillTable;
    +import org.apache.drill.exec.planner.logical.DrillTranslatableTable;
    +
    +import java.util.ArrayList;
    +import java.util.List;
    +
    +public class DrillRelMdSelectivity extends RelMdSelectivity {
    +  private static final org.slf4j.Logger logger =
    +      org.slf4j.LoggerFactory.getLogger(RelMdSelectivity.class);
    +
    +  private static final DrillRelMdSelectivity INSTANCE =
    +      new DrillRelMdSelectivity();
    +
    +  public static final RelMetadataProvider SOURCE =
    +      ReflectiveRelMetadataProvider.reflectiveSource(
    +          BuiltInMethod.SELECTIVITY.method, INSTANCE);
    +
    +  @Override
    +  public Double getSelectivity(RelNode rel, RexNode predicate) {
    +    if (rel instanceof TableScan) {
    +      return getScanSelectivity((TableScan) rel, predicate);
    +    } else if (rel instanceof DrillJoinRelBase) {
    +      return getJoinSelectivity(((DrillJoinRelBase) rel), predicate);
    +    } else if (rel instanceof SingleRel && !DrillRelOptUtil.guessRows(rel)) {
    +        return RelMetadataQuery.getSelectivity(((SingleRel) rel).getInput(), predicate);
    +    } else if (rel instanceof RelSubset && !DrillRelOptUtil.guessRows(rel)) {
    +      if (((RelSubset) rel).getBest() != null) {
    +        return RelMetadataQuery.getSelectivity(((RelSubset)rel).getBest(), predicate);
    +      } else if (((RelSubset)rel).getOriginal() != null) {
    +        return RelMetadataQuery.getSelectivity(((RelSubset)rel).getOriginal(), predicate);
    +      } else {
    +        return super.getSelectivity(rel, predicate);
    +      }
    +    } else {
    +      return super.getSelectivity(rel, predicate);
    +    }
    +  }
    +
    +  private Double getJoinSelectivity(DrillJoinRelBase rel, RexNode predicate) {
    +    double sel = 1.0;
    +    // determine which filters apply to the left vs right
    +    RexNode leftPred = null;
    +    RexNode rightPred = null;
    +    JoinRelType joinType = rel.getJoinType();
    +    final RexBuilder rexBuilder = rel.getCluster().getRexBuilder();
    +    int[] adjustments = new int[rel.getRowType().getFieldCount()];
    +
    +    if (DrillRelOptUtil.guessRows(rel)) {
    +      return super.getSelectivity(rel, predicate);
    +    }
    +
    +    if (predicate != null) {
    +      RexNode pred;
    +      List<RexNode> leftFilters = new ArrayList<RexNode>();
    +      List<RexNode> rightFilters = new ArrayList<RexNode>();
    +      List<RexNode> joinFilters = new ArrayList<RexNode>();
    +      List<RexNode> predList = RelOptUtil.conjunctions(predicate);
    +
    +      RelOptUtil.classifyFilters(
    +          rel,
    +          predList,
    +          joinType,
    +          joinType == JoinRelType.INNER,
    +          !joinType.generatesNullsOnLeft(),
    +          !joinType.generatesNullsOnRight(),
    +          joinFilters,
    +          leftFilters,
    +          rightFilters);
    +      leftPred =
    +          RexUtil.composeConjunction(rexBuilder, leftFilters, true);
    +      rightPred =
    +          RexUtil.composeConjunction(rexBuilder, rightFilters, true);
    +      for (RelNode child : rel.getInputs()) {
    +        RexNode modifiedPred = null;
    +
    +        if (child == rel.getLeft()) {
    +          pred = leftPred;
    +        } else {
    +          pred = rightPred;
    +        }
    +        if (pred != null) {
    +          // convert the predicate to reference the types of the children
    +          modifiedPred =
    +              pred.accept(new RelOptUtil.RexInputConverter(
    +              rexBuilder,
    +              null,
    +              child.getRowType().getFieldList(),
    +              adjustments));
    +        }
    +        sel *= RelMetadataQuery.getSelectivity(child, modifiedPred);
    +      }
    +      sel *= RelMdUtil.guessSelectivity(
    +          RexUtil.composeConjunction(rexBuilder, joinFilters, true));
    +    }
    +    return sel;
    +  }
    +
    +  private Double getScanSelectivity(TableScan scan, RexNode predicate) {
    +    DrillTable table = scan.getTable().unwrap(DrillTable.class);
    +    if (table == null) {
    +      table = scan.getTable().unwrap(DrillTranslatableTable.class).getDrillTable();
    +    }
    +    if (table == null || table.getStatsTable() == null) {
    +      return super.getSelectivity(scan, predicate);
    +    } else {
    +      return getScanSelectivityInternal(table, predicate, scan.getRowType());
    +    }
    +  }
    +
    +  private Double getScanSelectivityInternal(DrillTable table, RexNode predicate,
    +      RelDataType type) {
    +    double sel = 1.0;
    +
    +    if ((predicate == null) || predicate.isAlwaysTrue()) {
    +      return sel;
    +    }
    +
    +    for (RexNode pred : RelOptUtil.conjunctions(predicate)) {
    +      double orSel = 0;
    +      for (RexNode orPred : RelOptUtil.disjunctions(pred)) {
    +        //CALCITE guess
    +        Double guess = RelMdUtil.guessSelectivity(pred);
    +        if (orPred.isA(SqlKind.EQUALS)) {
    --- End diff --
    
    This does not seem to use the new information for expressions other than =. Please see DRILL-5254. Consider feeding the base p(a = value) = 1/NDV into the calculations described there.


---
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 #729: Drill 1328: Support table statistics for Parquet

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

    https://github.com/apache/drill/pull/729#discussion_r100869218
  
    --- Diff: exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/statistics/StatisticsMergeBatch.java ---
    @@ -0,0 +1,532 @@
    +/**
    + * 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.impl.statistics;
    +
    +import com.google.common.collect.Lists;
    +import com.google.common.collect.Maps;
    +import org.apache.drill.common.expression.LogicalExpression;
    +import org.apache.drill.common.expression.ErrorCollector;
    +import org.apache.drill.common.expression.ErrorCollectorImpl;
    +import org.apache.drill.common.expression.FunctionCallFactory;
    +import org.apache.drill.common.expression.SchemaPath;
    +import org.apache.drill.common.expression.ValueExpressions;
    +import org.apache.drill.common.types.TypeProtos;
    +import org.apache.drill.common.types.TypeProtos.MajorType;
    +import org.apache.drill.exec.exception.OutOfMemoryException;
    +import org.apache.drill.exec.exception.SchemaChangeException;
    +import org.apache.drill.exec.expr.ExpressionTreeMaterializer;
    +import org.apache.drill.exec.expr.TypeHelper;
    +import org.apache.drill.exec.expr.holders.BigIntHolder;
    +import org.apache.drill.exec.expr.holders.NullableFloat8Holder;
    +import org.apache.drill.exec.expr.holders.ObjectHolder;
    +import org.apache.drill.exec.expr.holders.ValueHolder;
    +import org.apache.drill.exec.ops.FragmentContext;
    +import org.apache.drill.exec.physical.config.StatisticsMerge;
    +import org.apache.drill.exec.record.AbstractSingleRecordBatch;
    +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.VectorContainer;
    +import org.apache.drill.exec.record.VectorWrapper;
    +import org.apache.drill.exec.vector.ValueVector;
    +import org.apache.drill.exec.vector.DateVector;
    +import org.apache.drill.exec.vector.BigIntVector;
    +import org.apache.drill.exec.vector.NullableBigIntVector;
    +import org.apache.drill.exec.vector.NullableFloat8Vector;
    +import org.apache.drill.exec.vector.NullableVarBinaryVector;
    +import org.apache.drill.exec.vector.VarCharVector;
    +import org.apache.drill.exec.vector.complex.MapVector;
    +
    +import java.io.ByteArrayInputStream;
    +import java.io.DataInputStream;
    +import java.io.IOException;
    +import java.util.List;
    +import java.util.GregorianCalendar;
    +import java.util.HashMap;
    +import java.util.Map;
    +import java.util.TimeZone;
    +
    +import com.clearspring.analytics.stream.cardinality.HyperLogLog;
    +
    +public class StatisticsMergeBatch extends AbstractSingleRecordBatch<StatisticsMerge> {
    +  private static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(StatisticsMergeBatch.class);
    +  private Map<String, String> functions;
    +  private boolean first = true;
    +  private boolean finished = false;
    +  private int schema = 0;
    +  private int recordCount = 0;
    +  private List<String> keyList = null;
    +  private Map<MaterializedField, ValueVector> dataSrcVecMap = null;
    +  // Map of non-map fields to VV in the incoming schema
    +  private Map<MaterializedField, ValueVector> copySrcVecMap = null;
    +  private Map<String, Map<String, ValueHolder>> aggregationMap = null;
    +  public StatisticsMergeBatch(StatisticsMerge popConfig, RecordBatch incoming,
    +                              FragmentContext context) throws OutOfMemoryException {
    +    super(popConfig, context, incoming);
    +    this.functions = new HashMap<>();
    +    this.aggregationMap = new HashMap<>();
    +
    +    /*for (String key : popConfig.getFunctions()) {
    +      aggregationMap.put(key, new HashMap<String, ValueHolder>());
    +      if (key.equalsIgnoreCase("statcount") || key.equalsIgnoreCase("nonnullstatcount")) {
    +        functions.put(key, "sum");
    +      } else if (key.equalsIgnoreCase("hll")) {
    +        functions.put(key, "hll_merge");
    +      } else if (key.equalsIgnoreCase("sum_width")) {
    +        functions.put(key, "avg_width");
    +      }
    +    }*/
    +    for (String key : popConfig.getFunctions()) {
    +      if (key.equalsIgnoreCase("sum_width")) {
    +        functions.put(key, "avg_width");
    +      } else if (key.equalsIgnoreCase("hll")) {
    +        functions.put(key, "hll_merge");
    +      } else {
    +        functions.put(key, key);
    +      }
    +      aggregationMap.put(functions.get(key), new HashMap<String, ValueHolder>());
    +    }
    +  }
    +
    +  private void createKeyColumn(String name, LogicalExpression expr, Map<MaterializedField, ValueVector> parentMap)
    +      throws SchemaChangeException {
    +    ErrorCollector collector = new ErrorCollectorImpl();
    +
    +    LogicalExpression mle = ExpressionTreeMaterializer.materialize(expr, incoming, collector,
    +        context.getFunctionRegistry());
    +
    +    MaterializedField outputField = MaterializedField.create(name, mle.getMajorType());
    +    ValueVector vector = TypeHelper.getNewVector(outputField, oContext.getAllocator());
    +    container.add(vector);
    +
    +    if (collector.hasErrors()) {
    +      throw new SchemaChangeException("Failure while materializing expression. "
    +          + collector.toErrorString());
    +    }
    +    parentMap.put(outputField, vector);
    +  }
    +
    +  private ValueVector addMapVector(String name, MapVector parent, LogicalExpression expr) throws SchemaChangeException {
    +    ErrorCollector collector = new ErrorCollectorImpl();
    +
    +    LogicalExpression mle = ExpressionTreeMaterializer.materialize(expr, incoming, collector,
    +        context.getFunctionRegistry());
    +
    +    Class<? extends ValueVector> vvc =
    +        TypeHelper.getValueVectorClass(mle.getMajorType().getMinorType(), mle.getMajorType().getMode());
    +    ValueVector vector = parent.addOrGet(name, mle.getMajorType(), vvc);
    +
    +    if (collector.hasErrors()) {
    +      throw new SchemaChangeException("Failure while materializing expression. "
    +          + collector.toErrorString());
    +    }
    +    return vector;
    +  }
    +
    +  /**
    +   * Identify the list of fields within a map which are unpivoted as columns in output
    +   */
    +  private void buildKeyList() {
    +    List<String> lastMapKeyList = null;
    +    for (VectorWrapper<?> vw : incoming) {
    +      if (vw.getField().getType().getMinorType() != TypeProtos.MinorType.MAP) {
    +        continue;
    +      }
    +
    +      keyList = Lists.newArrayList();
    +
    +      for (ValueVector vv : vw.getValueVector()) {
    +        keyList.add(vv.getField().getLastName());
    +      }
    +
    +      if (lastMapKeyList == null) {
    +        lastMapKeyList = keyList;
    +      } else {
    +        if (keyList.size() != lastMapKeyList.size() || !lastMapKeyList.containsAll(keyList)) {
    +          throw new UnsupportedOperationException("Maps have different fields");
    +        }
    +      }
    +    }
    +  }
    +
    +  private void buildOutputContainer() throws SchemaChangeException {
    +    dataSrcVecMap = Maps.newHashMap();
    +    copySrcVecMap = Maps.newHashMap();
    +    MajorType mt = null;
    +
    +    ErrorCollector collector = new ErrorCollectorImpl();
    +    GregorianCalendar calendar = new GregorianCalendar(TimeZone.getTimeZone("UTC"));
    +
    +    calendar.setTimeInMillis(System.currentTimeMillis());
    +    createKeyColumn("schema", ValueExpressions.getBigInt(schema++), copySrcVecMap);
    +    createKeyColumn("computed", ValueExpressions.getDate(calendar), copySrcVecMap);
    +
    +    for (VectorWrapper<?> vw : incoming) {
    +      addVectorToOutgoingContainer(vw.getField().getLastName(), vw, collector);
    +      /*MaterializedField ds = vw.getField();
    --- 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 #729: Drill 1328: Support table statistics for Parquet

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/729#discussion_r100681119
  
    --- Diff: exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/unpivot/UnpivotMapsRecordBatch.java ---
    @@ -0,0 +1,276 @@
    +/**
    + * 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.impl.unpivot;
    +
    +import java.util.List;
    +import java.util.Map;
    +
    +import org.apache.drill.common.types.TypeProtos.MajorType;
    +import org.apache.drill.common.types.TypeProtos.MinorType;
    +import org.apache.drill.exec.exception.OutOfMemoryException;
    +import org.apache.drill.exec.exception.SchemaChangeException;
    +import org.apache.drill.exec.expr.TypeHelper;
    +import org.apache.drill.exec.ops.FragmentContext;
    +import org.apache.drill.exec.physical.config.UnpivotMaps;
    +import org.apache.drill.exec.record.AbstractSingleRecordBatch;
    +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.VectorContainer;
    +import org.apache.drill.exec.record.VectorWrapper;
    +import org.apache.drill.exec.vector.ValueVector;
    +import org.apache.drill.exec.vector.complex.MapVector;
    +
    +import com.google.common.collect.Lists;
    +import com.google.common.collect.Maps;
    +
    +/**
    + * Unpivot maps. Assumptions are:
    + *  1) all child vectors in a map are of same type.
    + *  2) Each map contains the same number of fields and field names are also same (types could be different).
    + *
    + * Example input and output:
    + * Schema of input:
    + *    "schema"        : BIGINT - Schema number. For each schema change this number is incremented.
    + *    "computed"      : BIGINT - What time is it computed?
    + *    "columns" : MAP - Column names
    + *       "region_id"  : VARCHAR
    + *       "sales_city" : VARCHAR
    + *       "cnt"        : VARCHAR
    + *    "statscount" : MAP
    + *       "region_id"  : BIGINT - statscount(region_id) - aggregation over all values of region_id
    + *                      in incoming batch
    + *       "sales_city" : BIGINT - statscount(sales_city)
    + *       "cnt"        : BIGINT - statscount(cnt)
    + *    "nonnullstatcount" : MAP
    + *       "region_id"  : BIGINT - nonnullstatcount(region_id)
    + *       "sales_city" : BIGINT - nonnullstatcount(sales_city)
    + *       "cnt"        : BIGINT - nonnullstatcount(cnt)
    + *   .... another map for next stats function ....
    + *
    + * Schema of output:
    --- End diff --
    
    Actually, why can't the entire implementation use the "unpivoted" approach? Would make the code and batches MUCH simpler. We already have the schema id to indicate when we receive the first record of the next schema.


---
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 #729: Drill 1328: Support table statistics for Parquet

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/729#discussion_r100677562
  
    --- Diff: exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/statistics/StatisticsAggBatch.java ---
    @@ -0,0 +1,256 @@
    +/**
    + * 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.impl.statistics;
    +
    +import com.google.common.collect.Lists;
    +import com.sun.codemodel.JExpr;
    +import org.apache.drill.common.expression.ErrorCollector;
    +import org.apache.drill.common.expression.ErrorCollectorImpl;
    +import org.apache.drill.common.expression.FunctionCallFactory;
    +import org.apache.drill.common.expression.LogicalExpression;
    +import org.apache.drill.common.expression.SchemaPath;
    +import org.apache.drill.common.expression.ValueExpressions;
    +import org.apache.drill.exec.exception.ClassTransformationException;
    +import org.apache.drill.exec.exception.OutOfMemoryException;
    +import org.apache.drill.exec.exception.SchemaChangeException;
    +import org.apache.drill.exec.expr.ClassGenerator;
    +import org.apache.drill.exec.expr.CodeGenerator;
    +import org.apache.drill.exec.expr.ExpressionTreeMaterializer;
    +import org.apache.drill.exec.expr.TypeHelper;
    +import org.apache.drill.exec.expr.ValueVectorWriteExpression;
    +import org.apache.drill.exec.ops.FragmentContext;
    +import org.apache.drill.exec.physical.config.StatisticsAggregate;
    +import org.apache.drill.exec.physical.impl.aggregate.StreamingAggBatch;
    +import org.apache.drill.exec.physical.impl.aggregate.StreamingAggTemplate;
    +import org.apache.drill.exec.physical.impl.aggregate.StreamingAggregator;
    +import org.apache.drill.exec.planner.physical.StatsAggPrel.OperatorPhase;
    +import org.apache.drill.exec.record.BatchSchema.SelectionVectorMode;
    +import org.apache.drill.exec.record.MaterializedField;
    +import org.apache.drill.exec.record.RecordBatch;
    +import org.apache.drill.exec.record.TypedFieldId;
    +import org.apache.drill.exec.store.ImplicitColumnExplorer;
    +import org.apache.drill.exec.vector.ValueVector;
    +import org.apache.drill.exec.vector.complex.FieldIdUtil;
    +import org.apache.drill.exec.vector.complex.MapVector;
    +
    +import java.io.IOException;
    +import java.util.GregorianCalendar;
    +import java.util.List;
    +import java.util.TimeZone;
    +
    +/**
    + * TODO: This needs cleanup. Currently the key values are constants and we compare the constants
    + * for every record. Seems unnecessary.
    + *
    + * Example input and output:
    + * Schema of incoming batch: region_id (VARCHAR), sales_city (VARCHAR), cnt (BIGINT)
    + * Schema of output:
    + *    "schema" : BIGINT - Schema number. For each schema change this number is incremented.
    + *    "computed" : BIGINT - What time is it computed?
    + *    "columns"       : MAP - Column names
    + *       "region_id"  : VARCHAR
    + *       "sales_city" : VARCHAR
    + *       "cnt"        : VARCHAR
    + *    "statscount" : MAP
    + *       "region_id"  : BIGINT - statscount(region_id) - aggregation over all values of region_id
    + *                      in incoming batch
    + *       "sales_city" : BIGINT - statscount(sales_city)
    + *       "cnt"        : BIGINT - statscount(cnt)
    + *    "nonnullstatcount" : MAP
    + *       "region_id"  : BIGINT - nonnullstatcount(region_id)
    + *       "sales_city" : BIGINT - nonnullstatcount(sales_city)
    + *       "cnt"        : BIGINT - nonnullstatcount(cnt)
    + *   .... another map for next stats function ....
    + */
    +public class StatisticsAggBatch extends StreamingAggBatch {
    +  private List<String> functions;
    +  private int schema = 0;
    +
    +  public StatisticsAggBatch(StatisticsAggregate popConfig, RecordBatch incoming,
    +      FragmentContext context) throws OutOfMemoryException {
    +    super(popConfig, incoming, context);
    +    this.functions = popConfig.getFunctions();
    +  }
    +
    +  private void createKeyColumn(String name, LogicalExpression expr, List<LogicalExpression> keyExprs,
    +      List<TypedFieldId> keyOutputIds) throws SchemaChangeException {
    +    ErrorCollector collector = new ErrorCollectorImpl();
    +
    +    LogicalExpression mle = ExpressionTreeMaterializer.materialize(expr, incoming, collector,
    +        context.getFunctionRegistry());
    +
    +    MaterializedField outputField = MaterializedField.create(name, mle.getMajorType());
    +    ValueVector vector = TypeHelper.getNewVector(outputField, oContext.getAllocator());
    +
    +    keyExprs.add(mle);
    +    keyOutputIds.add(container.add(vector));
    +
    +    if (collector.hasErrors()) {
    +      throw new SchemaChangeException("Failure while materializing expression. "
    +          + collector.toErrorString());
    +    }
    +  }
    +
    +  private void createNestedKeyColumn(MapVector parent, String name, LogicalExpression expr,
    +      List<LogicalExpression> keyExprs, List<TypedFieldId> keyOutputIds)
    +          throws SchemaChangeException {
    +    ErrorCollector collector = new ErrorCollectorImpl();
    +
    +    LogicalExpression mle = ExpressionTreeMaterializer.materialize(expr, incoming, collector,
    +        context.getFunctionRegistry());
    +
    +    Class<? extends ValueVector> vvc =
    +        TypeHelper.getValueVectorClass(mle.getMajorType().getMinorType(),
    +            mle.getMajorType().getMode());
    +
    +    ValueVector vv = parent.addOrGet(name, mle.getMajorType(), vvc);
    +
    +    TypedFieldId pfid = container.getValueVectorId(SchemaPath.getSimplePath(parent.getField().getPath()));
    +    assert pfid.getFieldIds().length == 1;
    +    TypedFieldId.Builder builder = TypedFieldId.newBuilder();
    +    builder.addId(pfid.getFieldIds()[0]);
    +    TypedFieldId id =
    +        FieldIdUtil.getFieldIdIfMatches(parent, builder, true,
    +            SchemaPath.getSimplePath(vv.getField().getPath()).getRootSegment());
    +
    +    keyExprs.add(mle);
    +    keyOutputIds.add(id);
    +
    +    if (collector.hasErrors()) {
    +      throw new SchemaChangeException("Failure while materializing expression. "
    +          + collector.toErrorString());
    +    }
    +  }
    +
    +  private void addMapVector(String name, MapVector parent, LogicalExpression expr,
    +      List<LogicalExpression> valueExprs) throws SchemaChangeException {
    +    ErrorCollector collector = new ErrorCollectorImpl();
    +
    +    LogicalExpression mle = ExpressionTreeMaterializer.materialize(expr, incoming, collector,
    +        context.getFunctionRegistry());
    +
    +    Class<? extends ValueVector> vvc =
    +        TypeHelper.getValueVectorClass(mle.getMajorType().getMinorType(), mle.getMajorType().getMode());
    +    ValueVector vv = parent.addOrGet(name, mle.getMajorType(), vvc);
    +
    +    TypedFieldId pfid = container.getValueVectorId(SchemaPath.getSimplePath(parent.getField().getPath()));
    +    assert pfid.getFieldIds().length == 1;
    +    TypedFieldId.Builder builder = TypedFieldId.newBuilder();
    +    builder.addId(pfid.getFieldIds()[0]);
    +    TypedFieldId id = FieldIdUtil.getFieldIdIfMatches(parent, builder, true,
    +        SchemaPath.getSimplePath(vv.getField().getPath()).getRootSegment());
    +
    +    valueExprs.add(new ValueVectorWriteExpression(id, mle, true));
    +
    +    if (collector.hasErrors()) {
    +      throw new SchemaChangeException("Failure while materializing expression. "
    +          + collector.toErrorString());
    +    }
    +  }
    +
    +  private StreamingAggregator codegenAggregator(List<LogicalExpression> keyExprs,
    +      List<LogicalExpression> valueExprs, List<TypedFieldId> keyOutputIds)
    +          throws SchemaChangeException, ClassTransformationException, IOException {
    +    ClassGenerator<StreamingAggregator> cg =
    +        CodeGenerator.getRoot(StreamingAggTemplate.TEMPLATE_DEFINITION, context.getFunctionRegistry(),
    +            context.getOptions());
    +
    +    LogicalExpression[] keyExprsArray = new LogicalExpression[keyExprs.size()];
    +    LogicalExpression[] valueExprsArray = new LogicalExpression[valueExprs.size()];
    +    TypedFieldId[] keyOutputIdsArray = new TypedFieldId[keyOutputIds.size()];
    +
    +    keyExprs.toArray(keyExprsArray);
    +    valueExprs.toArray(valueExprsArray);
    +    keyOutputIds.toArray(keyOutputIdsArray);
    +
    +    setupIsSame(cg, keyExprsArray);
    +    setupIsSameApart(cg, keyExprsArray);
    +    addRecordValues(cg, valueExprsArray);
    +    outputRecordKeys(cg, keyOutputIdsArray, keyExprsArray);
    +    outputRecordKeysPrev(cg, keyOutputIdsArray, keyExprsArray);
    +
    +    cg.getBlock("resetValues")._return(JExpr.TRUE);
    +    getIndex(cg);
    +
    +    container.buildSchema(SelectionVectorMode.NONE);
    +    StreamingAggregator agg = context.getImplementationClass(cg);
    +    agg.setup(oContext, incoming, this);
    +    return agg;
    +  }
    +
    +  private boolean isImplicitFileColumn(MaterializedField mf) {
    +    return ImplicitColumnExplorer.initImplicitFileColumns(context.getOptions()).get(mf.getName()) != null;
    +  }
    +
    +  protected StreamingAggregator createAggregatorInternal()
    +      throws SchemaChangeException, ClassTransformationException, IOException {
    +    container.clear();
    +
    +    List<LogicalExpression> keyExprs = Lists.newArrayList();
    +    List<LogicalExpression> valueExprs = Lists.newArrayList();
    +    List<TypedFieldId> keyOutputIds = Lists.newArrayList();
    +    GregorianCalendar calendar = new GregorianCalendar(TimeZone.getTimeZone("UTC"));
    +    calendar.setTimeInMillis(System.currentTimeMillis());
    +
    +    if (this.getPopConfig() instanceof StatisticsAggregate
    +        && (((StatisticsAggregate) this.getPopConfig()).getPhase() == OperatorPhase.PHASE_1of1
    +            || ((StatisticsAggregate) this.getPopConfig()).getPhase() == OperatorPhase.PHASE_2of2)) {
    --- End diff --
    
    So, if we do the code below for Phase 1 of 1 and Phase 2 of 2, which phase are we excluding? Phase 1 of 2?
    
    This gets back to the name. Maybe we need to be clearer. In the Pop:
    
    ```
      int phaseCount;
      int phase;
    
      boolean isSinglePhase() { return phaseCount == 1; }
      boolean isFirstPhase() { return phase == 1; }
      boolean isFinalPhase() { return phase == 2 || phaseCount == 1; }
    ...
    
     if (statsConfig.isFinalPhase() {
       // The setup code 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 #729: Drill 1328: Support table statistics for Parquet

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/729#discussion_r102874862
  
    --- Diff: exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/statistics/MergedStatisticFactory.java ---
    @@ -0,0 +1,46 @@
    +/*
    + * 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.impl.statistics;
    +
    +public class MergedStatisticFactory {
    +  /*
    +   * Creates the appropriate statistics object given the name of the statistics and the input statistic
    +   */
    +  public static MergedStatistic getMergedStatistic(String outputStatName, String inputStatName) {
    +    if (outputStatName == null || inputStatName == null) {
    +      return null;
    +    } else if (outputStatName.equals(Statistic.COLNAME)) {
    +      return new ColumnMergedStatistic(outputStatName, inputStatName);
    +    } else if (outputStatName.equals(Statistic.COLTYPE)) {
    +      return new ColTypeMergedStatistic(outputStatName, inputStatName);
    +    } else if (outputStatName.equals(Statistic.STATCOUNT)) {
    +      return new StatCountMergedStatistic(outputStatName, inputStatName);
    +    } else if (outputStatName.equals(Statistic.NNSTATCOUNT)) {
    +      return new NNStatCountMergedStatistic(outputStatName, inputStatName);
    +    } else if (outputStatName.equals(Statistic.AVG_WIDTH)) {
    +      return new AvgWidthMergedStatistic(outputStatName, inputStatName);
    +    } else if (outputStatName.equals(Statistic.HLL_MERGE)) {
    +      return new HLLMergedStatistic(outputStatName, inputStatName);
    +    } else if (outputStatName.equals(Statistic.NDV)) {
    +      return new NDVMergedStatistic(outputStatName, inputStatName);
    --- End diff --
    
    Consider:
    
    ```
    if (outputStatName.equals(Statistic.NDV)) {
      stat = new NDVMergedStatistic();
    }
    ...
    stat.init(outputStatName, inputStatName);
    ```
    How often will this factory be used? If frequently, then consider:
    ```
    class Factory() {
       private Factory instance = new Factory();
       private HashMap<String,Class<? extends MergedStatistic>> statsClasses = new HashMap<>( );
    
       private Factory() {
         statsClasses.put(Statistic.NDV, NDVMergedStatistic.class);
         ...
      }
       public MergedStatistic getMergedStatistic(String outputStatName, String inputStatName) {
           return instance.newMergedStatistic(outputStatName, inputStatName);
      }
      private MergedStatistic instance.newMergedStatistic(String outputStatName, String inputStatName) {
        MergedStatistic stat = statsClasses.get(outputStatName).newInstance();
        stat.init(outputStatName, inputStatName);
        return stat;
      }
    ```
    Lots of ways to do the above; this is just an example.
     


---
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 #729: Drill 1328: Support table statistics for Parquet

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

    https://github.com/apache/drill/pull/729#discussion_r100709555
  
    --- Diff: exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/statistics/StatisticsAggBatch.java ---
    @@ -0,0 +1,256 @@
    +/**
    + * 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.impl.statistics;
    +
    +import com.google.common.collect.Lists;
    +import com.sun.codemodel.JExpr;
    +import org.apache.drill.common.expression.ErrorCollector;
    +import org.apache.drill.common.expression.ErrorCollectorImpl;
    +import org.apache.drill.common.expression.FunctionCallFactory;
    +import org.apache.drill.common.expression.LogicalExpression;
    +import org.apache.drill.common.expression.SchemaPath;
    +import org.apache.drill.common.expression.ValueExpressions;
    +import org.apache.drill.exec.exception.ClassTransformationException;
    +import org.apache.drill.exec.exception.OutOfMemoryException;
    +import org.apache.drill.exec.exception.SchemaChangeException;
    +import org.apache.drill.exec.expr.ClassGenerator;
    +import org.apache.drill.exec.expr.CodeGenerator;
    +import org.apache.drill.exec.expr.ExpressionTreeMaterializer;
    +import org.apache.drill.exec.expr.TypeHelper;
    +import org.apache.drill.exec.expr.ValueVectorWriteExpression;
    +import org.apache.drill.exec.ops.FragmentContext;
    +import org.apache.drill.exec.physical.config.StatisticsAggregate;
    +import org.apache.drill.exec.physical.impl.aggregate.StreamingAggBatch;
    +import org.apache.drill.exec.physical.impl.aggregate.StreamingAggTemplate;
    +import org.apache.drill.exec.physical.impl.aggregate.StreamingAggregator;
    +import org.apache.drill.exec.planner.physical.StatsAggPrel.OperatorPhase;
    +import org.apache.drill.exec.record.BatchSchema.SelectionVectorMode;
    +import org.apache.drill.exec.record.MaterializedField;
    +import org.apache.drill.exec.record.RecordBatch;
    +import org.apache.drill.exec.record.TypedFieldId;
    +import org.apache.drill.exec.store.ImplicitColumnExplorer;
    +import org.apache.drill.exec.vector.ValueVector;
    +import org.apache.drill.exec.vector.complex.FieldIdUtil;
    +import org.apache.drill.exec.vector.complex.MapVector;
    +
    +import java.io.IOException;
    +import java.util.GregorianCalendar;
    +import java.util.List;
    +import java.util.TimeZone;
    +
    +/**
    + * TODO: This needs cleanup. Currently the key values are constants and we compare the constants
    + * for every record. Seems unnecessary.
    + *
    + * Example input and output:
    + * Schema of incoming batch: region_id (VARCHAR), sales_city (VARCHAR), cnt (BIGINT)
    + * Schema of output:
    + *    "schema" : BIGINT - Schema number. For each schema change this number is incremented.
    + *    "computed" : BIGINT - What time is it computed?
    + *    "columns"       : MAP - Column names
    + *       "region_id"  : VARCHAR
    + *       "sales_city" : VARCHAR
    + *       "cnt"        : VARCHAR
    + *    "statscount" : MAP
    + *       "region_id"  : BIGINT - statscount(region_id) - aggregation over all values of region_id
    + *                      in incoming batch
    + *       "sales_city" : BIGINT - statscount(sales_city)
    + *       "cnt"        : BIGINT - statscount(cnt)
    + *    "nonnullstatcount" : MAP
    + *       "region_id"  : BIGINT - nonnullstatcount(region_id)
    + *       "sales_city" : BIGINT - nonnullstatcount(sales_city)
    + *       "cnt"        : BIGINT - nonnullstatcount(cnt)
    + *   .... another map for next stats function ....
    + */
    +public class StatisticsAggBatch extends StreamingAggBatch {
    +  private List<String> functions;
    +  private int schema = 0;
    +
    +  public StatisticsAggBatch(StatisticsAggregate popConfig, RecordBatch incoming,
    +      FragmentContext context) throws OutOfMemoryException {
    +    super(popConfig, incoming, context);
    +    this.functions = popConfig.getFunctions();
    +  }
    +
    +  private void createKeyColumn(String name, LogicalExpression expr, List<LogicalExpression> keyExprs,
    +      List<TypedFieldId> keyOutputIds) throws SchemaChangeException {
    +    ErrorCollector collector = new ErrorCollectorImpl();
    +
    +    LogicalExpression mle = ExpressionTreeMaterializer.materialize(expr, incoming, collector,
    +        context.getFunctionRegistry());
    +
    +    MaterializedField outputField = MaterializedField.create(name, mle.getMajorType());
    +    ValueVector vector = TypeHelper.getNewVector(outputField, oContext.getAllocator());
    +
    +    keyExprs.add(mle);
    +    keyOutputIds.add(container.add(vector));
    +
    +    if (collector.hasErrors()) {
    +      throw new SchemaChangeException("Failure while materializing expression. "
    +          + collector.toErrorString());
    +    }
    +  }
    +
    +  private void createNestedKeyColumn(MapVector parent, String name, LogicalExpression expr,
    +      List<LogicalExpression> keyExprs, List<TypedFieldId> keyOutputIds)
    +          throws SchemaChangeException {
    +    ErrorCollector collector = new ErrorCollectorImpl();
    +
    +    LogicalExpression mle = ExpressionTreeMaterializer.materialize(expr, incoming, collector,
    +        context.getFunctionRegistry());
    +
    +    Class<? extends ValueVector> vvc =
    +        TypeHelper.getValueVectorClass(mle.getMajorType().getMinorType(),
    +            mle.getMajorType().getMode());
    +
    +    ValueVector vv = parent.addOrGet(name, mle.getMajorType(), vvc);
    +
    +    TypedFieldId pfid = container.getValueVectorId(SchemaPath.getSimplePath(parent.getField().getPath()));
    +    assert pfid.getFieldIds().length == 1;
    +    TypedFieldId.Builder builder = TypedFieldId.newBuilder();
    +    builder.addId(pfid.getFieldIds()[0]);
    +    TypedFieldId id =
    +        FieldIdUtil.getFieldIdIfMatches(parent, builder, true,
    +            SchemaPath.getSimplePath(vv.getField().getPath()).getRootSegment());
    +
    +    keyExprs.add(mle);
    +    keyOutputIds.add(id);
    +
    +    if (collector.hasErrors()) {
    +      throw new SchemaChangeException("Failure while materializing expression. "
    +          + collector.toErrorString());
    +    }
    +  }
    +
    +  private void addMapVector(String name, MapVector parent, LogicalExpression expr,
    +      List<LogicalExpression> valueExprs) throws SchemaChangeException {
    +    ErrorCollector collector = new ErrorCollectorImpl();
    +
    +    LogicalExpression mle = ExpressionTreeMaterializer.materialize(expr, incoming, collector,
    +        context.getFunctionRegistry());
    +
    +    Class<? extends ValueVector> vvc =
    +        TypeHelper.getValueVectorClass(mle.getMajorType().getMinorType(), mle.getMajorType().getMode());
    +    ValueVector vv = parent.addOrGet(name, mle.getMajorType(), vvc);
    +
    +    TypedFieldId pfid = container.getValueVectorId(SchemaPath.getSimplePath(parent.getField().getPath()));
    +    assert pfid.getFieldIds().length == 1;
    +    TypedFieldId.Builder builder = TypedFieldId.newBuilder();
    +    builder.addId(pfid.getFieldIds()[0]);
    +    TypedFieldId id = FieldIdUtil.getFieldIdIfMatches(parent, builder, true,
    +        SchemaPath.getSimplePath(vv.getField().getPath()).getRootSegment());
    +
    +    valueExprs.add(new ValueVectorWriteExpression(id, mle, true));
    +
    +    if (collector.hasErrors()) {
    +      throw new SchemaChangeException("Failure while materializing expression. "
    +          + collector.toErrorString());
    +    }
    +  }
    +
    +  private StreamingAggregator codegenAggregator(List<LogicalExpression> keyExprs,
    +      List<LogicalExpression> valueExprs, List<TypedFieldId> keyOutputIds)
    +          throws SchemaChangeException, ClassTransformationException, IOException {
    +    ClassGenerator<StreamingAggregator> cg =
    +        CodeGenerator.getRoot(StreamingAggTemplate.TEMPLATE_DEFINITION, context.getFunctionRegistry(),
    +            context.getOptions());
    +
    +    LogicalExpression[] keyExprsArray = new LogicalExpression[keyExprs.size()];
    +    LogicalExpression[] valueExprsArray = new LogicalExpression[valueExprs.size()];
    +    TypedFieldId[] keyOutputIdsArray = new TypedFieldId[keyOutputIds.size()];
    +
    +    keyExprs.toArray(keyExprsArray);
    +    valueExprs.toArray(valueExprsArray);
    +    keyOutputIds.toArray(keyOutputIdsArray);
    +
    +    setupIsSame(cg, keyExprsArray);
    +    setupIsSameApart(cg, keyExprsArray);
    +    addRecordValues(cg, valueExprsArray);
    +    outputRecordKeys(cg, keyOutputIdsArray, keyExprsArray);
    +    outputRecordKeysPrev(cg, keyOutputIdsArray, keyExprsArray);
    +
    +    cg.getBlock("resetValues")._return(JExpr.TRUE);
    +    getIndex(cg);
    +
    +    container.buildSchema(SelectionVectorMode.NONE);
    +    StreamingAggregator agg = context.getImplementationClass(cg);
    +    agg.setup(oContext, incoming, this);
    +    return agg;
    +  }
    +
    +  private boolean isImplicitFileColumn(MaterializedField mf) {
    +    return ImplicitColumnExplorer.initImplicitFileColumns(context.getOptions()).get(mf.getName()) != null;
    +  }
    +
    +  protected StreamingAggregator createAggregatorInternal()
    +      throws SchemaChangeException, ClassTransformationException, IOException {
    +    container.clear();
    +
    +    List<LogicalExpression> keyExprs = Lists.newArrayList();
    +    List<LogicalExpression> valueExprs = Lists.newArrayList();
    +    List<TypedFieldId> keyOutputIds = Lists.newArrayList();
    +    GregorianCalendar calendar = new GregorianCalendar(TimeZone.getTimeZone("UTC"));
    +    calendar.setTimeInMillis(System.currentTimeMillis());
    +
    +    if (this.getPopConfig() instanceof StatisticsAggregate
    +        && (((StatisticsAggregate) this.getPopConfig()).getPhase() == OperatorPhase.PHASE_1of1
    +            || ((StatisticsAggregate) this.getPopConfig()).getPhase() == OperatorPhase.PHASE_2of2)) {
    --- End diff --
    
    This code was refactored out.


---
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 #729: Drill 1328: Support table statistics for Parquet

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/729#discussion_r100677315
  
    --- Diff: exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/statistics/StatisticsAggBatch.java ---
    @@ -0,0 +1,256 @@
    +/**
    + * 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.impl.statistics;
    +
    +import com.google.common.collect.Lists;
    +import com.sun.codemodel.JExpr;
    +import org.apache.drill.common.expression.ErrorCollector;
    +import org.apache.drill.common.expression.ErrorCollectorImpl;
    +import org.apache.drill.common.expression.FunctionCallFactory;
    +import org.apache.drill.common.expression.LogicalExpression;
    +import org.apache.drill.common.expression.SchemaPath;
    +import org.apache.drill.common.expression.ValueExpressions;
    +import org.apache.drill.exec.exception.ClassTransformationException;
    +import org.apache.drill.exec.exception.OutOfMemoryException;
    +import org.apache.drill.exec.exception.SchemaChangeException;
    +import org.apache.drill.exec.expr.ClassGenerator;
    +import org.apache.drill.exec.expr.CodeGenerator;
    +import org.apache.drill.exec.expr.ExpressionTreeMaterializer;
    +import org.apache.drill.exec.expr.TypeHelper;
    +import org.apache.drill.exec.expr.ValueVectorWriteExpression;
    +import org.apache.drill.exec.ops.FragmentContext;
    +import org.apache.drill.exec.physical.config.StatisticsAggregate;
    +import org.apache.drill.exec.physical.impl.aggregate.StreamingAggBatch;
    +import org.apache.drill.exec.physical.impl.aggregate.StreamingAggTemplate;
    +import org.apache.drill.exec.physical.impl.aggregate.StreamingAggregator;
    +import org.apache.drill.exec.planner.physical.StatsAggPrel.OperatorPhase;
    +import org.apache.drill.exec.record.BatchSchema.SelectionVectorMode;
    +import org.apache.drill.exec.record.MaterializedField;
    +import org.apache.drill.exec.record.RecordBatch;
    +import org.apache.drill.exec.record.TypedFieldId;
    +import org.apache.drill.exec.store.ImplicitColumnExplorer;
    +import org.apache.drill.exec.vector.ValueVector;
    +import org.apache.drill.exec.vector.complex.FieldIdUtil;
    +import org.apache.drill.exec.vector.complex.MapVector;
    +
    +import java.io.IOException;
    +import java.util.GregorianCalendar;
    +import java.util.List;
    +import java.util.TimeZone;
    +
    +/**
    + * TODO: This needs cleanup. Currently the key values are constants and we compare the constants
    + * for every record. Seems unnecessary.
    + *
    + * Example input and output:
    + * Schema of incoming batch: region_id (VARCHAR), sales_city (VARCHAR), cnt (BIGINT)
    + * Schema of output:
    + *    "schema" : BIGINT - Schema number. For each schema change this number is incremented.
    + *    "computed" : BIGINT - What time is it computed?
    + *    "columns"       : MAP - Column names
    + *       "region_id"  : VARCHAR
    + *       "sales_city" : VARCHAR
    + *       "cnt"        : VARCHAR
    + *    "statscount" : MAP
    + *       "region_id"  : BIGINT - statscount(region_id) - aggregation over all values of region_id
    + *                      in incoming batch
    + *       "sales_city" : BIGINT - statscount(sales_city)
    + *       "cnt"        : BIGINT - statscount(cnt)
    + *    "nonnullstatcount" : MAP
    + *       "region_id"  : BIGINT - nonnullstatcount(region_id)
    + *       "sales_city" : BIGINT - nonnullstatcount(sales_city)
    + *       "cnt"        : BIGINT - nonnullstatcount(cnt)
    + *   .... another map for next stats function ....
    + */
    +public class StatisticsAggBatch extends StreamingAggBatch {
    +  private List<String> functions;
    +  private int schema = 0;
    +
    +  public StatisticsAggBatch(StatisticsAggregate popConfig, RecordBatch incoming,
    +      FragmentContext context) throws OutOfMemoryException {
    +    super(popConfig, incoming, context);
    +    this.functions = popConfig.getFunctions();
    +  }
    +
    +  private void createKeyColumn(String name, LogicalExpression expr, List<LogicalExpression> keyExprs,
    +      List<TypedFieldId> keyOutputIds) throws SchemaChangeException {
    +    ErrorCollector collector = new ErrorCollectorImpl();
    +
    +    LogicalExpression mle = ExpressionTreeMaterializer.materialize(expr, incoming, collector,
    +        context.getFunctionRegistry());
    +
    +    MaterializedField outputField = MaterializedField.create(name, mle.getMajorType());
    +    ValueVector vector = TypeHelper.getNewVector(outputField, oContext.getAllocator());
    +
    +    keyExprs.add(mle);
    +    keyOutputIds.add(container.add(vector));
    +
    +    if (collector.hasErrors()) {
    +      throw new SchemaChangeException("Failure while materializing expression. "
    +          + collector.toErrorString());
    +    }
    +  }
    +
    +  private void createNestedKeyColumn(MapVector parent, String name, LogicalExpression expr,
    +      List<LogicalExpression> keyExprs, List<TypedFieldId> keyOutputIds)
    +          throws SchemaChangeException {
    +    ErrorCollector collector = new ErrorCollectorImpl();
    +
    +    LogicalExpression mle = ExpressionTreeMaterializer.materialize(expr, incoming, collector,
    +        context.getFunctionRegistry());
    +
    +    Class<? extends ValueVector> vvc =
    +        TypeHelper.getValueVectorClass(mle.getMajorType().getMinorType(),
    +            mle.getMajorType().getMode());
    +
    +    ValueVector vv = parent.addOrGet(name, mle.getMajorType(), vvc);
    +
    +    TypedFieldId pfid = container.getValueVectorId(SchemaPath.getSimplePath(parent.getField().getPath()));
    +    assert pfid.getFieldIds().length == 1;
    +    TypedFieldId.Builder builder = TypedFieldId.newBuilder();
    +    builder.addId(pfid.getFieldIds()[0]);
    +    TypedFieldId id =
    +        FieldIdUtil.getFieldIdIfMatches(parent, builder, true,
    +            SchemaPath.getSimplePath(vv.getField().getPath()).getRootSegment());
    +
    +    keyExprs.add(mle);
    +    keyOutputIds.add(id);
    +
    +    if (collector.hasErrors()) {
    +      throw new SchemaChangeException("Failure while materializing expression. "
    +          + collector.toErrorString());
    +    }
    +  }
    +
    +  private void addMapVector(String name, MapVector parent, LogicalExpression expr,
    +      List<LogicalExpression> valueExprs) throws SchemaChangeException {
    +    ErrorCollector collector = new ErrorCollectorImpl();
    +
    +    LogicalExpression mle = ExpressionTreeMaterializer.materialize(expr, incoming, collector,
    +        context.getFunctionRegistry());
    +
    +    Class<? extends ValueVector> vvc =
    +        TypeHelper.getValueVectorClass(mle.getMajorType().getMinorType(), mle.getMajorType().getMode());
    +    ValueVector vv = parent.addOrGet(name, mle.getMajorType(), vvc);
    +
    +    TypedFieldId pfid = container.getValueVectorId(SchemaPath.getSimplePath(parent.getField().getPath()));
    +    assert pfid.getFieldIds().length == 1;
    +    TypedFieldId.Builder builder = TypedFieldId.newBuilder();
    +    builder.addId(pfid.getFieldIds()[0]);
    +    TypedFieldId id = FieldIdUtil.getFieldIdIfMatches(parent, builder, true,
    +        SchemaPath.getSimplePath(vv.getField().getPath()).getRootSegment());
    +
    +    valueExprs.add(new ValueVectorWriteExpression(id, mle, true));
    +
    +    if (collector.hasErrors()) {
    +      throw new SchemaChangeException("Failure while materializing expression. "
    +          + collector.toErrorString());
    +    }
    +  }
    +
    +  private StreamingAggregator codegenAggregator(List<LogicalExpression> keyExprs,
    +      List<LogicalExpression> valueExprs, List<TypedFieldId> keyOutputIds)
    +          throws SchemaChangeException, ClassTransformationException, IOException {
    +    ClassGenerator<StreamingAggregator> cg =
    +        CodeGenerator.getRoot(StreamingAggTemplate.TEMPLATE_DEFINITION, context.getFunctionRegistry(),
    +            context.getOptions());
    --- End diff --
    
    If you would like to debug this code, add the following lines:
    
    ```
        cg.getCodeGenerator().plainJavaCapable(true);
        // Uncomment out this line to debug the generated code.
    //    cg.getCodeGenerator().saveCodeForDebugging(true);
    ```


---
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 #729: Drill 1328: Support table statistics for Parquet

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/729#discussion_r100681088
  
    --- Diff: exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/unpivot/UnpivotMapsBatchCreator.java ---
    @@ -0,0 +1,40 @@
    +/**
    + * 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.physical.impl.unpivot;
    +
    +import java.util.List;
    +
    +import org.apache.drill.common.exceptions.ExecutionSetupException;
    +import org.apache.drill.exec.ops.FragmentContext;
    +import org.apache.drill.exec.physical.config.UnpivotMaps;
    +import org.apache.drill.exec.physical.impl.BatchCreator;
    +import org.apache.drill.exec.record.CloseableRecordBatch;
    +import org.apache.drill.exec.record.RecordBatch;
    +
    +import com.google.common.base.Preconditions;
    +
    +@SuppressWarnings("unused")
    --- End diff --
    
    Why unused?


---
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 #729: Drill 1328: Support table statistics for Parquet

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

    https://github.com/apache/drill/pull/729#discussion_r102323849
  
    --- Diff: exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/statistics/StatisticsMergeBatch.java ---
    @@ -0,0 +1,532 @@
    +/**
    + * 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.impl.statistics;
    +
    +import com.google.common.collect.Lists;
    +import com.google.common.collect.Maps;
    +import org.apache.drill.common.expression.LogicalExpression;
    +import org.apache.drill.common.expression.ErrorCollector;
    +import org.apache.drill.common.expression.ErrorCollectorImpl;
    +import org.apache.drill.common.expression.FunctionCallFactory;
    +import org.apache.drill.common.expression.SchemaPath;
    +import org.apache.drill.common.expression.ValueExpressions;
    +import org.apache.drill.common.types.TypeProtos;
    +import org.apache.drill.common.types.TypeProtos.MajorType;
    +import org.apache.drill.exec.exception.OutOfMemoryException;
    +import org.apache.drill.exec.exception.SchemaChangeException;
    +import org.apache.drill.exec.expr.ExpressionTreeMaterializer;
    +import org.apache.drill.exec.expr.TypeHelper;
    +import org.apache.drill.exec.expr.holders.BigIntHolder;
    +import org.apache.drill.exec.expr.holders.NullableFloat8Holder;
    +import org.apache.drill.exec.expr.holders.ObjectHolder;
    +import org.apache.drill.exec.expr.holders.ValueHolder;
    +import org.apache.drill.exec.ops.FragmentContext;
    +import org.apache.drill.exec.physical.config.StatisticsMerge;
    +import org.apache.drill.exec.record.AbstractSingleRecordBatch;
    +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.VectorContainer;
    +import org.apache.drill.exec.record.VectorWrapper;
    +import org.apache.drill.exec.vector.ValueVector;
    +import org.apache.drill.exec.vector.DateVector;
    +import org.apache.drill.exec.vector.BigIntVector;
    +import org.apache.drill.exec.vector.NullableBigIntVector;
    +import org.apache.drill.exec.vector.NullableFloat8Vector;
    +import org.apache.drill.exec.vector.NullableVarBinaryVector;
    +import org.apache.drill.exec.vector.VarCharVector;
    +import org.apache.drill.exec.vector.complex.MapVector;
    +
    +import java.io.ByteArrayInputStream;
    +import java.io.DataInputStream;
    +import java.io.IOException;
    +import java.util.List;
    +import java.util.GregorianCalendar;
    +import java.util.HashMap;
    +import java.util.Map;
    +import java.util.TimeZone;
    +
    +import com.clearspring.analytics.stream.cardinality.HyperLogLog;
    +
    +public class StatisticsMergeBatch extends AbstractSingleRecordBatch<StatisticsMerge> {
    +  private static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(StatisticsMergeBatch.class);
    +  private Map<String, String> functions;
    +  private boolean first = true;
    +  private boolean finished = false;
    +  private int schema = 0;
    +  private int recordCount = 0;
    +  private List<String> keyList = null;
    +  private Map<MaterializedField, ValueVector> dataSrcVecMap = null;
    +  // Map of non-map fields to VV in the incoming schema
    +  private Map<MaterializedField, ValueVector> copySrcVecMap = null;
    +  private Map<String, Map<String, ValueHolder>> aggregationMap = null;
    +  public StatisticsMergeBatch(StatisticsMerge popConfig, RecordBatch incoming,
    +                              FragmentContext context) throws OutOfMemoryException {
    +    super(popConfig, context, incoming);
    +    this.functions = new HashMap<>();
    +    this.aggregationMap = new HashMap<>();
    +
    +    /*for (String key : popConfig.getFunctions()) {
    +      aggregationMap.put(key, new HashMap<String, ValueHolder>());
    +      if (key.equalsIgnoreCase("statcount") || key.equalsIgnoreCase("nonnullstatcount")) {
    +        functions.put(key, "sum");
    +      } else if (key.equalsIgnoreCase("hll")) {
    +        functions.put(key, "hll_merge");
    +      } else if (key.equalsIgnoreCase("sum_width")) {
    +        functions.put(key, "avg_width");
    +      }
    +    }*/
    +    for (String key : popConfig.getFunctions()) {
    +      if (key.equalsIgnoreCase("sum_width")) {
    +        functions.put(key, "avg_width");
    +      } else if (key.equalsIgnoreCase("hll")) {
    +        functions.put(key, "hll_merge");
    +      } else {
    +        functions.put(key, key);
    +      }
    +      aggregationMap.put(functions.get(key), new HashMap<String, ValueHolder>());
    +    }
    +  }
    +
    +  private void createKeyColumn(String name, LogicalExpression expr, Map<MaterializedField, ValueVector> parentMap)
    +      throws SchemaChangeException {
    +    ErrorCollector collector = new ErrorCollectorImpl();
    +
    +    LogicalExpression mle = ExpressionTreeMaterializer.materialize(expr, incoming, collector,
    +        context.getFunctionRegistry());
    +
    +    MaterializedField outputField = MaterializedField.create(name, mle.getMajorType());
    +    ValueVector vector = TypeHelper.getNewVector(outputField, oContext.getAllocator());
    +    container.add(vector);
    +
    +    if (collector.hasErrors()) {
    +      throw new SchemaChangeException("Failure while materializing expression. "
    +          + collector.toErrorString());
    +    }
    +    parentMap.put(outputField, vector);
    +  }
    +
    +  private ValueVector addMapVector(String name, MapVector parent, LogicalExpression expr) throws SchemaChangeException {
    +    ErrorCollector collector = new ErrorCollectorImpl();
    +
    +    LogicalExpression mle = ExpressionTreeMaterializer.materialize(expr, incoming, collector,
    +        context.getFunctionRegistry());
    +
    +    Class<? extends ValueVector> vvc =
    +        TypeHelper.getValueVectorClass(mle.getMajorType().getMinorType(), mle.getMajorType().getMode());
    +    ValueVector vector = parent.addOrGet(name, mle.getMajorType(), vvc);
    +
    +    if (collector.hasErrors()) {
    +      throw new SchemaChangeException("Failure while materializing expression. "
    +          + collector.toErrorString());
    +    }
    +    return vector;
    +  }
    +
    +  /**
    +   * Identify the list of fields within a map which are unpivoted as columns in output
    +   */
    +  private void buildKeyList() {
    +    List<String> lastMapKeyList = null;
    +    for (VectorWrapper<?> vw : incoming) {
    +      if (vw.getField().getType().getMinorType() != TypeProtos.MinorType.MAP) {
    +        continue;
    +      }
    +
    +      keyList = Lists.newArrayList();
    +
    +      for (ValueVector vv : vw.getValueVector()) {
    +        keyList.add(vv.getField().getLastName());
    +      }
    +
    +      if (lastMapKeyList == null) {
    +        lastMapKeyList = keyList;
    +      } else {
    +        if (keyList.size() != lastMapKeyList.size() || !lastMapKeyList.containsAll(keyList)) {
    +          throw new UnsupportedOperationException("Maps have different fields");
    +        }
    +      }
    +    }
    +  }
    +
    +  private void buildOutputContainer() throws SchemaChangeException {
    +    dataSrcVecMap = Maps.newHashMap();
    +    copySrcVecMap = Maps.newHashMap();
    +    MajorType mt = null;
    +
    +    ErrorCollector collector = new ErrorCollectorImpl();
    +    GregorianCalendar calendar = new GregorianCalendar(TimeZone.getTimeZone("UTC"));
    +
    +    calendar.setTimeInMillis(System.currentTimeMillis());
    +    createKeyColumn("schema", ValueExpressions.getBigInt(schema++), copySrcVecMap);
    +    createKeyColumn("computed", ValueExpressions.getDate(calendar), copySrcVecMap);
    +
    +    for (VectorWrapper<?> vw : incoming) {
    +      addVectorToOutgoingContainer(vw.getField().getLastName(), vw, collector);
    +      /*MaterializedField ds = vw.getField();
    +      String field = vw.getField().getLastName();
    +      // Input map vector
    +      MapVector mapVector = (MapVector) vw.getValueVector();
    +      assert mapVector.getPrimitiveVectors().size() > 0;
    +      // Proceed to create output map vector with same name e.g. statcount etc.
    +      mt = mapVector.getField().getType();
    +      MaterializedField mf = MaterializedField.create(functions.get(field), mt);
    +      assert !dataSrcVecMap.containsKey(mf);
    +      ValueVector vector = TypeHelper.getNewVector(mf, oContext.getAllocator());
    +      container.add(vector);
    +      MapVector outputMapVector = (MapVector) vector;
    +
    +      for (ValueVector vv : mapVector) {
    +        String fieldName = vv.getField().getLastName();
    +        if (!keyList.contains(fieldName)) {
    +          throw new UnsupportedOperationException("Unpivot data vector " +
    +              ds + " contains key " + fieldName + " not contained in key source!");
    +        }
    +        if (vv.getField().getType().getMinorType() == TypeProtos.MinorType.MAP) {
    +          throw new UnsupportedOperationException("Unpivot of nested map is not supported!");
    +        }
    +        if (field.equals("column")) {
    +          outputMapVector.addOrGet(fieldName, vv.getField().getType(), vv.getClass());
    +        } else {
    +          List<LogicalExpression> args = Lists.newArrayList();
    +          //TODO: Something else to access value of col such as emp_id?
    +          args.add(SchemaPath.getSimplePath(vv.getField().getPath()));
    +          //TODO: Put in the mapVector
    +          LogicalExpression call = FunctionCallFactory.createExpression(functions.get(field), args);
    +          //TODO: Is this sufficient to add to new Map?
    +          ValueVector vector1 = addMapVector(fieldName, outputMapVector, call);
    +          if (collector.hasErrors()) {
    +            throw new SchemaChangeException("Failure while materializing expression. "
    +                + collector.toErrorString());
    +          }
    +        }
    +      }
    +      dataSrcVecMap.put(ds, outputMapVector);*/
    +    }
    +    //Now create NDV in the outgoing container which was not avaliable in the incoming
    +    for (VectorWrapper<?> vw : incoming) {
    +      if (vw.getField().getLastName().equalsIgnoreCase("sum_width")) {//NullableFloat8 type
    +        addVectorToOutgoingContainer("ndv", vw, collector);
    +        break;
    +      }
    +    }
    +    container.setRecordCount(0);
    +    recordCount = 0;
    +    container.buildSchema(incoming.getSchema().getSelectionVectorMode());
    +  }
    +
    +  private void addVectorToOutgoingContainer(String field, VectorWrapper vw, ErrorCollector collector)
    +     throws SchemaChangeException {
    +    // Input map vector
    +    MapVector mapVector = (MapVector) vw.getValueVector();
    +    MaterializedField mf;
    +    assert mapVector.getPrimitiveVectors().size() > 0;
    +    // Proceed to create output map vector with same name e.g. statcount etc.
    +    MajorType mt = mapVector.getField().getType();
    +    if (functions.get(field) != null) {
    +      mf = MaterializedField.create(functions.get(field), mt);
    +    } else {
    +      mf = MaterializedField.create(field, mt);
    +    }
    +    assert !dataSrcVecMap.containsKey(mf);
    +    ValueVector vector = TypeHelper.getNewVector(mf, oContext.getAllocator());
    +    container.add(vector);
    +    MapVector outputMapVector = (MapVector) vector;
    +
    +    for (ValueVector vv : mapVector) {
    +      String fieldName = vv.getField().getLastName();
    +      if (!keyList.contains(fieldName)) {
    +        throw new UnsupportedOperationException("Unpivot data vector " +
    +                field + " contains key " + fieldName + " not contained in key source!");
    +      }
    +      if (vv.getField().getType().getMinorType() == TypeProtos.MinorType.MAP) {
    +        throw new UnsupportedOperationException("Unpivot of nested map is not supported!");
    +      }
    +      if (field.equals("column")) {
    +        outputMapVector.addOrGet(fieldName, vv.getField().getType(), vv.getClass());
    +      } else {
    +        List<LogicalExpression> args = Lists.newArrayList();
    +        LogicalExpression call;
    +        //TODO: Something else to access value of col such as emp_id?
    +        args.add(SchemaPath.getSimplePath(vv.getField().getPath()));
    +        //TODO: Put in the mapVector
    +        if (functions.get(field) != null) {
    +          call = FunctionCallFactory.createExpression(functions.get(field), args);
    +        } else {
    +          call = FunctionCallFactory.createExpression(field, args);
    +        }
    +        //TODO: Is this sufficient to add to new Map?
    +        ValueVector vector1 = addMapVector(fieldName, outputMapVector, call);
    +        if (collector.hasErrors()) {
    +          throw new SchemaChangeException("Failure while materializing expression. "
    +                  + collector.toErrorString());
    +        }
    +      }
    +    }
    +  }
    +
    +  @Override
    +  protected boolean setupNewSchema() throws SchemaChangeException {
    +    container.clear();
    +    buildKeyList();
    +    buildOutputContainer();
    +    return true;
    +  }
    +
    +  @Override
    +  protected IterOutcome doWork() {
    +    int outRecordCount = incoming.getRecordCount();
    +    HashMap<String, Long> nonNullRows = new HashMap<>();
    +
    +    for (VectorWrapper<?> vw : incoming) {
    +      ValueVector vv = vw.getValueVector();
    +      if (vv.getField().getType().getMinorType() != TypeProtos.MinorType.MAP) {
    +        //We only expect Maps in the incoming. STOP, if this is not the case.
    +        return IterOutcome.STOP;
    +      }
    +      MapVector mapVec = (MapVector) vv;
    +      Map<String, ValueHolder> statMap = aggregationMap.get(vv.getField().getLastName());
    +
    +      for (ValueVector mapElt : mapVec) {
    +        if (vv.getField().getLastName().equalsIgnoreCase("column")) {
    +          BigIntHolder nameHolder;
    +          if (statMap.get(mapElt.getField().getLastName()) != null) {
    +            nameHolder = (BigIntHolder) statMap.get(mapElt.getField().getLastName());
    +          } else {
    +            nameHolder = new BigIntHolder();
    +            statMap.put(mapElt.getField().getLastName(), nameHolder);
    +          }
    +          nameHolder.value = 1;
    +        } else if (vv.getField().getLastName().equalsIgnoreCase("statcount")
    +            || vv.getField().getLastName().equalsIgnoreCase("nonnullstatcount")) {
    +          BigIntHolder sumHolder;
    +          String colName = mapElt.getField().getLastName();
    +          if (statMap.get(colName) != null) {
    +            sumHolder = (BigIntHolder) statMap.get(colName);
    +          } else {
    +            sumHolder = new BigIntHolder();
    +            statMap.put(colName, sumHolder);
    +          }
    +          //TODO: assert size = 1
    +          //TODO: logger
    +          Object val = mapElt.getAccessor().getObject(0);
    +          if (val != null) {
    +            sumHolder.value += (long)val;
    +          }
    +        } else if (vv.getField().getLastName().equalsIgnoreCase("sum_width")) {
    +          NullableFloat8Holder sumHolder;
    +          String colName = mapElt.getField().getLastName();
    +          if (statMap == null) {
    +            statMap = aggregationMap.get(functions.get(vv.getField().getLastName()));
    +          }
    +          if (statMap.get(colName) != null) {
    +            sumHolder = (NullableFloat8Holder) statMap.get(colName);
    +          } else {
    +            sumHolder = new NullableFloat8Holder();
    +            statMap.put(colName, sumHolder);
    +          }
    +          //TODO: assert size = 1
    +          //TODO: logger
    +          Object val = mapElt.getAccessor().getObject(0);
    +          if (val != null) {
    +            sumHolder.value += (double) val;
    +            sumHolder.isSet = 1;
    +          }
    +        } else if (vv.getField().getLastName().equalsIgnoreCase("hll")) {
    +          ObjectHolder hllHolder;
    +          String colName = mapElt.getField().getLastName();
    +          if (statMap == null) {
    +            statMap = aggregationMap.get(functions.get(vv.getField().getLastName()));
    +          }
    +          if (statMap.get(colName) != null) {
    +            hllHolder = (ObjectHolder) statMap.get(colName);
    +          } else {
    +            hllHolder = new ObjectHolder();
    +            hllHolder.obj = new HyperLogLog(context.getContextInformation().getHllAccuracy());
    +            statMap.put(colName, hllHolder);
    +          }
    +          NullableVarBinaryVector hllVector = (NullableVarBinaryVector) mapElt;
    +          try {
    +            if (hllVector.getAccessor().isSet(0) == 1) {
    +              ByteArrayInputStream bais = new ByteArrayInputStream(hllVector.getAccessor().getObject(0), 0,
    +                  mapElt.getBufferSize());
    +              HyperLogLog other = HyperLogLog.Builder.build(new DataInputStream(bais));
    +              ((HyperLogLog) hllHolder.obj).addAll(other);
    +            }
    +          } catch (Exception ex) {
    +            //TODO: Catch IOException/CardinalityMergeException
    +            //TODO: logger
    +            return IterOutcome.STOP;
    +          }
    +        }
    +      }
    +      // Add NDV value vector map using HLL map (since the NDV map is directly generated from HLL and not produced by the underlying
    +      // Statistics Agg)
    +      Map<String, ValueHolder> hllMap = aggregationMap.get("hll");
    +      if (hllMap != null) {
    +        aggregationMap.put("ndv", hllMap);
    +      }
    +    }
    +    return IterOutcome.OK;
    +  }
    +
    +  public VectorContainer getOutgoingContainer() {
    +    return this.container;
    +  }
    +
    +  @Override
    +  public IterOutcome innerNext() {
    +    IterOutcome outcome;
    +    boolean didSomeWork = false;
    +    if (finished) {
    +      return IterOutcome.NONE;
    +    }
    +    try {
    +      outer: while (true) {
    +        outcome = next(incoming);
    +        switch (outcome) {
    +          case NONE:
    +            break outer;
    +          case OUT_OF_MEMORY:
    +          case NOT_YET:
    +          case STOP:
    +            return outcome;
    +          case OK_NEW_SCHEMA:
    +            if (first) {
    +              first =false;
    +              if (!setupNewSchema()) {
    +                outcome = IterOutcome.OK;
    +              }
    +              return outcome;
    +            }
    +            //fall through
    +          case OK:
    +            assert first == false : "First batch should be OK_NEW_SCHEMA";
    +            IterOutcome out = doWork();
    +            didSomeWork = true;
    +            if (out != IterOutcome.OK) {
    +              return out;
    +            }
    +            break;
    +          default:
    +            throw new UnsupportedOperationException("Unsupported upstream state " + outcome);
    +        }
    +      }
    +    } catch (SchemaChangeException ex) {
    +      kill(false);
    +      logger.error("Failure during query", ex);
    +      context.fail(ex);
    --- End diff --
    
    Done. Please take a look.


---
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 #729: Drill 1328: Support table statistics for Parquet

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/729#discussion_r100677797
  
    --- Diff: exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/statistics/StatisticsAggBatch.java ---
    @@ -0,0 +1,256 @@
    +/**
    + * 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.impl.statistics;
    +
    +import com.google.common.collect.Lists;
    +import com.sun.codemodel.JExpr;
    +import org.apache.drill.common.expression.ErrorCollector;
    +import org.apache.drill.common.expression.ErrorCollectorImpl;
    +import org.apache.drill.common.expression.FunctionCallFactory;
    +import org.apache.drill.common.expression.LogicalExpression;
    +import org.apache.drill.common.expression.SchemaPath;
    +import org.apache.drill.common.expression.ValueExpressions;
    +import org.apache.drill.exec.exception.ClassTransformationException;
    +import org.apache.drill.exec.exception.OutOfMemoryException;
    +import org.apache.drill.exec.exception.SchemaChangeException;
    +import org.apache.drill.exec.expr.ClassGenerator;
    +import org.apache.drill.exec.expr.CodeGenerator;
    +import org.apache.drill.exec.expr.ExpressionTreeMaterializer;
    +import org.apache.drill.exec.expr.TypeHelper;
    +import org.apache.drill.exec.expr.ValueVectorWriteExpression;
    +import org.apache.drill.exec.ops.FragmentContext;
    +import org.apache.drill.exec.physical.config.StatisticsAggregate;
    +import org.apache.drill.exec.physical.impl.aggregate.StreamingAggBatch;
    +import org.apache.drill.exec.physical.impl.aggregate.StreamingAggTemplate;
    +import org.apache.drill.exec.physical.impl.aggregate.StreamingAggregator;
    +import org.apache.drill.exec.planner.physical.StatsAggPrel.OperatorPhase;
    +import org.apache.drill.exec.record.BatchSchema.SelectionVectorMode;
    +import org.apache.drill.exec.record.MaterializedField;
    +import org.apache.drill.exec.record.RecordBatch;
    +import org.apache.drill.exec.record.TypedFieldId;
    +import org.apache.drill.exec.store.ImplicitColumnExplorer;
    +import org.apache.drill.exec.vector.ValueVector;
    +import org.apache.drill.exec.vector.complex.FieldIdUtil;
    +import org.apache.drill.exec.vector.complex.MapVector;
    +
    +import java.io.IOException;
    +import java.util.GregorianCalendar;
    +import java.util.List;
    +import java.util.TimeZone;
    +
    +/**
    + * TODO: This needs cleanup. Currently the key values are constants and we compare the constants
    + * for every record. Seems unnecessary.
    + *
    + * Example input and output:
    + * Schema of incoming batch: region_id (VARCHAR), sales_city (VARCHAR), cnt (BIGINT)
    + * Schema of output:
    + *    "schema" : BIGINT - Schema number. For each schema change this number is incremented.
    + *    "computed" : BIGINT - What time is it computed?
    + *    "columns"       : MAP - Column names
    + *       "region_id"  : VARCHAR
    + *       "sales_city" : VARCHAR
    + *       "cnt"        : VARCHAR
    + *    "statscount" : MAP
    + *       "region_id"  : BIGINT - statscount(region_id) - aggregation over all values of region_id
    + *                      in incoming batch
    + *       "sales_city" : BIGINT - statscount(sales_city)
    + *       "cnt"        : BIGINT - statscount(cnt)
    + *    "nonnullstatcount" : MAP
    + *       "region_id"  : BIGINT - nonnullstatcount(region_id)
    + *       "sales_city" : BIGINT - nonnullstatcount(sales_city)
    + *       "cnt"        : BIGINT - nonnullstatcount(cnt)
    + *   .... another map for next stats function ....
    + */
    +public class StatisticsAggBatch extends StreamingAggBatch {
    +  private List<String> functions;
    +  private int schema = 0;
    +
    +  public StatisticsAggBatch(StatisticsAggregate popConfig, RecordBatch incoming,
    +      FragmentContext context) throws OutOfMemoryException {
    +    super(popConfig, incoming, context);
    +    this.functions = popConfig.getFunctions();
    +  }
    +
    +  private void createKeyColumn(String name, LogicalExpression expr, List<LogicalExpression> keyExprs,
    +      List<TypedFieldId> keyOutputIds) throws SchemaChangeException {
    +    ErrorCollector collector = new ErrorCollectorImpl();
    +
    +    LogicalExpression mle = ExpressionTreeMaterializer.materialize(expr, incoming, collector,
    +        context.getFunctionRegistry());
    +
    +    MaterializedField outputField = MaterializedField.create(name, mle.getMajorType());
    +    ValueVector vector = TypeHelper.getNewVector(outputField, oContext.getAllocator());
    +
    +    keyExprs.add(mle);
    +    keyOutputIds.add(container.add(vector));
    +
    +    if (collector.hasErrors()) {
    +      throw new SchemaChangeException("Failure while materializing expression. "
    +          + collector.toErrorString());
    +    }
    +  }
    +
    +  private void createNestedKeyColumn(MapVector parent, String name, LogicalExpression expr,
    +      List<LogicalExpression> keyExprs, List<TypedFieldId> keyOutputIds)
    +          throws SchemaChangeException {
    +    ErrorCollector collector = new ErrorCollectorImpl();
    +
    +    LogicalExpression mle = ExpressionTreeMaterializer.materialize(expr, incoming, collector,
    +        context.getFunctionRegistry());
    +
    +    Class<? extends ValueVector> vvc =
    +        TypeHelper.getValueVectorClass(mle.getMajorType().getMinorType(),
    +            mle.getMajorType().getMode());
    +
    +    ValueVector vv = parent.addOrGet(name, mle.getMajorType(), vvc);
    +
    +    TypedFieldId pfid = container.getValueVectorId(SchemaPath.getSimplePath(parent.getField().getPath()));
    +    assert pfid.getFieldIds().length == 1;
    +    TypedFieldId.Builder builder = TypedFieldId.newBuilder();
    +    builder.addId(pfid.getFieldIds()[0]);
    +    TypedFieldId id =
    +        FieldIdUtil.getFieldIdIfMatches(parent, builder, true,
    +            SchemaPath.getSimplePath(vv.getField().getPath()).getRootSegment());
    +
    +    keyExprs.add(mle);
    +    keyOutputIds.add(id);
    +
    +    if (collector.hasErrors()) {
    +      throw new SchemaChangeException("Failure while materializing expression. "
    +          + collector.toErrorString());
    +    }
    +  }
    +
    +  private void addMapVector(String name, MapVector parent, LogicalExpression expr,
    +      List<LogicalExpression> valueExprs) throws SchemaChangeException {
    +    ErrorCollector collector = new ErrorCollectorImpl();
    +
    +    LogicalExpression mle = ExpressionTreeMaterializer.materialize(expr, incoming, collector,
    +        context.getFunctionRegistry());
    +
    +    Class<? extends ValueVector> vvc =
    +        TypeHelper.getValueVectorClass(mle.getMajorType().getMinorType(), mle.getMajorType().getMode());
    +    ValueVector vv = parent.addOrGet(name, mle.getMajorType(), vvc);
    +
    +    TypedFieldId pfid = container.getValueVectorId(SchemaPath.getSimplePath(parent.getField().getPath()));
    +    assert pfid.getFieldIds().length == 1;
    +    TypedFieldId.Builder builder = TypedFieldId.newBuilder();
    +    builder.addId(pfid.getFieldIds()[0]);
    +    TypedFieldId id = FieldIdUtil.getFieldIdIfMatches(parent, builder, true,
    +        SchemaPath.getSimplePath(vv.getField().getPath()).getRootSegment());
    +
    +    valueExprs.add(new ValueVectorWriteExpression(id, mle, true));
    +
    +    if (collector.hasErrors()) {
    +      throw new SchemaChangeException("Failure while materializing expression. "
    +          + collector.toErrorString());
    +    }
    +  }
    +
    +  private StreamingAggregator codegenAggregator(List<LogicalExpression> keyExprs,
    +      List<LogicalExpression> valueExprs, List<TypedFieldId> keyOutputIds)
    +          throws SchemaChangeException, ClassTransformationException, IOException {
    +    ClassGenerator<StreamingAggregator> cg =
    +        CodeGenerator.getRoot(StreamingAggTemplate.TEMPLATE_DEFINITION, context.getFunctionRegistry(),
    +            context.getOptions());
    +
    +    LogicalExpression[] keyExprsArray = new LogicalExpression[keyExprs.size()];
    +    LogicalExpression[] valueExprsArray = new LogicalExpression[valueExprs.size()];
    +    TypedFieldId[] keyOutputIdsArray = new TypedFieldId[keyOutputIds.size()];
    +
    +    keyExprs.toArray(keyExprsArray);
    +    valueExprs.toArray(valueExprsArray);
    +    keyOutputIds.toArray(keyOutputIdsArray);
    +
    +    setupIsSame(cg, keyExprsArray);
    +    setupIsSameApart(cg, keyExprsArray);
    +    addRecordValues(cg, valueExprsArray);
    +    outputRecordKeys(cg, keyOutputIdsArray, keyExprsArray);
    +    outputRecordKeysPrev(cg, keyOutputIdsArray, keyExprsArray);
    +
    +    cg.getBlock("resetValues")._return(JExpr.TRUE);
    +    getIndex(cg);
    +
    +    container.buildSchema(SelectionVectorMode.NONE);
    +    StreamingAggregator agg = context.getImplementationClass(cg);
    +    agg.setup(oContext, incoming, this);
    +    return agg;
    +  }
    +
    +  private boolean isImplicitFileColumn(MaterializedField mf) {
    +    return ImplicitColumnExplorer.initImplicitFileColumns(context.getOptions()).get(mf.getName()) != null;
    +  }
    +
    +  protected StreamingAggregator createAggregatorInternal()
    +      throws SchemaChangeException, ClassTransformationException, IOException {
    +    container.clear();
    +
    +    List<LogicalExpression> keyExprs = Lists.newArrayList();
    +    List<LogicalExpression> valueExprs = Lists.newArrayList();
    +    List<TypedFieldId> keyOutputIds = Lists.newArrayList();
    +    GregorianCalendar calendar = new GregorianCalendar(TimeZone.getTimeZone("UTC"));
    +    calendar.setTimeInMillis(System.currentTimeMillis());
    +
    +    if (this.getPopConfig() instanceof StatisticsAggregate
    +        && (((StatisticsAggregate) this.getPopConfig()).getPhase() == OperatorPhase.PHASE_1of1
    +            || ((StatisticsAggregate) this.getPopConfig()).getPhase() == OperatorPhase.PHASE_2of2)) {
    +      createKeyColumn("schema",
    +          ValueExpressions.getBigInt(schema++),
    +          keyExprs,
    +          keyOutputIds
    +      );
    +      createKeyColumn("computed",
    +          ValueExpressions.getDate(calendar),
    +          keyExprs,
    +          keyOutputIds
    +      );
    +    }
    +
    +    MapVector cparent = new MapVector("column", oContext.getAllocator(), null);
    +    container.add(cparent);
    +    for (MaterializedField mf : incoming.getSchema()) {
    +      // Ignore implicit columns
    +      if (!isImplicitFileColumn(mf)) {
    +        createNestedKeyColumn(
    +            cparent,
    +            mf.getLastName(),
    +            ValueExpressions.getChar(mf.getLastName()),
    +            keyExprs,
    +            keyOutputIds
    +        );
    +      }
    +    }
    +
    +    for (String func : functions) {
    +      MapVector parent = new MapVector(func, oContext.getAllocator(), null);
    +      container.add(parent);
    +
    +      for (MaterializedField mf : incoming.getSchema()) {
    +        if (!isImplicitFileColumn(mf)) {
    +          List<LogicalExpression> args = Lists.newArrayList();
    +          args.add(SchemaPath.getSimplePath(mf.getPath()));
    +          LogicalExpression call = FunctionCallFactory.createExpression(func, args);
    +          addMapVector(mf.getLastName(), parent, call, valueExprs);
    --- End diff --
    
    Same issue as above: can't handle nested maps. And, we don't seem to do anything to reject maps. Is it OK to just do the wrong thing at runtime? Are users supposed to know not to apply stats to data with maps with reused column 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 #729: Drill 1328: Support table statistics for Parquet

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/729#discussion_r100677493
  
    --- Diff: exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/statistics/StatisticsAggBatch.java ---
    @@ -0,0 +1,256 @@
    +/**
    + * 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.impl.statistics;
    +
    +import com.google.common.collect.Lists;
    +import com.sun.codemodel.JExpr;
    +import org.apache.drill.common.expression.ErrorCollector;
    +import org.apache.drill.common.expression.ErrorCollectorImpl;
    +import org.apache.drill.common.expression.FunctionCallFactory;
    +import org.apache.drill.common.expression.LogicalExpression;
    +import org.apache.drill.common.expression.SchemaPath;
    +import org.apache.drill.common.expression.ValueExpressions;
    +import org.apache.drill.exec.exception.ClassTransformationException;
    +import org.apache.drill.exec.exception.OutOfMemoryException;
    +import org.apache.drill.exec.exception.SchemaChangeException;
    +import org.apache.drill.exec.expr.ClassGenerator;
    +import org.apache.drill.exec.expr.CodeGenerator;
    +import org.apache.drill.exec.expr.ExpressionTreeMaterializer;
    +import org.apache.drill.exec.expr.TypeHelper;
    +import org.apache.drill.exec.expr.ValueVectorWriteExpression;
    +import org.apache.drill.exec.ops.FragmentContext;
    +import org.apache.drill.exec.physical.config.StatisticsAggregate;
    +import org.apache.drill.exec.physical.impl.aggregate.StreamingAggBatch;
    +import org.apache.drill.exec.physical.impl.aggregate.StreamingAggTemplate;
    +import org.apache.drill.exec.physical.impl.aggregate.StreamingAggregator;
    +import org.apache.drill.exec.planner.physical.StatsAggPrel.OperatorPhase;
    +import org.apache.drill.exec.record.BatchSchema.SelectionVectorMode;
    +import org.apache.drill.exec.record.MaterializedField;
    +import org.apache.drill.exec.record.RecordBatch;
    +import org.apache.drill.exec.record.TypedFieldId;
    +import org.apache.drill.exec.store.ImplicitColumnExplorer;
    +import org.apache.drill.exec.vector.ValueVector;
    +import org.apache.drill.exec.vector.complex.FieldIdUtil;
    +import org.apache.drill.exec.vector.complex.MapVector;
    +
    +import java.io.IOException;
    +import java.util.GregorianCalendar;
    +import java.util.List;
    +import java.util.TimeZone;
    +
    +/**
    + * TODO: This needs cleanup. Currently the key values are constants and we compare the constants
    + * for every record. Seems unnecessary.
    + *
    + * Example input and output:
    + * Schema of incoming batch: region_id (VARCHAR), sales_city (VARCHAR), cnt (BIGINT)
    + * Schema of output:
    + *    "schema" : BIGINT - Schema number. For each schema change this number is incremented.
    + *    "computed" : BIGINT - What time is it computed?
    + *    "columns"       : MAP - Column names
    + *       "region_id"  : VARCHAR
    + *       "sales_city" : VARCHAR
    + *       "cnt"        : VARCHAR
    + *    "statscount" : MAP
    + *       "region_id"  : BIGINT - statscount(region_id) - aggregation over all values of region_id
    + *                      in incoming batch
    + *       "sales_city" : BIGINT - statscount(sales_city)
    + *       "cnt"        : BIGINT - statscount(cnt)
    + *    "nonnullstatcount" : MAP
    + *       "region_id"  : BIGINT - nonnullstatcount(region_id)
    + *       "sales_city" : BIGINT - nonnullstatcount(sales_city)
    + *       "cnt"        : BIGINT - nonnullstatcount(cnt)
    + *   .... another map for next stats function ....
    + */
    +public class StatisticsAggBatch extends StreamingAggBatch {
    +  private List<String> functions;
    +  private int schema = 0;
    +
    +  public StatisticsAggBatch(StatisticsAggregate popConfig, RecordBatch incoming,
    +      FragmentContext context) throws OutOfMemoryException {
    +    super(popConfig, incoming, context);
    +    this.functions = popConfig.getFunctions();
    +  }
    +
    +  private void createKeyColumn(String name, LogicalExpression expr, List<LogicalExpression> keyExprs,
    +      List<TypedFieldId> keyOutputIds) throws SchemaChangeException {
    +    ErrorCollector collector = new ErrorCollectorImpl();
    +
    +    LogicalExpression mle = ExpressionTreeMaterializer.materialize(expr, incoming, collector,
    +        context.getFunctionRegistry());
    +
    +    MaterializedField outputField = MaterializedField.create(name, mle.getMajorType());
    +    ValueVector vector = TypeHelper.getNewVector(outputField, oContext.getAllocator());
    +
    +    keyExprs.add(mle);
    +    keyOutputIds.add(container.add(vector));
    +
    +    if (collector.hasErrors()) {
    +      throw new SchemaChangeException("Failure while materializing expression. "
    +          + collector.toErrorString());
    +    }
    +  }
    +
    +  private void createNestedKeyColumn(MapVector parent, String name, LogicalExpression expr,
    +      List<LogicalExpression> keyExprs, List<TypedFieldId> keyOutputIds)
    +          throws SchemaChangeException {
    +    ErrorCollector collector = new ErrorCollectorImpl();
    +
    +    LogicalExpression mle = ExpressionTreeMaterializer.materialize(expr, incoming, collector,
    +        context.getFunctionRegistry());
    +
    +    Class<? extends ValueVector> vvc =
    +        TypeHelper.getValueVectorClass(mle.getMajorType().getMinorType(),
    +            mle.getMajorType().getMode());
    +
    +    ValueVector vv = parent.addOrGet(name, mle.getMajorType(), vvc);
    +
    +    TypedFieldId pfid = container.getValueVectorId(SchemaPath.getSimplePath(parent.getField().getPath()));
    +    assert pfid.getFieldIds().length == 1;
    +    TypedFieldId.Builder builder = TypedFieldId.newBuilder();
    +    builder.addId(pfid.getFieldIds()[0]);
    +    TypedFieldId id =
    +        FieldIdUtil.getFieldIdIfMatches(parent, builder, true,
    +            SchemaPath.getSimplePath(vv.getField().getPath()).getRootSegment());
    +
    +    keyExprs.add(mle);
    +    keyOutputIds.add(id);
    +
    +    if (collector.hasErrors()) {
    +      throw new SchemaChangeException("Failure while materializing expression. "
    +          + collector.toErrorString());
    +    }
    +  }
    +
    +  private void addMapVector(String name, MapVector parent, LogicalExpression expr,
    +      List<LogicalExpression> valueExprs) throws SchemaChangeException {
    +    ErrorCollector collector = new ErrorCollectorImpl();
    +
    +    LogicalExpression mle = ExpressionTreeMaterializer.materialize(expr, incoming, collector,
    +        context.getFunctionRegistry());
    +
    +    Class<? extends ValueVector> vvc =
    +        TypeHelper.getValueVectorClass(mle.getMajorType().getMinorType(), mle.getMajorType().getMode());
    +    ValueVector vv = parent.addOrGet(name, mle.getMajorType(), vvc);
    +
    +    TypedFieldId pfid = container.getValueVectorId(SchemaPath.getSimplePath(parent.getField().getPath()));
    +    assert pfid.getFieldIds().length == 1;
    +    TypedFieldId.Builder builder = TypedFieldId.newBuilder();
    +    builder.addId(pfid.getFieldIds()[0]);
    +    TypedFieldId id = FieldIdUtil.getFieldIdIfMatches(parent, builder, true,
    +        SchemaPath.getSimplePath(vv.getField().getPath()).getRootSegment());
    +
    +    valueExprs.add(new ValueVectorWriteExpression(id, mle, true));
    +
    +    if (collector.hasErrors()) {
    +      throw new SchemaChangeException("Failure while materializing expression. "
    +          + collector.toErrorString());
    +    }
    +  }
    +
    +  private StreamingAggregator codegenAggregator(List<LogicalExpression> keyExprs,
    +      List<LogicalExpression> valueExprs, List<TypedFieldId> keyOutputIds)
    +          throws SchemaChangeException, ClassTransformationException, IOException {
    +    ClassGenerator<StreamingAggregator> cg =
    +        CodeGenerator.getRoot(StreamingAggTemplate.TEMPLATE_DEFINITION, context.getFunctionRegistry(),
    +            context.getOptions());
    +
    +    LogicalExpression[] keyExprsArray = new LogicalExpression[keyExprs.size()];
    +    LogicalExpression[] valueExprsArray = new LogicalExpression[valueExprs.size()];
    +    TypedFieldId[] keyOutputIdsArray = new TypedFieldId[keyOutputIds.size()];
    +
    +    keyExprs.toArray(keyExprsArray);
    +    valueExprs.toArray(valueExprsArray);
    +    keyOutputIds.toArray(keyOutputIdsArray);
    +
    +    setupIsSame(cg, keyExprsArray);
    +    setupIsSameApart(cg, keyExprsArray);
    +    addRecordValues(cg, valueExprsArray);
    +    outputRecordKeys(cg, keyOutputIdsArray, keyExprsArray);
    +    outputRecordKeysPrev(cg, keyOutputIdsArray, keyExprsArray);
    +
    +    cg.getBlock("resetValues")._return(JExpr.TRUE);
    +    getIndex(cg);
    +
    +    container.buildSchema(SelectionVectorMode.NONE);
    +    StreamingAggregator agg = context.getImplementationClass(cg);
    +    agg.setup(oContext, incoming, this);
    +    return agg;
    +  }
    +
    +  private boolean isImplicitFileColumn(MaterializedField mf) {
    +    return ImplicitColumnExplorer.initImplicitFileColumns(context.getOptions()).get(mf.getName()) != null;
    +  }
    +
    +  protected StreamingAggregator createAggregatorInternal()
    +      throws SchemaChangeException, ClassTransformationException, IOException {
    +    container.clear();
    +
    +    List<LogicalExpression> keyExprs = Lists.newArrayList();
    +    List<LogicalExpression> valueExprs = Lists.newArrayList();
    +    List<TypedFieldId> keyOutputIds = Lists.newArrayList();
    +    GregorianCalendar calendar = new GregorianCalendar(TimeZone.getTimeZone("UTC"));
    +    calendar.setTimeInMillis(System.currentTimeMillis());
    +
    +    if (this.getPopConfig() instanceof StatisticsAggregate
    +        && (((StatisticsAggregate) this.getPopConfig()).getPhase() == OperatorPhase.PHASE_1of1
    +            || ((StatisticsAggregate) this.getPopConfig()).getPhase() == OperatorPhase.PHASE_2of2)) {
    --- End diff --
    
    It may be better to just store the `StatisticsAggregate` as a member variable in the constructor as we generally do in other operators.


---
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 #729: Drill 1328: Support table statistics for Parquet

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

    https://github.com/apache/drill/pull/729#discussion_r103366767
  
    --- Diff: exec/java-exec/src/main/java/org/apache/drill/exec/ops/QueryContext.java ---
    @@ -283,4 +288,22 @@ public void close() throws Exception {
           closed = true;
         }
       }
    +
    +  /**
    +  * @param stmtType : Sets the type {@link SqlStatementType} of the statement e.g. CTAS, ANALYZE
    +  */
    +  public void setSQLStatementType(SqlStatementType stmtType) {
    +    if (this.stmtType == null) {
    +      this.stmtType = stmtType;
    +    } else {
    +      throw new UnsupportedOperationException("SQL Statement type is already set");
    +    }
    +  }
    +
    +  /**
    +   * @return Get the type {@link SqlStatementType} of the statement e.g. CTAS, ANALYZE
    +   */
    +  public SqlStatementType getSQLStatementType() {
    +    return this.stmtType;
    --- 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 #729: Drill 1328: Support table statistics for Parquet

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

    https://github.com/apache/drill/pull/729#discussion_r103408205
  
    --- Diff: exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/statistics/MergedStatisticFactory.java ---
    @@ -0,0 +1,46 @@
    +/*
    + * 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.impl.statistics;
    +
    +public class MergedStatisticFactory {
    +  /*
    +   * Creates the appropriate statistics object given the name of the statistics and the input statistic
    +   */
    +  public static MergedStatistic getMergedStatistic(String outputStatName, String inputStatName) {
    +    if (outputStatName == null || inputStatName == null) {
    +      return null;
    +    } else if (outputStatName.equals(Statistic.COLNAME)) {
    +      return new ColumnMergedStatistic(outputStatName, inputStatName);
    +    } else if (outputStatName.equals(Statistic.COLTYPE)) {
    +      return new ColTypeMergedStatistic(outputStatName, inputStatName);
    +    } else if (outputStatName.equals(Statistic.STATCOUNT)) {
    +      return new StatCountMergedStatistic(outputStatName, inputStatName);
    +    } else if (outputStatName.equals(Statistic.NNSTATCOUNT)) {
    +      return new NNStatCountMergedStatistic(outputStatName, inputStatName);
    +    } else if (outputStatName.equals(Statistic.AVG_WIDTH)) {
    +      return new AvgWidthMergedStatistic(outputStatName, inputStatName);
    +    } else if (outputStatName.equals(Statistic.HLL_MERGE)) {
    +      return new HLLMergedStatistic(outputStatName, inputStatName);
    +    } else if (outputStatName.equals(Statistic.NDV)) {
    +      return new NDVMergedStatistic(outputStatName, inputStatName);
    +    } else {
    +      return null;
    --- 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 #729: Drill 1328: Support table statistics for Parquet

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

    https://github.com/apache/drill/pull/729#discussion_r103607754
  
    --- Diff: exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/statistics/StatisticsAggBatch.java ---
    @@ -0,0 +1,256 @@
    +/**
    + * 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.impl.statistics;
    +
    +import com.google.common.collect.Lists;
    +import com.sun.codemodel.JExpr;
    +import org.apache.drill.common.expression.ErrorCollector;
    +import org.apache.drill.common.expression.ErrorCollectorImpl;
    +import org.apache.drill.common.expression.FunctionCallFactory;
    +import org.apache.drill.common.expression.LogicalExpression;
    +import org.apache.drill.common.expression.SchemaPath;
    +import org.apache.drill.common.expression.ValueExpressions;
    +import org.apache.drill.exec.exception.ClassTransformationException;
    +import org.apache.drill.exec.exception.OutOfMemoryException;
    +import org.apache.drill.exec.exception.SchemaChangeException;
    +import org.apache.drill.exec.expr.ClassGenerator;
    +import org.apache.drill.exec.expr.CodeGenerator;
    +import org.apache.drill.exec.expr.ExpressionTreeMaterializer;
    +import org.apache.drill.exec.expr.TypeHelper;
    +import org.apache.drill.exec.expr.ValueVectorWriteExpression;
    +import org.apache.drill.exec.ops.FragmentContext;
    +import org.apache.drill.exec.physical.config.StatisticsAggregate;
    +import org.apache.drill.exec.physical.impl.aggregate.StreamingAggBatch;
    +import org.apache.drill.exec.physical.impl.aggregate.StreamingAggTemplate;
    +import org.apache.drill.exec.physical.impl.aggregate.StreamingAggregator;
    +import org.apache.drill.exec.planner.physical.StatsAggPrel.OperatorPhase;
    +import org.apache.drill.exec.record.BatchSchema.SelectionVectorMode;
    +import org.apache.drill.exec.record.MaterializedField;
    +import org.apache.drill.exec.record.RecordBatch;
    +import org.apache.drill.exec.record.TypedFieldId;
    +import org.apache.drill.exec.store.ImplicitColumnExplorer;
    +import org.apache.drill.exec.vector.ValueVector;
    +import org.apache.drill.exec.vector.complex.FieldIdUtil;
    +import org.apache.drill.exec.vector.complex.MapVector;
    +
    +import java.io.IOException;
    +import java.util.GregorianCalendar;
    +import java.util.List;
    +import java.util.TimeZone;
    +
    +/**
    + * TODO: This needs cleanup. Currently the key values are constants and we compare the constants
    + * for every record. Seems unnecessary.
    + *
    + * Example input and output:
    + * Schema of incoming batch: region_id (VARCHAR), sales_city (VARCHAR), cnt (BIGINT)
    + * Schema of output:
    + *    "schema" : BIGINT - Schema number. For each schema change this number is incremented.
    + *    "computed" : BIGINT - What time is it computed?
    + *    "columns"       : MAP - Column names
    + *       "region_id"  : VARCHAR
    + *       "sales_city" : VARCHAR
    + *       "cnt"        : VARCHAR
    + *    "statscount" : MAP
    + *       "region_id"  : BIGINT - statscount(region_id) - aggregation over all values of region_id
    + *                      in incoming batch
    + *       "sales_city" : BIGINT - statscount(sales_city)
    + *       "cnt"        : BIGINT - statscount(cnt)
    + *    "nonnullstatcount" : MAP
    + *       "region_id"  : BIGINT - nonnullstatcount(region_id)
    + *       "sales_city" : BIGINT - nonnullstatcount(sales_city)
    + *       "cnt"        : BIGINT - nonnullstatcount(cnt)
    + *   .... another map for next stats function ....
    + */
    +public class StatisticsAggBatch extends StreamingAggBatch {
    +  private List<String> functions;
    +  private int schema = 0;
    +
    +  public StatisticsAggBatch(StatisticsAggregate popConfig, RecordBatch incoming,
    +      FragmentContext context) throws OutOfMemoryException {
    +    super(popConfig, incoming, context);
    +    this.functions = popConfig.getFunctions();
    +  }
    +
    +  private void createKeyColumn(String name, LogicalExpression expr, List<LogicalExpression> keyExprs,
    +      List<TypedFieldId> keyOutputIds) throws SchemaChangeException {
    +    ErrorCollector collector = new ErrorCollectorImpl();
    +
    +    LogicalExpression mle = ExpressionTreeMaterializer.materialize(expr, incoming, collector,
    +        context.getFunctionRegistry());
    +
    +    MaterializedField outputField = MaterializedField.create(name, mle.getMajorType());
    +    ValueVector vector = TypeHelper.getNewVector(outputField, oContext.getAllocator());
    +
    +    keyExprs.add(mle);
    +    keyOutputIds.add(container.add(vector));
    +
    +    if (collector.hasErrors()) {
    +      throw new SchemaChangeException("Failure while materializing expression. "
    +          + collector.toErrorString());
    +    }
    +  }
    +
    +  private void createNestedKeyColumn(MapVector parent, String name, LogicalExpression expr,
    +      List<LogicalExpression> keyExprs, List<TypedFieldId> keyOutputIds)
    +          throws SchemaChangeException {
    +    ErrorCollector collector = new ErrorCollectorImpl();
    +
    +    LogicalExpression mle = ExpressionTreeMaterializer.materialize(expr, incoming, collector,
    +        context.getFunctionRegistry());
    +
    +    Class<? extends ValueVector> vvc =
    +        TypeHelper.getValueVectorClass(mle.getMajorType().getMinorType(),
    +            mle.getMajorType().getMode());
    +
    +    ValueVector vv = parent.addOrGet(name, mle.getMajorType(), vvc);
    +
    +    TypedFieldId pfid = container.getValueVectorId(SchemaPath.getSimplePath(parent.getField().getPath()));
    +    assert pfid.getFieldIds().length == 1;
    +    TypedFieldId.Builder builder = TypedFieldId.newBuilder();
    +    builder.addId(pfid.getFieldIds()[0]);
    +    TypedFieldId id =
    +        FieldIdUtil.getFieldIdIfMatches(parent, builder, true,
    +            SchemaPath.getSimplePath(vv.getField().getPath()).getRootSegment());
    +
    +    keyExprs.add(mle);
    +    keyOutputIds.add(id);
    +
    +    if (collector.hasErrors()) {
    +      throw new SchemaChangeException("Failure while materializing expression. "
    +          + collector.toErrorString());
    +    }
    +  }
    +
    +  private void addMapVector(String name, MapVector parent, LogicalExpression expr,
    +      List<LogicalExpression> valueExprs) throws SchemaChangeException {
    +    ErrorCollector collector = new ErrorCollectorImpl();
    +
    +    LogicalExpression mle = ExpressionTreeMaterializer.materialize(expr, incoming, collector,
    +        context.getFunctionRegistry());
    +
    +    Class<? extends ValueVector> vvc =
    +        TypeHelper.getValueVectorClass(mle.getMajorType().getMinorType(), mle.getMajorType().getMode());
    +    ValueVector vv = parent.addOrGet(name, mle.getMajorType(), vvc);
    +
    +    TypedFieldId pfid = container.getValueVectorId(SchemaPath.getSimplePath(parent.getField().getPath()));
    +    assert pfid.getFieldIds().length == 1;
    +    TypedFieldId.Builder builder = TypedFieldId.newBuilder();
    +    builder.addId(pfid.getFieldIds()[0]);
    +    TypedFieldId id = FieldIdUtil.getFieldIdIfMatches(parent, builder, true,
    +        SchemaPath.getSimplePath(vv.getField().getPath()).getRootSegment());
    +
    +    valueExprs.add(new ValueVectorWriteExpression(id, mle, true));
    +
    +    if (collector.hasErrors()) {
    +      throw new SchemaChangeException("Failure while materializing expression. "
    +          + collector.toErrorString());
    +    }
    +  }
    +
    +  private StreamingAggregator codegenAggregator(List<LogicalExpression> keyExprs,
    +      List<LogicalExpression> valueExprs, List<TypedFieldId> keyOutputIds)
    +          throws SchemaChangeException, ClassTransformationException, IOException {
    +    ClassGenerator<StreamingAggregator> cg =
    +        CodeGenerator.getRoot(StreamingAggTemplate.TEMPLATE_DEFINITION, context.getFunctionRegistry(),
    +            context.getOptions());
    +
    +    LogicalExpression[] keyExprsArray = new LogicalExpression[keyExprs.size()];
    +    LogicalExpression[] valueExprsArray = new LogicalExpression[valueExprs.size()];
    +    TypedFieldId[] keyOutputIdsArray = new TypedFieldId[keyOutputIds.size()];
    +
    +    keyExprs.toArray(keyExprsArray);
    +    valueExprs.toArray(valueExprsArray);
    +    keyOutputIds.toArray(keyOutputIdsArray);
    +
    +    setupIsSame(cg, keyExprsArray);
    +    setupIsSameApart(cg, keyExprsArray);
    +    addRecordValues(cg, valueExprsArray);
    +    outputRecordKeys(cg, keyOutputIdsArray, keyExprsArray);
    +    outputRecordKeysPrev(cg, keyOutputIdsArray, keyExprsArray);
    +
    +    cg.getBlock("resetValues")._return(JExpr.TRUE);
    +    getIndex(cg);
    +
    +    container.buildSchema(SelectionVectorMode.NONE);
    +    StreamingAggregator agg = context.getImplementationClass(cg);
    +    agg.setup(oContext, incoming, this);
    +    return agg;
    +  }
    +
    +  private boolean isImplicitFileColumn(MaterializedField mf) {
    +    return ImplicitColumnExplorer.initImplicitFileColumns(context.getOptions()).get(mf.getName()) != null;
    +  }
    +
    +  protected StreamingAggregator createAggregatorInternal()
    --- End diff --
    
    We override this method. Added the `@Override` annotation. Not sure if comments are disappearing - I remember adding a comment 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 #729: Drill 1328: Support table statistics for Parquet

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/729#discussion_r100677447
  
    --- Diff: exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/statistics/StatisticsAggBatch.java ---
    @@ -0,0 +1,256 @@
    +/**
    + * 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.impl.statistics;
    +
    +import com.google.common.collect.Lists;
    +import com.sun.codemodel.JExpr;
    +import org.apache.drill.common.expression.ErrorCollector;
    +import org.apache.drill.common.expression.ErrorCollectorImpl;
    +import org.apache.drill.common.expression.FunctionCallFactory;
    +import org.apache.drill.common.expression.LogicalExpression;
    +import org.apache.drill.common.expression.SchemaPath;
    +import org.apache.drill.common.expression.ValueExpressions;
    +import org.apache.drill.exec.exception.ClassTransformationException;
    +import org.apache.drill.exec.exception.OutOfMemoryException;
    +import org.apache.drill.exec.exception.SchemaChangeException;
    +import org.apache.drill.exec.expr.ClassGenerator;
    +import org.apache.drill.exec.expr.CodeGenerator;
    +import org.apache.drill.exec.expr.ExpressionTreeMaterializer;
    +import org.apache.drill.exec.expr.TypeHelper;
    +import org.apache.drill.exec.expr.ValueVectorWriteExpression;
    +import org.apache.drill.exec.ops.FragmentContext;
    +import org.apache.drill.exec.physical.config.StatisticsAggregate;
    +import org.apache.drill.exec.physical.impl.aggregate.StreamingAggBatch;
    +import org.apache.drill.exec.physical.impl.aggregate.StreamingAggTemplate;
    +import org.apache.drill.exec.physical.impl.aggregate.StreamingAggregator;
    +import org.apache.drill.exec.planner.physical.StatsAggPrel.OperatorPhase;
    +import org.apache.drill.exec.record.BatchSchema.SelectionVectorMode;
    +import org.apache.drill.exec.record.MaterializedField;
    +import org.apache.drill.exec.record.RecordBatch;
    +import org.apache.drill.exec.record.TypedFieldId;
    +import org.apache.drill.exec.store.ImplicitColumnExplorer;
    +import org.apache.drill.exec.vector.ValueVector;
    +import org.apache.drill.exec.vector.complex.FieldIdUtil;
    +import org.apache.drill.exec.vector.complex.MapVector;
    +
    +import java.io.IOException;
    +import java.util.GregorianCalendar;
    +import java.util.List;
    +import java.util.TimeZone;
    +
    +/**
    + * TODO: This needs cleanup. Currently the key values are constants and we compare the constants
    + * for every record. Seems unnecessary.
    + *
    + * Example input and output:
    + * Schema of incoming batch: region_id (VARCHAR), sales_city (VARCHAR), cnt (BIGINT)
    + * Schema of output:
    + *    "schema" : BIGINT - Schema number. For each schema change this number is incremented.
    + *    "computed" : BIGINT - What time is it computed?
    + *    "columns"       : MAP - Column names
    + *       "region_id"  : VARCHAR
    + *       "sales_city" : VARCHAR
    + *       "cnt"        : VARCHAR
    + *    "statscount" : MAP
    + *       "region_id"  : BIGINT - statscount(region_id) - aggregation over all values of region_id
    + *                      in incoming batch
    + *       "sales_city" : BIGINT - statscount(sales_city)
    + *       "cnt"        : BIGINT - statscount(cnt)
    + *    "nonnullstatcount" : MAP
    + *       "region_id"  : BIGINT - nonnullstatcount(region_id)
    + *       "sales_city" : BIGINT - nonnullstatcount(sales_city)
    + *       "cnt"        : BIGINT - nonnullstatcount(cnt)
    + *   .... another map for next stats function ....
    + */
    +public class StatisticsAggBatch extends StreamingAggBatch {
    +  private List<String> functions;
    +  private int schema = 0;
    +
    +  public StatisticsAggBatch(StatisticsAggregate popConfig, RecordBatch incoming,
    +      FragmentContext context) throws OutOfMemoryException {
    +    super(popConfig, incoming, context);
    +    this.functions = popConfig.getFunctions();
    +  }
    +
    +  private void createKeyColumn(String name, LogicalExpression expr, List<LogicalExpression> keyExprs,
    +      List<TypedFieldId> keyOutputIds) throws SchemaChangeException {
    +    ErrorCollector collector = new ErrorCollectorImpl();
    +
    +    LogicalExpression mle = ExpressionTreeMaterializer.materialize(expr, incoming, collector,
    +        context.getFunctionRegistry());
    +
    +    MaterializedField outputField = MaterializedField.create(name, mle.getMajorType());
    +    ValueVector vector = TypeHelper.getNewVector(outputField, oContext.getAllocator());
    +
    +    keyExprs.add(mle);
    +    keyOutputIds.add(container.add(vector));
    +
    +    if (collector.hasErrors()) {
    +      throw new SchemaChangeException("Failure while materializing expression. "
    +          + collector.toErrorString());
    +    }
    +  }
    +
    +  private void createNestedKeyColumn(MapVector parent, String name, LogicalExpression expr,
    +      List<LogicalExpression> keyExprs, List<TypedFieldId> keyOutputIds)
    +          throws SchemaChangeException {
    +    ErrorCollector collector = new ErrorCollectorImpl();
    +
    +    LogicalExpression mle = ExpressionTreeMaterializer.materialize(expr, incoming, collector,
    +        context.getFunctionRegistry());
    +
    +    Class<? extends ValueVector> vvc =
    +        TypeHelper.getValueVectorClass(mle.getMajorType().getMinorType(),
    +            mle.getMajorType().getMode());
    +
    +    ValueVector vv = parent.addOrGet(name, mle.getMajorType(), vvc);
    +
    +    TypedFieldId pfid = container.getValueVectorId(SchemaPath.getSimplePath(parent.getField().getPath()));
    +    assert pfid.getFieldIds().length == 1;
    +    TypedFieldId.Builder builder = TypedFieldId.newBuilder();
    +    builder.addId(pfid.getFieldIds()[0]);
    +    TypedFieldId id =
    +        FieldIdUtil.getFieldIdIfMatches(parent, builder, true,
    +            SchemaPath.getSimplePath(vv.getField().getPath()).getRootSegment());
    +
    +    keyExprs.add(mle);
    +    keyOutputIds.add(id);
    +
    +    if (collector.hasErrors()) {
    +      throw new SchemaChangeException("Failure while materializing expression. "
    +          + collector.toErrorString());
    +    }
    +  }
    +
    +  private void addMapVector(String name, MapVector parent, LogicalExpression expr,
    +      List<LogicalExpression> valueExprs) throws SchemaChangeException {
    +    ErrorCollector collector = new ErrorCollectorImpl();
    +
    +    LogicalExpression mle = ExpressionTreeMaterializer.materialize(expr, incoming, collector,
    +        context.getFunctionRegistry());
    +
    +    Class<? extends ValueVector> vvc =
    +        TypeHelper.getValueVectorClass(mle.getMajorType().getMinorType(), mle.getMajorType().getMode());
    +    ValueVector vv = parent.addOrGet(name, mle.getMajorType(), vvc);
    +
    +    TypedFieldId pfid = container.getValueVectorId(SchemaPath.getSimplePath(parent.getField().getPath()));
    +    assert pfid.getFieldIds().length == 1;
    +    TypedFieldId.Builder builder = TypedFieldId.newBuilder();
    +    builder.addId(pfid.getFieldIds()[0]);
    +    TypedFieldId id = FieldIdUtil.getFieldIdIfMatches(parent, builder, true,
    +        SchemaPath.getSimplePath(vv.getField().getPath()).getRootSegment());
    +
    +    valueExprs.add(new ValueVectorWriteExpression(id, mle, true));
    +
    +    if (collector.hasErrors()) {
    +      throw new SchemaChangeException("Failure while materializing expression. "
    +          + collector.toErrorString());
    +    }
    +  }
    +
    +  private StreamingAggregator codegenAggregator(List<LogicalExpression> keyExprs,
    +      List<LogicalExpression> valueExprs, List<TypedFieldId> keyOutputIds)
    +          throws SchemaChangeException, ClassTransformationException, IOException {
    +    ClassGenerator<StreamingAggregator> cg =
    +        CodeGenerator.getRoot(StreamingAggTemplate.TEMPLATE_DEFINITION, context.getFunctionRegistry(),
    +            context.getOptions());
    +
    +    LogicalExpression[] keyExprsArray = new LogicalExpression[keyExprs.size()];
    +    LogicalExpression[] valueExprsArray = new LogicalExpression[valueExprs.size()];
    +    TypedFieldId[] keyOutputIdsArray = new TypedFieldId[keyOutputIds.size()];
    +
    +    keyExprs.toArray(keyExprsArray);
    +    valueExprs.toArray(valueExprsArray);
    +    keyOutputIds.toArray(keyOutputIdsArray);
    +
    +    setupIsSame(cg, keyExprsArray);
    +    setupIsSameApart(cg, keyExprsArray);
    +    addRecordValues(cg, valueExprsArray);
    +    outputRecordKeys(cg, keyOutputIdsArray, keyExprsArray);
    +    outputRecordKeysPrev(cg, keyOutputIdsArray, keyExprsArray);
    +
    +    cg.getBlock("resetValues")._return(JExpr.TRUE);
    +    getIndex(cg);
    +
    +    container.buildSchema(SelectionVectorMode.NONE);
    +    StreamingAggregator agg = context.getImplementationClass(cg);
    +    agg.setup(oContext, incoming, this);
    +    return agg;
    +  }
    +
    +  private boolean isImplicitFileColumn(MaterializedField mf) {
    +    return ImplicitColumnExplorer.initImplicitFileColumns(context.getOptions()).get(mf.getName()) != null;
    +  }
    +
    +  protected StreamingAggregator createAggregatorInternal()
    +      throws SchemaChangeException, ClassTransformationException, IOException {
    +    container.clear();
    +
    +    List<LogicalExpression> keyExprs = Lists.newArrayList();
    +    List<LogicalExpression> valueExprs = Lists.newArrayList();
    +    List<TypedFieldId> keyOutputIds = Lists.newArrayList();
    +    GregorianCalendar calendar = new GregorianCalendar(TimeZone.getTimeZone("UTC"));
    +    calendar.setTimeInMillis(System.currentTimeMillis());
    +
    +    if (this.getPopConfig() instanceof StatisticsAggregate
    --- End diff --
    
    `this.` prefix not needed unless you feel strongly about it. But, the code does not use `this.` for other fields. Copy & paste artifact?


---
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 #729: Drill 1328: Support table statistics for Parquet

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/729#discussion_r100676723
  
    --- Diff: exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/aggregate/InternalBatch.java ---
    @@ -88,4 +91,7 @@ public void clear() {
         return container.getValueAccessorById(clazz, fieldIds);
       }
     
    +  public FragmentContext getContext() {
    +    return incoming.getContext();
    --- End diff --
    
    OK, so we hold onto the incoming batch just to get the FragmentContext. If so, please pass the FragmentContext into the constructor of this class rather than passing in the (irrelevant) incoming batch.


---
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 #729: Drill 1328: Support table statistics for Parquet

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

    https://github.com/apache/drill/pull/729#discussion_r103402346
  
    --- Diff: exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/statistics/AvgWidthMergedStatistic.java ---
    @@ -0,0 +1,135 @@
    +/*
    + * 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.impl.statistics;
    +
    +import org.apache.drill.common.types.TypeProtos;
    +import org.apache.drill.exec.expr.holders.NullableFloat8Holder;
    +import org.apache.drill.exec.expr.holders.ValueHolder;
    +import org.apache.drill.exec.vector.NullableFloat8Vector;
    +import org.apache.drill.exec.vector.ValueVector;
    +import org.apache.drill.exec.vector.complex.MapVector;
    +
    +import java.util.HashMap;
    +import java.util.List;
    +import java.util.Map;
    +
    +public class AvgWidthMergedStatistic extends AbstractMergedStatistic {
    +
    +  private String name;
    +  private String inputName;
    +  private boolean configureComplete = false;
    +  private boolean mergeComplete = false;
    +  private Map<String, ValueHolder> sumHolder;
    +  MergedStatistic types, nonNullStatCounts, statCounts;
    +
    +  public AvgWidthMergedStatistic (String name, String inputName) {
    +    this.name = name;
    +    this.inputName = inputName;
    +    this.sumHolder = new HashMap<>();
    +    types = nonNullStatCounts = statCounts = null;
    +  }
    +
    +  @Override
    +  public String getName() {
    +    return name;
    +  }
    +
    +  @Override
    +  public String getInput() {
    +    return inputName;
    +  }
    +
    +  @Override
    +  public void merge(ValueVector input) {
    +    // Check the input is a Map Vector
    +    assert (input.getField().getType().getMinorType() == TypeProtos.MinorType.MAP);
    +    MapVector inputMap = (MapVector) input;
    +    for (ValueVector vv : inputMap) {
    +      String colName = vv.getField().getLastName();
    +      NullableFloat8Holder colSumHolder;
    +      if (sumHolder.get(colName) != null) {
    +        colSumHolder = (NullableFloat8Holder) sumHolder.get(colName);
    +      } else {
    +        colSumHolder = new NullableFloat8Holder();
    +        sumHolder.put(colName, colSumHolder);
    +      }
    +      Object val = vv.getAccessor().getObject(0);
    --- 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 #729: Drill 1328: Support table statistics for Parquet

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/729#discussion_r100677070
  
    --- Diff: exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/statistics/StatisticsAggBatch.java ---
    @@ -0,0 +1,256 @@
    +/**
    + * 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.impl.statistics;
    +
    +import com.google.common.collect.Lists;
    +import com.sun.codemodel.JExpr;
    +import org.apache.drill.common.expression.ErrorCollector;
    +import org.apache.drill.common.expression.ErrorCollectorImpl;
    +import org.apache.drill.common.expression.FunctionCallFactory;
    +import org.apache.drill.common.expression.LogicalExpression;
    +import org.apache.drill.common.expression.SchemaPath;
    +import org.apache.drill.common.expression.ValueExpressions;
    +import org.apache.drill.exec.exception.ClassTransformationException;
    +import org.apache.drill.exec.exception.OutOfMemoryException;
    +import org.apache.drill.exec.exception.SchemaChangeException;
    +import org.apache.drill.exec.expr.ClassGenerator;
    +import org.apache.drill.exec.expr.CodeGenerator;
    +import org.apache.drill.exec.expr.ExpressionTreeMaterializer;
    +import org.apache.drill.exec.expr.TypeHelper;
    +import org.apache.drill.exec.expr.ValueVectorWriteExpression;
    +import org.apache.drill.exec.ops.FragmentContext;
    +import org.apache.drill.exec.physical.config.StatisticsAggregate;
    +import org.apache.drill.exec.physical.impl.aggregate.StreamingAggBatch;
    +import org.apache.drill.exec.physical.impl.aggregate.StreamingAggTemplate;
    +import org.apache.drill.exec.physical.impl.aggregate.StreamingAggregator;
    +import org.apache.drill.exec.planner.physical.StatsAggPrel.OperatorPhase;
    +import org.apache.drill.exec.record.BatchSchema.SelectionVectorMode;
    +import org.apache.drill.exec.record.MaterializedField;
    +import org.apache.drill.exec.record.RecordBatch;
    +import org.apache.drill.exec.record.TypedFieldId;
    +import org.apache.drill.exec.store.ImplicitColumnExplorer;
    +import org.apache.drill.exec.vector.ValueVector;
    +import org.apache.drill.exec.vector.complex.FieldIdUtil;
    +import org.apache.drill.exec.vector.complex.MapVector;
    +
    +import java.io.IOException;
    +import java.util.GregorianCalendar;
    +import java.util.List;
    +import java.util.TimeZone;
    +
    +/**
    + * TODO: This needs cleanup. Currently the key values are constants and we compare the constants
    + * for every record. Seems unnecessary.
    + *
    + * Example input and output:
    + * Schema of incoming batch: region_id (VARCHAR), sales_city (VARCHAR), cnt (BIGINT)
    + * Schema of output:
    + *    "schema" : BIGINT - Schema number. For each schema change this number is incremented.
    + *    "computed" : BIGINT - What time is it computed?
    + *    "columns"       : MAP - Column names
    + *       "region_id"  : VARCHAR
    + *       "sales_city" : VARCHAR
    + *       "cnt"        : VARCHAR
    + *    "statscount" : MAP
    + *       "region_id"  : BIGINT - statscount(region_id) - aggregation over all values of region_id
    + *                      in incoming batch
    + *       "sales_city" : BIGINT - statscount(sales_city)
    + *       "cnt"        : BIGINT - statscount(cnt)
    + *    "nonnullstatcount" : MAP
    --- End diff --
    
    So the idea is that there is some set of stats functions. each gives rise to a map field? And within the map, we have the field name as key, and output of the function as a value? That value must always be a BIGINT, or the type of the map values depends on the output type of the function?


---
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 #729: Drill 1328 r4

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/729#discussion_r99390401
  
    --- Diff: exec/java-exec/src/main/codegen/data/Parser.tdd ---
    @@ -39,7 +39,13 @@
         "METADATA",
         "DATABASE",
         "IF",
    -    "JAR"
    +    "JAR",
    +    "ANALYZE",
    +    "COMPUTE",
    +    "ESTIMATE",
    +    "STATISTICS",
    +    "SAMPLE",
    +    "PERCENT"
    --- End diff --
    
    Adding so many "good" words as keywords will break existing queries:
    ```
    SELECT estimate, percent FROM sample
    ```
    Would, presumably, work prior to this commit, but will fail after. Or, were those keywords already reserved?
    
    Do we have to consider backward compatibility 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 #729: Drill 1328: Support table statistics for Parquet

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/729#discussion_r100677336
  
    --- Diff: exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/statistics/StatisticsAggBatch.java ---
    @@ -0,0 +1,256 @@
    +/**
    + * 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.impl.statistics;
    +
    +import com.google.common.collect.Lists;
    +import com.sun.codemodel.JExpr;
    +import org.apache.drill.common.expression.ErrorCollector;
    +import org.apache.drill.common.expression.ErrorCollectorImpl;
    +import org.apache.drill.common.expression.FunctionCallFactory;
    +import org.apache.drill.common.expression.LogicalExpression;
    +import org.apache.drill.common.expression.SchemaPath;
    +import org.apache.drill.common.expression.ValueExpressions;
    +import org.apache.drill.exec.exception.ClassTransformationException;
    +import org.apache.drill.exec.exception.OutOfMemoryException;
    +import org.apache.drill.exec.exception.SchemaChangeException;
    +import org.apache.drill.exec.expr.ClassGenerator;
    +import org.apache.drill.exec.expr.CodeGenerator;
    +import org.apache.drill.exec.expr.ExpressionTreeMaterializer;
    +import org.apache.drill.exec.expr.TypeHelper;
    +import org.apache.drill.exec.expr.ValueVectorWriteExpression;
    +import org.apache.drill.exec.ops.FragmentContext;
    +import org.apache.drill.exec.physical.config.StatisticsAggregate;
    +import org.apache.drill.exec.physical.impl.aggregate.StreamingAggBatch;
    +import org.apache.drill.exec.physical.impl.aggregate.StreamingAggTemplate;
    +import org.apache.drill.exec.physical.impl.aggregate.StreamingAggregator;
    +import org.apache.drill.exec.planner.physical.StatsAggPrel.OperatorPhase;
    +import org.apache.drill.exec.record.BatchSchema.SelectionVectorMode;
    +import org.apache.drill.exec.record.MaterializedField;
    +import org.apache.drill.exec.record.RecordBatch;
    +import org.apache.drill.exec.record.TypedFieldId;
    +import org.apache.drill.exec.store.ImplicitColumnExplorer;
    +import org.apache.drill.exec.vector.ValueVector;
    +import org.apache.drill.exec.vector.complex.FieldIdUtil;
    +import org.apache.drill.exec.vector.complex.MapVector;
    +
    +import java.io.IOException;
    +import java.util.GregorianCalendar;
    +import java.util.List;
    +import java.util.TimeZone;
    +
    +/**
    + * TODO: This needs cleanup. Currently the key values are constants and we compare the constants
    + * for every record. Seems unnecessary.
    + *
    + * Example input and output:
    + * Schema of incoming batch: region_id (VARCHAR), sales_city (VARCHAR), cnt (BIGINT)
    + * Schema of output:
    + *    "schema" : BIGINT - Schema number. For each schema change this number is incremented.
    + *    "computed" : BIGINT - What time is it computed?
    + *    "columns"       : MAP - Column names
    + *       "region_id"  : VARCHAR
    + *       "sales_city" : VARCHAR
    + *       "cnt"        : VARCHAR
    + *    "statscount" : MAP
    + *       "region_id"  : BIGINT - statscount(region_id) - aggregation over all values of region_id
    + *                      in incoming batch
    + *       "sales_city" : BIGINT - statscount(sales_city)
    + *       "cnt"        : BIGINT - statscount(cnt)
    + *    "nonnullstatcount" : MAP
    + *       "region_id"  : BIGINT - nonnullstatcount(region_id)
    + *       "sales_city" : BIGINT - nonnullstatcount(sales_city)
    + *       "cnt"        : BIGINT - nonnullstatcount(cnt)
    + *   .... another map for next stats function ....
    + */
    +public class StatisticsAggBatch extends StreamingAggBatch {
    +  private List<String> functions;
    +  private int schema = 0;
    +
    +  public StatisticsAggBatch(StatisticsAggregate popConfig, RecordBatch incoming,
    +      FragmentContext context) throws OutOfMemoryException {
    +    super(popConfig, incoming, context);
    +    this.functions = popConfig.getFunctions();
    +  }
    +
    +  private void createKeyColumn(String name, LogicalExpression expr, List<LogicalExpression> keyExprs,
    +      List<TypedFieldId> keyOutputIds) throws SchemaChangeException {
    +    ErrorCollector collector = new ErrorCollectorImpl();
    +
    +    LogicalExpression mle = ExpressionTreeMaterializer.materialize(expr, incoming, collector,
    +        context.getFunctionRegistry());
    +
    +    MaterializedField outputField = MaterializedField.create(name, mle.getMajorType());
    +    ValueVector vector = TypeHelper.getNewVector(outputField, oContext.getAllocator());
    +
    +    keyExprs.add(mle);
    +    keyOutputIds.add(container.add(vector));
    +
    +    if (collector.hasErrors()) {
    +      throw new SchemaChangeException("Failure while materializing expression. "
    +          + collector.toErrorString());
    +    }
    +  }
    +
    +  private void createNestedKeyColumn(MapVector parent, String name, LogicalExpression expr,
    +      List<LogicalExpression> keyExprs, List<TypedFieldId> keyOutputIds)
    +          throws SchemaChangeException {
    +    ErrorCollector collector = new ErrorCollectorImpl();
    +
    +    LogicalExpression mle = ExpressionTreeMaterializer.materialize(expr, incoming, collector,
    +        context.getFunctionRegistry());
    +
    +    Class<? extends ValueVector> vvc =
    +        TypeHelper.getValueVectorClass(mle.getMajorType().getMinorType(),
    +            mle.getMajorType().getMode());
    +
    +    ValueVector vv = parent.addOrGet(name, mle.getMajorType(), vvc);
    +
    +    TypedFieldId pfid = container.getValueVectorId(SchemaPath.getSimplePath(parent.getField().getPath()));
    +    assert pfid.getFieldIds().length == 1;
    +    TypedFieldId.Builder builder = TypedFieldId.newBuilder();
    +    builder.addId(pfid.getFieldIds()[0]);
    +    TypedFieldId id =
    +        FieldIdUtil.getFieldIdIfMatches(parent, builder, true,
    +            SchemaPath.getSimplePath(vv.getField().getPath()).getRootSegment());
    +
    +    keyExprs.add(mle);
    +    keyOutputIds.add(id);
    +
    +    if (collector.hasErrors()) {
    +      throw new SchemaChangeException("Failure while materializing expression. "
    +          + collector.toErrorString());
    +    }
    +  }
    +
    +  private void addMapVector(String name, MapVector parent, LogicalExpression expr,
    +      List<LogicalExpression> valueExprs) throws SchemaChangeException {
    +    ErrorCollector collector = new ErrorCollectorImpl();
    +
    +    LogicalExpression mle = ExpressionTreeMaterializer.materialize(expr, incoming, collector,
    +        context.getFunctionRegistry());
    +
    +    Class<? extends ValueVector> vvc =
    +        TypeHelper.getValueVectorClass(mle.getMajorType().getMinorType(), mle.getMajorType().getMode());
    +    ValueVector vv = parent.addOrGet(name, mle.getMajorType(), vvc);
    +
    +    TypedFieldId pfid = container.getValueVectorId(SchemaPath.getSimplePath(parent.getField().getPath()));
    +    assert pfid.getFieldIds().length == 1;
    +    TypedFieldId.Builder builder = TypedFieldId.newBuilder();
    +    builder.addId(pfid.getFieldIds()[0]);
    +    TypedFieldId id = FieldIdUtil.getFieldIdIfMatches(parent, builder, true,
    +        SchemaPath.getSimplePath(vv.getField().getPath()).getRootSegment());
    +
    +    valueExprs.add(new ValueVectorWriteExpression(id, mle, true));
    +
    +    if (collector.hasErrors()) {
    +      throw new SchemaChangeException("Failure while materializing expression. "
    +          + collector.toErrorString());
    +    }
    +  }
    +
    +  private StreamingAggregator codegenAggregator(List<LogicalExpression> keyExprs,
    +      List<LogicalExpression> valueExprs, List<TypedFieldId> keyOutputIds)
    +          throws SchemaChangeException, ClassTransformationException, IOException {
    +    ClassGenerator<StreamingAggregator> cg =
    +        CodeGenerator.getRoot(StreamingAggTemplate.TEMPLATE_DEFINITION, context.getFunctionRegistry(),
    +            context.getOptions());
    +
    +    LogicalExpression[] keyExprsArray = new LogicalExpression[keyExprs.size()];
    +    LogicalExpression[] valueExprsArray = new LogicalExpression[valueExprs.size()];
    +    TypedFieldId[] keyOutputIdsArray = new TypedFieldId[keyOutputIds.size()];
    +
    +    keyExprs.toArray(keyExprsArray);
    +    valueExprs.toArray(valueExprsArray);
    +    keyOutputIds.toArray(keyOutputIdsArray);
    +
    +    setupIsSame(cg, keyExprsArray);
    +    setupIsSameApart(cg, keyExprsArray);
    +    addRecordValues(cg, valueExprsArray);
    +    outputRecordKeys(cg, keyOutputIdsArray, keyExprsArray);
    +    outputRecordKeysPrev(cg, keyOutputIdsArray, keyExprsArray);
    +
    +    cg.getBlock("resetValues")._return(JExpr.TRUE);
    +    getIndex(cg);
    +
    +    container.buildSchema(SelectionVectorMode.NONE);
    +    StreamingAggregator agg = context.getImplementationClass(cg);
    +    agg.setup(oContext, incoming, this);
    +    return agg;
    +  }
    +
    +  private boolean isImplicitFileColumn(MaterializedField mf) {
    +    return ImplicitColumnExplorer.initImplicitFileColumns(context.getOptions()).get(mf.getName()) != null;
    --- End diff --
    
    Performance issue? We build the implicit column explorer, it seems for each and every column. If the table has 100 columns we build it 100 times?


---
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 #729: Drill 1328: Support table statistics for Parquet

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/729#discussion_r100676978
  
    --- Diff: exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/statistics/StatisticsAggBatch.java ---
    @@ -0,0 +1,256 @@
    +/**
    + * 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.impl.statistics;
    +
    +import com.google.common.collect.Lists;
    +import com.sun.codemodel.JExpr;
    +import org.apache.drill.common.expression.ErrorCollector;
    +import org.apache.drill.common.expression.ErrorCollectorImpl;
    +import org.apache.drill.common.expression.FunctionCallFactory;
    +import org.apache.drill.common.expression.LogicalExpression;
    +import org.apache.drill.common.expression.SchemaPath;
    +import org.apache.drill.common.expression.ValueExpressions;
    +import org.apache.drill.exec.exception.ClassTransformationException;
    +import org.apache.drill.exec.exception.OutOfMemoryException;
    +import org.apache.drill.exec.exception.SchemaChangeException;
    +import org.apache.drill.exec.expr.ClassGenerator;
    +import org.apache.drill.exec.expr.CodeGenerator;
    +import org.apache.drill.exec.expr.ExpressionTreeMaterializer;
    +import org.apache.drill.exec.expr.TypeHelper;
    +import org.apache.drill.exec.expr.ValueVectorWriteExpression;
    +import org.apache.drill.exec.ops.FragmentContext;
    +import org.apache.drill.exec.physical.config.StatisticsAggregate;
    +import org.apache.drill.exec.physical.impl.aggregate.StreamingAggBatch;
    +import org.apache.drill.exec.physical.impl.aggregate.StreamingAggTemplate;
    +import org.apache.drill.exec.physical.impl.aggregate.StreamingAggregator;
    +import org.apache.drill.exec.planner.physical.StatsAggPrel.OperatorPhase;
    +import org.apache.drill.exec.record.BatchSchema.SelectionVectorMode;
    +import org.apache.drill.exec.record.MaterializedField;
    +import org.apache.drill.exec.record.RecordBatch;
    +import org.apache.drill.exec.record.TypedFieldId;
    +import org.apache.drill.exec.store.ImplicitColumnExplorer;
    +import org.apache.drill.exec.vector.ValueVector;
    +import org.apache.drill.exec.vector.complex.FieldIdUtil;
    +import org.apache.drill.exec.vector.complex.MapVector;
    +
    +import java.io.IOException;
    +import java.util.GregorianCalendar;
    +import java.util.List;
    +import java.util.TimeZone;
    +
    +/**
    + * TODO: This needs cleanup. Currently the key values are constants and we compare the constants
    + * for every record. Seems unnecessary.
    + *
    + * Example input and output:
    + * Schema of incoming batch: region_id (VARCHAR), sales_city (VARCHAR), cnt (BIGINT)
    + * Schema of output:
    + *    "schema" : BIGINT - Schema number. For each schema change this number is incremented.
    + *    "computed" : BIGINT - What time is it computed?
    + *    "columns"       : MAP - Column names
    --- End diff --
    
    Maps are key/value collections. (A map in Drill is really a nested record or tuple.)
    
    Should I infer that the map contains column names as keys, types as a values? How are types encoded? The name associated with the type in MinorType? Or, do we also include the additional MajorType info such as Required/Nullable/Repeated? For decimals, do we also track precision, etc?
    
    For names, if the name is nested, is this the full path name "a.b"? Are names case sensitive?


---
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 #729: Drill 1328: Support table statistics for Parquet

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/729#discussion_r102868884
  
    --- Diff: exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/statistics/AbstractMergedStatistic.java ---
    @@ -0,0 +1,52 @@
    +/*
    + * 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.impl.statistics;
    +
    +import org.apache.drill.exec.vector.ValueVector;
    +
    +public abstract class AbstractMergedStatistic extends Statistic implements MergedStatistic {
    +  @Override
    +  public String getName() {
    +    throw new UnsupportedOperationException("getName() not implemented");
    +  }
    --- End diff --
    
    The classic way to do this is:
    
    ```
    public abstract String getName();
    ```
    
    Or, since this is implementing/extending some other class, just leave the method unimplemented. This way, the compiler will tell you that you forgot to implement a required method.


---
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 #729: Drill 1328: Support table statistics for Parquet

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/729#discussion_r100680867
  
    --- Diff: exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/statistics/StatisticsMergeBatch.java ---
    @@ -0,0 +1,532 @@
    +/**
    + * 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.impl.statistics;
    +
    +import com.google.common.collect.Lists;
    +import com.google.common.collect.Maps;
    +import org.apache.drill.common.expression.LogicalExpression;
    +import org.apache.drill.common.expression.ErrorCollector;
    +import org.apache.drill.common.expression.ErrorCollectorImpl;
    +import org.apache.drill.common.expression.FunctionCallFactory;
    +import org.apache.drill.common.expression.SchemaPath;
    +import org.apache.drill.common.expression.ValueExpressions;
    +import org.apache.drill.common.types.TypeProtos;
    +import org.apache.drill.common.types.TypeProtos.MajorType;
    +import org.apache.drill.exec.exception.OutOfMemoryException;
    +import org.apache.drill.exec.exception.SchemaChangeException;
    +import org.apache.drill.exec.expr.ExpressionTreeMaterializer;
    +import org.apache.drill.exec.expr.TypeHelper;
    +import org.apache.drill.exec.expr.holders.BigIntHolder;
    +import org.apache.drill.exec.expr.holders.NullableFloat8Holder;
    +import org.apache.drill.exec.expr.holders.ObjectHolder;
    +import org.apache.drill.exec.expr.holders.ValueHolder;
    +import org.apache.drill.exec.ops.FragmentContext;
    +import org.apache.drill.exec.physical.config.StatisticsMerge;
    +import org.apache.drill.exec.record.AbstractSingleRecordBatch;
    +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.VectorContainer;
    +import org.apache.drill.exec.record.VectorWrapper;
    +import org.apache.drill.exec.vector.ValueVector;
    +import org.apache.drill.exec.vector.DateVector;
    +import org.apache.drill.exec.vector.BigIntVector;
    +import org.apache.drill.exec.vector.NullableBigIntVector;
    +import org.apache.drill.exec.vector.NullableFloat8Vector;
    +import org.apache.drill.exec.vector.NullableVarBinaryVector;
    +import org.apache.drill.exec.vector.VarCharVector;
    +import org.apache.drill.exec.vector.complex.MapVector;
    +
    +import java.io.ByteArrayInputStream;
    +import java.io.DataInputStream;
    +import java.io.IOException;
    +import java.util.List;
    +import java.util.GregorianCalendar;
    +import java.util.HashMap;
    +import java.util.Map;
    +import java.util.TimeZone;
    +
    +import com.clearspring.analytics.stream.cardinality.HyperLogLog;
    +
    +public class StatisticsMergeBatch extends AbstractSingleRecordBatch<StatisticsMerge> {
    +  private static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(StatisticsMergeBatch.class);
    +  private Map<String, String> functions;
    +  private boolean first = true;
    +  private boolean finished = false;
    +  private int schema = 0;
    +  private int recordCount = 0;
    +  private List<String> keyList = null;
    +  private Map<MaterializedField, ValueVector> dataSrcVecMap = null;
    +  // Map of non-map fields to VV in the incoming schema
    +  private Map<MaterializedField, ValueVector> copySrcVecMap = null;
    +  private Map<String, Map<String, ValueHolder>> aggregationMap = null;
    +  public StatisticsMergeBatch(StatisticsMerge popConfig, RecordBatch incoming,
    +                              FragmentContext context) throws OutOfMemoryException {
    +    super(popConfig, context, incoming);
    +    this.functions = new HashMap<>();
    +    this.aggregationMap = new HashMap<>();
    +
    +    /*for (String key : popConfig.getFunctions()) {
    +      aggregationMap.put(key, new HashMap<String, ValueHolder>());
    +      if (key.equalsIgnoreCase("statcount") || key.equalsIgnoreCase("nonnullstatcount")) {
    +        functions.put(key, "sum");
    +      } else if (key.equalsIgnoreCase("hll")) {
    +        functions.put(key, "hll_merge");
    +      } else if (key.equalsIgnoreCase("sum_width")) {
    +        functions.put(key, "avg_width");
    +      }
    +    }*/
    +    for (String key : popConfig.getFunctions()) {
    +      if (key.equalsIgnoreCase("sum_width")) {
    +        functions.put(key, "avg_width");
    +      } else if (key.equalsIgnoreCase("hll")) {
    +        functions.put(key, "hll_merge");
    +      } else {
    +        functions.put(key, key);
    +      }
    +      aggregationMap.put(functions.get(key), new HashMap<String, ValueHolder>());
    +    }
    +  }
    +
    +  private void createKeyColumn(String name, LogicalExpression expr, Map<MaterializedField, ValueVector> parentMap)
    +      throws SchemaChangeException {
    +    ErrorCollector collector = new ErrorCollectorImpl();
    +
    +    LogicalExpression mle = ExpressionTreeMaterializer.materialize(expr, incoming, collector,
    +        context.getFunctionRegistry());
    +
    +    MaterializedField outputField = MaterializedField.create(name, mle.getMajorType());
    +    ValueVector vector = TypeHelper.getNewVector(outputField, oContext.getAllocator());
    +    container.add(vector);
    +
    +    if (collector.hasErrors()) {
    +      throw new SchemaChangeException("Failure while materializing expression. "
    +          + collector.toErrorString());
    +    }
    +    parentMap.put(outputField, vector);
    +  }
    +
    +  private ValueVector addMapVector(String name, MapVector parent, LogicalExpression expr) throws SchemaChangeException {
    +    ErrorCollector collector = new ErrorCollectorImpl();
    +
    +    LogicalExpression mle = ExpressionTreeMaterializer.materialize(expr, incoming, collector,
    +        context.getFunctionRegistry());
    +
    +    Class<? extends ValueVector> vvc =
    +        TypeHelper.getValueVectorClass(mle.getMajorType().getMinorType(), mle.getMajorType().getMode());
    +    ValueVector vector = parent.addOrGet(name, mle.getMajorType(), vvc);
    +
    +    if (collector.hasErrors()) {
    +      throw new SchemaChangeException("Failure while materializing expression. "
    +          + collector.toErrorString());
    +    }
    +    return vector;
    +  }
    +
    +  /**
    +   * Identify the list of fields within a map which are unpivoted as columns in output
    +   */
    +  private void buildKeyList() {
    +    List<String> lastMapKeyList = null;
    +    for (VectorWrapper<?> vw : incoming) {
    +      if (vw.getField().getType().getMinorType() != TypeProtos.MinorType.MAP) {
    +        continue;
    +      }
    +
    +      keyList = Lists.newArrayList();
    +
    +      for (ValueVector vv : vw.getValueVector()) {
    +        keyList.add(vv.getField().getLastName());
    +      }
    +
    +      if (lastMapKeyList == null) {
    +        lastMapKeyList = keyList;
    +      } else {
    +        if (keyList.size() != lastMapKeyList.size() || !lastMapKeyList.containsAll(keyList)) {
    +          throw new UnsupportedOperationException("Maps have different fields");
    +        }
    +      }
    +    }
    +  }
    +
    +  private void buildOutputContainer() throws SchemaChangeException {
    +    dataSrcVecMap = Maps.newHashMap();
    +    copySrcVecMap = Maps.newHashMap();
    +    MajorType mt = null;
    +
    +    ErrorCollector collector = new ErrorCollectorImpl();
    +    GregorianCalendar calendar = new GregorianCalendar(TimeZone.getTimeZone("UTC"));
    +
    +    calendar.setTimeInMillis(System.currentTimeMillis());
    +    createKeyColumn("schema", ValueExpressions.getBigInt(schema++), copySrcVecMap);
    +    createKeyColumn("computed", ValueExpressions.getDate(calendar), copySrcVecMap);
    +
    +    for (VectorWrapper<?> vw : incoming) {
    +      addVectorToOutgoingContainer(vw.getField().getLastName(), vw, collector);
    +      /*MaterializedField ds = vw.getField();
    +      String field = vw.getField().getLastName();
    +      // Input map vector
    +      MapVector mapVector = (MapVector) vw.getValueVector();
    +      assert mapVector.getPrimitiveVectors().size() > 0;
    +      // Proceed to create output map vector with same name e.g. statcount etc.
    +      mt = mapVector.getField().getType();
    +      MaterializedField mf = MaterializedField.create(functions.get(field), mt);
    +      assert !dataSrcVecMap.containsKey(mf);
    +      ValueVector vector = TypeHelper.getNewVector(mf, oContext.getAllocator());
    +      container.add(vector);
    +      MapVector outputMapVector = (MapVector) vector;
    +
    +      for (ValueVector vv : mapVector) {
    +        String fieldName = vv.getField().getLastName();
    +        if (!keyList.contains(fieldName)) {
    +          throw new UnsupportedOperationException("Unpivot data vector " +
    +              ds + " contains key " + fieldName + " not contained in key source!");
    +        }
    +        if (vv.getField().getType().getMinorType() == TypeProtos.MinorType.MAP) {
    +          throw new UnsupportedOperationException("Unpivot of nested map is not supported!");
    +        }
    +        if (field.equals("column")) {
    +          outputMapVector.addOrGet(fieldName, vv.getField().getType(), vv.getClass());
    +        } else {
    +          List<LogicalExpression> args = Lists.newArrayList();
    +          //TODO: Something else to access value of col such as emp_id?
    +          args.add(SchemaPath.getSimplePath(vv.getField().getPath()));
    +          //TODO: Put in the mapVector
    +          LogicalExpression call = FunctionCallFactory.createExpression(functions.get(field), args);
    +          //TODO: Is this sufficient to add to new Map?
    +          ValueVector vector1 = addMapVector(fieldName, outputMapVector, call);
    +          if (collector.hasErrors()) {
    +            throw new SchemaChangeException("Failure while materializing expression. "
    +                + collector.toErrorString());
    +          }
    +        }
    +      }
    +      dataSrcVecMap.put(ds, outputMapVector);*/
    +    }
    +    //Now create NDV in the outgoing container which was not avaliable in the incoming
    +    for (VectorWrapper<?> vw : incoming) {
    +      if (vw.getField().getLastName().equalsIgnoreCase("sum_width")) {//NullableFloat8 type
    +        addVectorToOutgoingContainer("ndv", vw, collector);
    +        break;
    +      }
    +    }
    +    container.setRecordCount(0);
    +    recordCount = 0;
    +    container.buildSchema(incoming.getSchema().getSelectionVectorMode());
    +  }
    +
    +  private void addVectorToOutgoingContainer(String field, VectorWrapper vw, ErrorCollector collector)
    +     throws SchemaChangeException {
    +    // Input map vector
    +    MapVector mapVector = (MapVector) vw.getValueVector();
    +    MaterializedField mf;
    +    assert mapVector.getPrimitiveVectors().size() > 0;
    +    // Proceed to create output map vector with same name e.g. statcount etc.
    +    MajorType mt = mapVector.getField().getType();
    +    if (functions.get(field) != null) {
    +      mf = MaterializedField.create(functions.get(field), mt);
    +    } else {
    +      mf = MaterializedField.create(field, mt);
    +    }
    +    assert !dataSrcVecMap.containsKey(mf);
    +    ValueVector vector = TypeHelper.getNewVector(mf, oContext.getAllocator());
    +    container.add(vector);
    +    MapVector outputMapVector = (MapVector) vector;
    +
    +    for (ValueVector vv : mapVector) {
    +      String fieldName = vv.getField().getLastName();
    +      if (!keyList.contains(fieldName)) {
    +        throw new UnsupportedOperationException("Unpivot data vector " +
    --- End diff --
    
    `IllegalStateException` Actually, very confused about why the code even gets into this state: didn't we already check the names earlier? Isn't the code itself deciding to add the name? So, one part of the code is catching bugs in another part?
    
    Maybe use `assert` or `Preconditions` if we are just checking for bugs?


---
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 #729: Drill 1328: Support table statistics for Parquet

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

    https://github.com/apache/drill/pull/729#discussion_r103612078
  
    --- Diff: exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/statistics/StatisticsAggBatch.java ---
    @@ -0,0 +1,256 @@
    +/**
    + * 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.impl.statistics;
    +
    +import com.google.common.collect.Lists;
    +import com.sun.codemodel.JExpr;
    +import org.apache.drill.common.expression.ErrorCollector;
    +import org.apache.drill.common.expression.ErrorCollectorImpl;
    +import org.apache.drill.common.expression.FunctionCallFactory;
    +import org.apache.drill.common.expression.LogicalExpression;
    +import org.apache.drill.common.expression.SchemaPath;
    +import org.apache.drill.common.expression.ValueExpressions;
    +import org.apache.drill.exec.exception.ClassTransformationException;
    +import org.apache.drill.exec.exception.OutOfMemoryException;
    +import org.apache.drill.exec.exception.SchemaChangeException;
    +import org.apache.drill.exec.expr.ClassGenerator;
    +import org.apache.drill.exec.expr.CodeGenerator;
    +import org.apache.drill.exec.expr.ExpressionTreeMaterializer;
    +import org.apache.drill.exec.expr.TypeHelper;
    +import org.apache.drill.exec.expr.ValueVectorWriteExpression;
    +import org.apache.drill.exec.ops.FragmentContext;
    +import org.apache.drill.exec.physical.config.StatisticsAggregate;
    +import org.apache.drill.exec.physical.impl.aggregate.StreamingAggBatch;
    +import org.apache.drill.exec.physical.impl.aggregate.StreamingAggTemplate;
    +import org.apache.drill.exec.physical.impl.aggregate.StreamingAggregator;
    +import org.apache.drill.exec.planner.physical.StatsAggPrel.OperatorPhase;
    +import org.apache.drill.exec.record.BatchSchema.SelectionVectorMode;
    +import org.apache.drill.exec.record.MaterializedField;
    +import org.apache.drill.exec.record.RecordBatch;
    +import org.apache.drill.exec.record.TypedFieldId;
    +import org.apache.drill.exec.store.ImplicitColumnExplorer;
    +import org.apache.drill.exec.vector.ValueVector;
    +import org.apache.drill.exec.vector.complex.FieldIdUtil;
    +import org.apache.drill.exec.vector.complex.MapVector;
    +
    +import java.io.IOException;
    +import java.util.GregorianCalendar;
    +import java.util.List;
    +import java.util.TimeZone;
    +
    +/**
    + * TODO: This needs cleanup. Currently the key values are constants and we compare the constants
    + * for every record. Seems unnecessary.
    + *
    + * Example input and output:
    + * Schema of incoming batch: region_id (VARCHAR), sales_city (VARCHAR), cnt (BIGINT)
    + * Schema of output:
    + *    "schema" : BIGINT - Schema number. For each schema change this number is incremented.
    + *    "computed" : BIGINT - What time is it computed?
    + *    "columns"       : MAP - Column names
    + *       "region_id"  : VARCHAR
    + *       "sales_city" : VARCHAR
    + *       "cnt"        : VARCHAR
    + *    "statscount" : MAP
    --- End diff --
    
    Explained here and in `AnalyzePrule` where all the functions are defined.


---
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 #729: Drill 1328: Support table statistics for Parquet

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/729#discussion_r100677197
  
    --- Diff: exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/statistics/StatisticsAggBatch.java ---
    @@ -0,0 +1,256 @@
    +/**
    + * 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.impl.statistics;
    +
    +import com.google.common.collect.Lists;
    +import com.sun.codemodel.JExpr;
    +import org.apache.drill.common.expression.ErrorCollector;
    +import org.apache.drill.common.expression.ErrorCollectorImpl;
    +import org.apache.drill.common.expression.FunctionCallFactory;
    +import org.apache.drill.common.expression.LogicalExpression;
    +import org.apache.drill.common.expression.SchemaPath;
    +import org.apache.drill.common.expression.ValueExpressions;
    +import org.apache.drill.exec.exception.ClassTransformationException;
    +import org.apache.drill.exec.exception.OutOfMemoryException;
    +import org.apache.drill.exec.exception.SchemaChangeException;
    +import org.apache.drill.exec.expr.ClassGenerator;
    +import org.apache.drill.exec.expr.CodeGenerator;
    +import org.apache.drill.exec.expr.ExpressionTreeMaterializer;
    +import org.apache.drill.exec.expr.TypeHelper;
    +import org.apache.drill.exec.expr.ValueVectorWriteExpression;
    +import org.apache.drill.exec.ops.FragmentContext;
    +import org.apache.drill.exec.physical.config.StatisticsAggregate;
    +import org.apache.drill.exec.physical.impl.aggregate.StreamingAggBatch;
    +import org.apache.drill.exec.physical.impl.aggregate.StreamingAggTemplate;
    +import org.apache.drill.exec.physical.impl.aggregate.StreamingAggregator;
    +import org.apache.drill.exec.planner.physical.StatsAggPrel.OperatorPhase;
    +import org.apache.drill.exec.record.BatchSchema.SelectionVectorMode;
    +import org.apache.drill.exec.record.MaterializedField;
    +import org.apache.drill.exec.record.RecordBatch;
    +import org.apache.drill.exec.record.TypedFieldId;
    +import org.apache.drill.exec.store.ImplicitColumnExplorer;
    +import org.apache.drill.exec.vector.ValueVector;
    +import org.apache.drill.exec.vector.complex.FieldIdUtil;
    +import org.apache.drill.exec.vector.complex.MapVector;
    +
    +import java.io.IOException;
    +import java.util.GregorianCalendar;
    +import java.util.List;
    +import java.util.TimeZone;
    +
    +/**
    + * TODO: This needs cleanup. Currently the key values are constants and we compare the constants
    + * for every record. Seems unnecessary.
    + *
    + * Example input and output:
    + * Schema of incoming batch: region_id (VARCHAR), sales_city (VARCHAR), cnt (BIGINT)
    + * Schema of output:
    + *    "schema" : BIGINT - Schema number. For each schema change this number is incremented.
    + *    "computed" : BIGINT - What time is it computed?
    + *    "columns"       : MAP - Column names
    + *       "region_id"  : VARCHAR
    + *       "sales_city" : VARCHAR
    + *       "cnt"        : VARCHAR
    + *    "statscount" : MAP
    + *       "region_id"  : BIGINT - statscount(region_id) - aggregation over all values of region_id
    + *                      in incoming batch
    + *       "sales_city" : BIGINT - statscount(sales_city)
    + *       "cnt"        : BIGINT - statscount(cnt)
    + *    "nonnullstatcount" : MAP
    + *       "region_id"  : BIGINT - nonnullstatcount(region_id)
    + *       "sales_city" : BIGINT - nonnullstatcount(sales_city)
    + *       "cnt"        : BIGINT - nonnullstatcount(cnt)
    + *   .... another map for next stats function ....
    + */
    +public class StatisticsAggBatch extends StreamingAggBatch {
    +  private List<String> functions;
    +  private int schema = 0;
    +
    +  public StatisticsAggBatch(StatisticsAggregate popConfig, RecordBatch incoming,
    +      FragmentContext context) throws OutOfMemoryException {
    +    super(popConfig, incoming, context);
    +    this.functions = popConfig.getFunctions();
    +  }
    +
    +  private void createKeyColumn(String name, LogicalExpression expr, List<LogicalExpression> keyExprs,
    +      List<TypedFieldId> keyOutputIds) throws SchemaChangeException {
    +    ErrorCollector collector = new ErrorCollectorImpl();
    +
    +    LogicalExpression mle = ExpressionTreeMaterializer.materialize(expr, incoming, collector,
    +        context.getFunctionRegistry());
    +
    +    MaterializedField outputField = MaterializedField.create(name, mle.getMajorType());
    +    ValueVector vector = TypeHelper.getNewVector(outputField, oContext.getAllocator());
    +
    +    keyExprs.add(mle);
    +    keyOutputIds.add(container.add(vector));
    +
    +    if (collector.hasErrors()) {
    +      throw new SchemaChangeException("Failure while materializing expression. "
    +          + collector.toErrorString());
    +    }
    +  }
    +
    +  private void createNestedKeyColumn(MapVector parent, String name, LogicalExpression expr,
    +      List<LogicalExpression> keyExprs, List<TypedFieldId> keyOutputIds)
    +          throws SchemaChangeException {
    +    ErrorCollector collector = new ErrorCollectorImpl();
    +
    +    LogicalExpression mle = ExpressionTreeMaterializer.materialize(expr, incoming, collector,
    +        context.getFunctionRegistry());
    +
    +    Class<? extends ValueVector> vvc =
    +        TypeHelper.getValueVectorClass(mle.getMajorType().getMinorType(),
    +            mle.getMajorType().getMode());
    +
    +    ValueVector vv = parent.addOrGet(name, mle.getMajorType(), vvc);
    +
    +    TypedFieldId pfid = container.getValueVectorId(SchemaPath.getSimplePath(parent.getField().getPath()));
    +    assert pfid.getFieldIds().length == 1;
    +    TypedFieldId.Builder builder = TypedFieldId.newBuilder();
    +    builder.addId(pfid.getFieldIds()[0]);
    +    TypedFieldId id =
    +        FieldIdUtil.getFieldIdIfMatches(parent, builder, true,
    +            SchemaPath.getSimplePath(vv.getField().getPath()).getRootSegment());
    +
    +    keyExprs.add(mle);
    +    keyOutputIds.add(id);
    +
    +    if (collector.hasErrors()) {
    --- End diff --
    
    Same comment as above.


---
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 #729: Drill 1328: Support table statistics for Parquet

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

    https://github.com/apache/drill/pull/729#discussion_r103365674
  
    --- Diff: exec/java-exec/src/main/java/org/apache/drill/exec/ExecConstants.java ---
    @@ -390,4 +391,15 @@
     
       String DYNAMIC_UDF_SUPPORT_ENABLED = "exec.udf.enable_dynamic_support";
       BooleanValidator DYNAMIC_UDF_SUPPORT_ENABLED_VALIDATOR = new BooleanValidator(DYNAMIC_UDF_SUPPORT_ENABLED, true, true);
    +
    +  /**
    +   * Option whose value is a long value representing the number of bits required for computing ndv (using HLL)
    +   */
    +  LongValidator NDV_MEMORY_LIMIT = new PositiveLongValidator("exec.statistics.ndv_memory_limit", 30, 20);
    +
    +  /**
    +   * Option whose value represents the current version of the statistics. Decreasing the value will generate
    +   * the older version of statistics
    +   */
    +  LongValidator STATISTICS_VERSION = new NonNegativeLongValidator("exec.statistics.capability_version", 1, 1);
    --- End diff --
    
    Say in the next version(v2), we add histograms. Computing stats is expensive so users might prefer to remain on the present version(v1) maybe because their queries do not involve too many inequalities. Always generating the latest version of the stats will force the users to compute the latest and greatest stats without needing them. On the other hand, providing individual control of which statistic to compute moves too much burden onto the user to figure out exactly which statistics would help their use-cases.


---
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 #729: Drill 1328: Support table statistics for Parquet

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/729#discussion_r100680474
  
    --- Diff: exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/statistics/StatisticsMergeBatch.java ---
    @@ -0,0 +1,532 @@
    +/**
    + * 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.impl.statistics;
    +
    +import com.google.common.collect.Lists;
    +import com.google.common.collect.Maps;
    +import org.apache.drill.common.expression.LogicalExpression;
    +import org.apache.drill.common.expression.ErrorCollector;
    +import org.apache.drill.common.expression.ErrorCollectorImpl;
    +import org.apache.drill.common.expression.FunctionCallFactory;
    +import org.apache.drill.common.expression.SchemaPath;
    +import org.apache.drill.common.expression.ValueExpressions;
    +import org.apache.drill.common.types.TypeProtos;
    +import org.apache.drill.common.types.TypeProtos.MajorType;
    +import org.apache.drill.exec.exception.OutOfMemoryException;
    +import org.apache.drill.exec.exception.SchemaChangeException;
    +import org.apache.drill.exec.expr.ExpressionTreeMaterializer;
    +import org.apache.drill.exec.expr.TypeHelper;
    +import org.apache.drill.exec.expr.holders.BigIntHolder;
    +import org.apache.drill.exec.expr.holders.NullableFloat8Holder;
    +import org.apache.drill.exec.expr.holders.ObjectHolder;
    +import org.apache.drill.exec.expr.holders.ValueHolder;
    +import org.apache.drill.exec.ops.FragmentContext;
    +import org.apache.drill.exec.physical.config.StatisticsMerge;
    +import org.apache.drill.exec.record.AbstractSingleRecordBatch;
    +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.VectorContainer;
    +import org.apache.drill.exec.record.VectorWrapper;
    +import org.apache.drill.exec.vector.ValueVector;
    +import org.apache.drill.exec.vector.DateVector;
    +import org.apache.drill.exec.vector.BigIntVector;
    +import org.apache.drill.exec.vector.NullableBigIntVector;
    +import org.apache.drill.exec.vector.NullableFloat8Vector;
    +import org.apache.drill.exec.vector.NullableVarBinaryVector;
    +import org.apache.drill.exec.vector.VarCharVector;
    +import org.apache.drill.exec.vector.complex.MapVector;
    +
    +import java.io.ByteArrayInputStream;
    +import java.io.DataInputStream;
    +import java.io.IOException;
    +import java.util.List;
    +import java.util.GregorianCalendar;
    +import java.util.HashMap;
    +import java.util.Map;
    +import java.util.TimeZone;
    +
    +import com.clearspring.analytics.stream.cardinality.HyperLogLog;
    +
    +public class StatisticsMergeBatch extends AbstractSingleRecordBatch<StatisticsMerge> {
    +  private static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(StatisticsMergeBatch.class);
    +  private Map<String, String> functions;
    +  private boolean first = true;
    +  private boolean finished = false;
    +  private int schema = 0;
    +  private int recordCount = 0;
    +  private List<String> keyList = null;
    +  private Map<MaterializedField, ValueVector> dataSrcVecMap = null;
    +  // Map of non-map fields to VV in the incoming schema
    +  private Map<MaterializedField, ValueVector> copySrcVecMap = null;
    +  private Map<String, Map<String, ValueHolder>> aggregationMap = null;
    +  public StatisticsMergeBatch(StatisticsMerge popConfig, RecordBatch incoming,
    +                              FragmentContext context) throws OutOfMemoryException {
    +    super(popConfig, context, incoming);
    +    this.functions = new HashMap<>();
    +    this.aggregationMap = new HashMap<>();
    +
    +    /*for (String key : popConfig.getFunctions()) {
    +      aggregationMap.put(key, new HashMap<String, ValueHolder>());
    +      if (key.equalsIgnoreCase("statcount") || key.equalsIgnoreCase("nonnullstatcount")) {
    +        functions.put(key, "sum");
    +      } else if (key.equalsIgnoreCase("hll")) {
    +        functions.put(key, "hll_merge");
    +      } else if (key.equalsIgnoreCase("sum_width")) {
    +        functions.put(key, "avg_width");
    +      }
    +    }*/
    +    for (String key : popConfig.getFunctions()) {
    +      if (key.equalsIgnoreCase("sum_width")) {
    +        functions.put(key, "avg_width");
    +      } else if (key.equalsIgnoreCase("hll")) {
    +        functions.put(key, "hll_merge");
    +      } else {
    +        functions.put(key, key);
    +      }
    +      aggregationMap.put(functions.get(key), new HashMap<String, ValueHolder>());
    +    }
    +  }
    +
    +  private void createKeyColumn(String name, LogicalExpression expr, Map<MaterializedField, ValueVector> parentMap)
    +      throws SchemaChangeException {
    +    ErrorCollector collector = new ErrorCollectorImpl();
    +
    +    LogicalExpression mle = ExpressionTreeMaterializer.materialize(expr, incoming, collector,
    +        context.getFunctionRegistry());
    +
    +    MaterializedField outputField = MaterializedField.create(name, mle.getMajorType());
    +    ValueVector vector = TypeHelper.getNewVector(outputField, oContext.getAllocator());
    +    container.add(vector);
    +
    +    if (collector.hasErrors()) {
    +      throw new SchemaChangeException("Failure while materializing expression. "
    +          + collector.toErrorString());
    +    }
    +    parentMap.put(outputField, vector);
    +  }
    +
    +  private ValueVector addMapVector(String name, MapVector parent, LogicalExpression expr) throws SchemaChangeException {
    +    ErrorCollector collector = new ErrorCollectorImpl();
    +
    +    LogicalExpression mle = ExpressionTreeMaterializer.materialize(expr, incoming, collector,
    +        context.getFunctionRegistry());
    +
    +    Class<? extends ValueVector> vvc =
    +        TypeHelper.getValueVectorClass(mle.getMajorType().getMinorType(), mle.getMajorType().getMode());
    +    ValueVector vector = parent.addOrGet(name, mle.getMajorType(), vvc);
    +
    +    if (collector.hasErrors()) {
    +      throw new SchemaChangeException("Failure while materializing expression. "
    +          + collector.toErrorString());
    +    }
    +    return vector;
    +  }
    +
    +  /**
    +   * Identify the list of fields within a map which are unpivoted as columns in output
    +   */
    +  private void buildKeyList() {
    +    List<String> lastMapKeyList = null;
    +    for (VectorWrapper<?> vw : incoming) {
    +      if (vw.getField().getType().getMinorType() != TypeProtos.MinorType.MAP) {
    +        continue;
    +      }
    +
    +      keyList = Lists.newArrayList();
    +
    +      for (ValueVector vv : vw.getValueVector()) {
    +        keyList.add(vv.getField().getLastName());
    +      }
    +
    +      if (lastMapKeyList == null) {
    +        lastMapKeyList = keyList;
    +      } else {
    +        if (keyList.size() != lastMapKeyList.size() || !lastMapKeyList.containsAll(keyList)) {
    +          throw new UnsupportedOperationException("Maps have different fields");
    --- End diff --
    
    First, should throw `IllegalStateException`: something is wrong with the code rather than the user asking for an operation we don't support.
    
    Second, perhaps a bit of a better explanation? Which maps? Fields different than what?


---
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 #729: Drill 1328: Support table statistics for Parquet

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

    https://github.com/apache/drill/pull/729#discussion_r103617262
  
    --- Diff: exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/statistics/AvgWidthMergedStatistic.java ---
    @@ -0,0 +1,135 @@
    +/*
    + * 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.impl.statistics;
    +
    +import org.apache.drill.common.types.TypeProtos;
    +import org.apache.drill.exec.expr.holders.NullableFloat8Holder;
    +import org.apache.drill.exec.expr.holders.ValueHolder;
    +import org.apache.drill.exec.vector.NullableFloat8Vector;
    +import org.apache.drill.exec.vector.ValueVector;
    +import org.apache.drill.exec.vector.complex.MapVector;
    +
    +import java.util.HashMap;
    +import java.util.List;
    +import java.util.Map;
    +
    +public class AvgWidthMergedStatistic extends AbstractMergedStatistic {
    +
    +  private String name;
    +  private String inputName;
    +  private boolean configureComplete = false;
    +  private boolean mergeComplete = false;
    +  private Map<String, ValueHolder> sumHolder;
    +  MergedStatistic types, nonNullStatCounts, statCounts;
    +
    +  public AvgWidthMergedStatistic (String name, String inputName) {
    +    this.name = name;
    +    this.inputName = inputName;
    +    this.sumHolder = new HashMap<>();
    +    types = nonNullStatCounts = statCounts = null;
    +  }
    +
    +  @Override
    +  public String getName() {
    +    return name;
    +  }
    +
    +  @Override
    +  public String getInput() {
    +    return inputName;
    +  }
    +
    +  @Override
    +  public void merge(ValueVector input) {
    +    // Check the input is a Map Vector
    +    assert (input.getField().getType().getMinorType() == TypeProtos.MinorType.MAP);
    +    MapVector inputMap = (MapVector) input;
    +    for (ValueVector vv : inputMap) {
    +      String colName = vv.getField().getLastName();
    +      NullableFloat8Holder colSumHolder;
    +      if (sumHolder.get(colName) != null) {
    +        colSumHolder = (NullableFloat8Holder) sumHolder.get(colName);
    +      } else {
    +        colSumHolder = new NullableFloat8Holder();
    +        sumHolder.put(colName, colSumHolder);
    +      }
    +      Object val = vv.getAccessor().getObject(0);
    +      if (val != null) {
    +        colSumHolder.value += (double) val;
    +        colSumHolder.isSet = 1;
    +      }
    +    }
    +  }
    +
    +  @Override
    +  public Object getStat(String colName) {
    +      if (mergeComplete != true) {
    +        throw new IllegalStateException(
    +            String.format("Statistic `%s` has not completed merging statistics", name));
    +      }
    +      NullableFloat8Holder colSumHolder = (NullableFloat8Holder) sumHolder.get(colName);
    +      return (long) (colSumHolder.value/ getRowCount(colName));
    +    }
    +
    +  @Override
    +  public void setOutput(ValueVector output) {
    +    // Check the input is a Map Vector
    +    assert (output.getField().getType().getMinorType() == TypeProtos.MinorType.MAP);
    +    // Dependencies have been configured correctly
    +    assert (configureComplete == true);
    +    MapVector outputMap = (MapVector) output;
    +
    +    for (ValueVector outMapCol : outputMap) {
    +      String colName = outMapCol.getField().getLastName();
    +      NullableFloat8Holder colSumHolder = (NullableFloat8Holder) sumHolder.get(colName);
    +      NullableFloat8Vector vv = (NullableFloat8Vector) outMapCol;
    +      vv.allocateNewSafe();
    +      vv.getMutator().setSafe(0, (colSumHolder.value / getRowCount(colName)));
    +    }
    +    mergeComplete = true;
    +  }
    +
    +  @Override
    +  public void configure(Object configurations) {
    +    List<MergedStatistic> statistics = (List<MergedStatistic>) configurations;
    +    for (MergedStatistic statistic : statistics) {
    +      if (statistic.getName().equals("type")) {
    +        types = statistic;
    +      } else if (statistic.getName().equals("statcount")) {
    +        statCounts = statistic;
    +      } else if (statistic.getName().equals("nonnullstatcount")) {
    +        nonNullStatCounts = statistic;
    +      }
    +    }
    +    assert (types != null && statCounts != null && nonNullStatCounts != null);
    +    configureComplete = true;
    +  }
    +
    +  private long getRowCount(String colName) {
    +    int type = (int) types.getStat(colName);
    +    // If variable type - then use the nonNullCount. Otherwise, use the Count,
    +    // since even NULL values take up the same space.
    +    if (type == TypeProtos.MinorType.VAR16CHAR.getNumber()
    +        || type == TypeProtos.MinorType.VARCHAR.getNumber()
    +        || type == TypeProtos.MinorType.VARBINARY.getNumber()) {
    +      return (long) nonNullStatCounts.getStat(colName);
    +    } else {
    +      return (long) statCounts.getStat(colName);
    --- End diff --
    
    Since the `MapVector` itself does not guarantee any order for the `ValueVectors` inside, we would have to reconstruct the index every time in `MergedStatistic.merge()` which would be expensive. For `MergedStatistic.output()` we only do it once per column. Also, all the MergedStatistic operations would be done on one RecordBatch, so it should have minimal performance impact, if any. Keeping the code simple seems to be the better choice 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 #729: Drill 1328: Support table statistics for Parquet

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

    https://github.com/apache/drill/pull/729#discussion_r103367008
  
    --- Diff: exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/statistics/AbstractMergedStatistic.java ---
    @@ -0,0 +1,52 @@
    +/*
    + * 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.impl.statistics;
    +
    +import org.apache.drill.exec.vector.ValueVector;
    +
    +public abstract class AbstractMergedStatistic extends Statistic implements MergedStatistic {
    +  @Override
    +  public String getName() {
    +    throw new UnsupportedOperationException("getName() not implemented");
    +  }
    --- 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 #729: Drill 1328: Support table statistics for Parquet

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

    https://github.com/apache/drill/pull/729#discussion_r102324705
  
    --- Diff: exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/statistics/StatisticsMergeBatch.java ---
    @@ -0,0 +1,532 @@
    +/**
    + * 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.impl.statistics;
    +
    +import com.google.common.collect.Lists;
    +import com.google.common.collect.Maps;
    +import org.apache.drill.common.expression.LogicalExpression;
    +import org.apache.drill.common.expression.ErrorCollector;
    +import org.apache.drill.common.expression.ErrorCollectorImpl;
    +import org.apache.drill.common.expression.FunctionCallFactory;
    +import org.apache.drill.common.expression.SchemaPath;
    +import org.apache.drill.common.expression.ValueExpressions;
    +import org.apache.drill.common.types.TypeProtos;
    +import org.apache.drill.common.types.TypeProtos.MajorType;
    +import org.apache.drill.exec.exception.OutOfMemoryException;
    +import org.apache.drill.exec.exception.SchemaChangeException;
    +import org.apache.drill.exec.expr.ExpressionTreeMaterializer;
    +import org.apache.drill.exec.expr.TypeHelper;
    +import org.apache.drill.exec.expr.holders.BigIntHolder;
    +import org.apache.drill.exec.expr.holders.NullableFloat8Holder;
    +import org.apache.drill.exec.expr.holders.ObjectHolder;
    +import org.apache.drill.exec.expr.holders.ValueHolder;
    +import org.apache.drill.exec.ops.FragmentContext;
    +import org.apache.drill.exec.physical.config.StatisticsMerge;
    +import org.apache.drill.exec.record.AbstractSingleRecordBatch;
    +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.VectorContainer;
    +import org.apache.drill.exec.record.VectorWrapper;
    +import org.apache.drill.exec.vector.ValueVector;
    +import org.apache.drill.exec.vector.DateVector;
    +import org.apache.drill.exec.vector.BigIntVector;
    +import org.apache.drill.exec.vector.NullableBigIntVector;
    +import org.apache.drill.exec.vector.NullableFloat8Vector;
    +import org.apache.drill.exec.vector.NullableVarBinaryVector;
    +import org.apache.drill.exec.vector.VarCharVector;
    +import org.apache.drill.exec.vector.complex.MapVector;
    +
    +import java.io.ByteArrayInputStream;
    +import java.io.DataInputStream;
    +import java.io.IOException;
    +import java.util.List;
    +import java.util.GregorianCalendar;
    +import java.util.HashMap;
    +import java.util.Map;
    +import java.util.TimeZone;
    +
    +import com.clearspring.analytics.stream.cardinality.HyperLogLog;
    +
    +public class StatisticsMergeBatch extends AbstractSingleRecordBatch<StatisticsMerge> {
    +  private static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(StatisticsMergeBatch.class);
    +  private Map<String, String> functions;
    +  private boolean first = true;
    +  private boolean finished = false;
    +  private int schema = 0;
    +  private int recordCount = 0;
    +  private List<String> keyList = null;
    +  private Map<MaterializedField, ValueVector> dataSrcVecMap = null;
    +  // Map of non-map fields to VV in the incoming schema
    +  private Map<MaterializedField, ValueVector> copySrcVecMap = null;
    +  private Map<String, Map<String, ValueHolder>> aggregationMap = null;
    +  public StatisticsMergeBatch(StatisticsMerge popConfig, RecordBatch incoming,
    +                              FragmentContext context) throws OutOfMemoryException {
    +    super(popConfig, context, incoming);
    +    this.functions = new HashMap<>();
    +    this.aggregationMap = new HashMap<>();
    +
    +    /*for (String key : popConfig.getFunctions()) {
    +      aggregationMap.put(key, new HashMap<String, ValueHolder>());
    +      if (key.equalsIgnoreCase("statcount") || key.equalsIgnoreCase("nonnullstatcount")) {
    +        functions.put(key, "sum");
    +      } else if (key.equalsIgnoreCase("hll")) {
    +        functions.put(key, "hll_merge");
    +      } else if (key.equalsIgnoreCase("sum_width")) {
    +        functions.put(key, "avg_width");
    +      }
    +    }*/
    +    for (String key : popConfig.getFunctions()) {
    +      if (key.equalsIgnoreCase("sum_width")) {
    +        functions.put(key, "avg_width");
    +      } else if (key.equalsIgnoreCase("hll")) {
    +        functions.put(key, "hll_merge");
    +      } else {
    +        functions.put(key, key);
    +      }
    +      aggregationMap.put(functions.get(key), new HashMap<String, ValueHolder>());
    +    }
    +  }
    +
    +  private void createKeyColumn(String name, LogicalExpression expr, Map<MaterializedField, ValueVector> parentMap)
    +      throws SchemaChangeException {
    +    ErrorCollector collector = new ErrorCollectorImpl();
    +
    +    LogicalExpression mle = ExpressionTreeMaterializer.materialize(expr, incoming, collector,
    +        context.getFunctionRegistry());
    +
    +    MaterializedField outputField = MaterializedField.create(name, mle.getMajorType());
    +    ValueVector vector = TypeHelper.getNewVector(outputField, oContext.getAllocator());
    +    container.add(vector);
    +
    +    if (collector.hasErrors()) {
    +      throw new SchemaChangeException("Failure while materializing expression. "
    +          + collector.toErrorString());
    +    }
    +    parentMap.put(outputField, vector);
    +  }
    +
    +  private ValueVector addMapVector(String name, MapVector parent, LogicalExpression expr) throws SchemaChangeException {
    +    ErrorCollector collector = new ErrorCollectorImpl();
    +
    +    LogicalExpression mle = ExpressionTreeMaterializer.materialize(expr, incoming, collector,
    +        context.getFunctionRegistry());
    +
    +    Class<? extends ValueVector> vvc =
    +        TypeHelper.getValueVectorClass(mle.getMajorType().getMinorType(), mle.getMajorType().getMode());
    +    ValueVector vector = parent.addOrGet(name, mle.getMajorType(), vvc);
    +
    +    if (collector.hasErrors()) {
    +      throw new SchemaChangeException("Failure while materializing expression. "
    +          + collector.toErrorString());
    +    }
    +    return vector;
    +  }
    +
    +  /**
    +   * Identify the list of fields within a map which are unpivoted as columns in output
    +   */
    +  private void buildKeyList() {
    +    List<String> lastMapKeyList = null;
    +    for (VectorWrapper<?> vw : incoming) {
    +      if (vw.getField().getType().getMinorType() != TypeProtos.MinorType.MAP) {
    +        continue;
    +      }
    +
    +      keyList = Lists.newArrayList();
    +
    +      for (ValueVector vv : vw.getValueVector()) {
    +        keyList.add(vv.getField().getLastName());
    +      }
    +
    +      if (lastMapKeyList == null) {
    +        lastMapKeyList = keyList;
    +      } else {
    +        if (keyList.size() != lastMapKeyList.size() || !lastMapKeyList.containsAll(keyList)) {
    +          throw new UnsupportedOperationException("Maps have different fields");
    +        }
    +      }
    +    }
    +  }
    +
    +  private void buildOutputContainer() throws SchemaChangeException {
    +    dataSrcVecMap = Maps.newHashMap();
    +    copySrcVecMap = Maps.newHashMap();
    +    MajorType mt = null;
    +
    +    ErrorCollector collector = new ErrorCollectorImpl();
    --- End diff --
    
    Code refactored. Please take a look.


---
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 #729: Drill 1328: Support table statistics for Parquet

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/729#discussion_r100677629
  
    --- Diff: exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/statistics/StatisticsAggBatch.java ---
    @@ -0,0 +1,256 @@
    +/**
    + * 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.impl.statistics;
    +
    +import com.google.common.collect.Lists;
    +import com.sun.codemodel.JExpr;
    +import org.apache.drill.common.expression.ErrorCollector;
    +import org.apache.drill.common.expression.ErrorCollectorImpl;
    +import org.apache.drill.common.expression.FunctionCallFactory;
    +import org.apache.drill.common.expression.LogicalExpression;
    +import org.apache.drill.common.expression.SchemaPath;
    +import org.apache.drill.common.expression.ValueExpressions;
    +import org.apache.drill.exec.exception.ClassTransformationException;
    +import org.apache.drill.exec.exception.OutOfMemoryException;
    +import org.apache.drill.exec.exception.SchemaChangeException;
    +import org.apache.drill.exec.expr.ClassGenerator;
    +import org.apache.drill.exec.expr.CodeGenerator;
    +import org.apache.drill.exec.expr.ExpressionTreeMaterializer;
    +import org.apache.drill.exec.expr.TypeHelper;
    +import org.apache.drill.exec.expr.ValueVectorWriteExpression;
    +import org.apache.drill.exec.ops.FragmentContext;
    +import org.apache.drill.exec.physical.config.StatisticsAggregate;
    +import org.apache.drill.exec.physical.impl.aggregate.StreamingAggBatch;
    +import org.apache.drill.exec.physical.impl.aggregate.StreamingAggTemplate;
    +import org.apache.drill.exec.physical.impl.aggregate.StreamingAggregator;
    +import org.apache.drill.exec.planner.physical.StatsAggPrel.OperatorPhase;
    +import org.apache.drill.exec.record.BatchSchema.SelectionVectorMode;
    +import org.apache.drill.exec.record.MaterializedField;
    +import org.apache.drill.exec.record.RecordBatch;
    +import org.apache.drill.exec.record.TypedFieldId;
    +import org.apache.drill.exec.store.ImplicitColumnExplorer;
    +import org.apache.drill.exec.vector.ValueVector;
    +import org.apache.drill.exec.vector.complex.FieldIdUtil;
    +import org.apache.drill.exec.vector.complex.MapVector;
    +
    +import java.io.IOException;
    +import java.util.GregorianCalendar;
    +import java.util.List;
    +import java.util.TimeZone;
    +
    +/**
    + * TODO: This needs cleanup. Currently the key values are constants and we compare the constants
    + * for every record. Seems unnecessary.
    + *
    + * Example input and output:
    + * Schema of incoming batch: region_id (VARCHAR), sales_city (VARCHAR), cnt (BIGINT)
    + * Schema of output:
    + *    "schema" : BIGINT - Schema number. For each schema change this number is incremented.
    + *    "computed" : BIGINT - What time is it computed?
    + *    "columns"       : MAP - Column names
    + *       "region_id"  : VARCHAR
    + *       "sales_city" : VARCHAR
    + *       "cnt"        : VARCHAR
    + *    "statscount" : MAP
    + *       "region_id"  : BIGINT - statscount(region_id) - aggregation over all values of region_id
    + *                      in incoming batch
    + *       "sales_city" : BIGINT - statscount(sales_city)
    + *       "cnt"        : BIGINT - statscount(cnt)
    + *    "nonnullstatcount" : MAP
    + *       "region_id"  : BIGINT - nonnullstatcount(region_id)
    + *       "sales_city" : BIGINT - nonnullstatcount(sales_city)
    + *       "cnt"        : BIGINT - nonnullstatcount(cnt)
    + *   .... another map for next stats function ....
    + */
    +public class StatisticsAggBatch extends StreamingAggBatch {
    +  private List<String> functions;
    +  private int schema = 0;
    +
    +  public StatisticsAggBatch(StatisticsAggregate popConfig, RecordBatch incoming,
    +      FragmentContext context) throws OutOfMemoryException {
    +    super(popConfig, incoming, context);
    +    this.functions = popConfig.getFunctions();
    +  }
    +
    +  private void createKeyColumn(String name, LogicalExpression expr, List<LogicalExpression> keyExprs,
    +      List<TypedFieldId> keyOutputIds) throws SchemaChangeException {
    +    ErrorCollector collector = new ErrorCollectorImpl();
    +
    +    LogicalExpression mle = ExpressionTreeMaterializer.materialize(expr, incoming, collector,
    +        context.getFunctionRegistry());
    +
    +    MaterializedField outputField = MaterializedField.create(name, mle.getMajorType());
    +    ValueVector vector = TypeHelper.getNewVector(outputField, oContext.getAllocator());
    +
    +    keyExprs.add(mle);
    +    keyOutputIds.add(container.add(vector));
    +
    +    if (collector.hasErrors()) {
    +      throw new SchemaChangeException("Failure while materializing expression. "
    +          + collector.toErrorString());
    +    }
    +  }
    +
    +  private void createNestedKeyColumn(MapVector parent, String name, LogicalExpression expr,
    +      List<LogicalExpression> keyExprs, List<TypedFieldId> keyOutputIds)
    +          throws SchemaChangeException {
    +    ErrorCollector collector = new ErrorCollectorImpl();
    +
    +    LogicalExpression mle = ExpressionTreeMaterializer.materialize(expr, incoming, collector,
    +        context.getFunctionRegistry());
    +
    +    Class<? extends ValueVector> vvc =
    +        TypeHelper.getValueVectorClass(mle.getMajorType().getMinorType(),
    +            mle.getMajorType().getMode());
    +
    +    ValueVector vv = parent.addOrGet(name, mle.getMajorType(), vvc);
    +
    +    TypedFieldId pfid = container.getValueVectorId(SchemaPath.getSimplePath(parent.getField().getPath()));
    +    assert pfid.getFieldIds().length == 1;
    +    TypedFieldId.Builder builder = TypedFieldId.newBuilder();
    +    builder.addId(pfid.getFieldIds()[0]);
    +    TypedFieldId id =
    +        FieldIdUtil.getFieldIdIfMatches(parent, builder, true,
    +            SchemaPath.getSimplePath(vv.getField().getPath()).getRootSegment());
    +
    +    keyExprs.add(mle);
    +    keyOutputIds.add(id);
    +
    +    if (collector.hasErrors()) {
    +      throw new SchemaChangeException("Failure while materializing expression. "
    +          + collector.toErrorString());
    +    }
    +  }
    +
    +  private void addMapVector(String name, MapVector parent, LogicalExpression expr,
    +      List<LogicalExpression> valueExprs) throws SchemaChangeException {
    +    ErrorCollector collector = new ErrorCollectorImpl();
    +
    +    LogicalExpression mle = ExpressionTreeMaterializer.materialize(expr, incoming, collector,
    +        context.getFunctionRegistry());
    +
    +    Class<? extends ValueVector> vvc =
    +        TypeHelper.getValueVectorClass(mle.getMajorType().getMinorType(), mle.getMajorType().getMode());
    +    ValueVector vv = parent.addOrGet(name, mle.getMajorType(), vvc);
    +
    +    TypedFieldId pfid = container.getValueVectorId(SchemaPath.getSimplePath(parent.getField().getPath()));
    +    assert pfid.getFieldIds().length == 1;
    +    TypedFieldId.Builder builder = TypedFieldId.newBuilder();
    +    builder.addId(pfid.getFieldIds()[0]);
    +    TypedFieldId id = FieldIdUtil.getFieldIdIfMatches(parent, builder, true,
    +        SchemaPath.getSimplePath(vv.getField().getPath()).getRootSegment());
    +
    +    valueExprs.add(new ValueVectorWriteExpression(id, mle, true));
    +
    +    if (collector.hasErrors()) {
    +      throw new SchemaChangeException("Failure while materializing expression. "
    +          + collector.toErrorString());
    +    }
    +  }
    +
    +  private StreamingAggregator codegenAggregator(List<LogicalExpression> keyExprs,
    +      List<LogicalExpression> valueExprs, List<TypedFieldId> keyOutputIds)
    +          throws SchemaChangeException, ClassTransformationException, IOException {
    +    ClassGenerator<StreamingAggregator> cg =
    +        CodeGenerator.getRoot(StreamingAggTemplate.TEMPLATE_DEFINITION, context.getFunctionRegistry(),
    +            context.getOptions());
    +
    +    LogicalExpression[] keyExprsArray = new LogicalExpression[keyExprs.size()];
    +    LogicalExpression[] valueExprsArray = new LogicalExpression[valueExprs.size()];
    +    TypedFieldId[] keyOutputIdsArray = new TypedFieldId[keyOutputIds.size()];
    +
    +    keyExprs.toArray(keyExprsArray);
    +    valueExprs.toArray(valueExprsArray);
    +    keyOutputIds.toArray(keyOutputIdsArray);
    +
    +    setupIsSame(cg, keyExprsArray);
    +    setupIsSameApart(cg, keyExprsArray);
    +    addRecordValues(cg, valueExprsArray);
    +    outputRecordKeys(cg, keyOutputIdsArray, keyExprsArray);
    +    outputRecordKeysPrev(cg, keyOutputIdsArray, keyExprsArray);
    +
    +    cg.getBlock("resetValues")._return(JExpr.TRUE);
    +    getIndex(cg);
    +
    +    container.buildSchema(SelectionVectorMode.NONE);
    +    StreamingAggregator agg = context.getImplementationClass(cg);
    +    agg.setup(oContext, incoming, this);
    +    return agg;
    +  }
    +
    +  private boolean isImplicitFileColumn(MaterializedField mf) {
    +    return ImplicitColumnExplorer.initImplicitFileColumns(context.getOptions()).get(mf.getName()) != null;
    +  }
    +
    +  protected StreamingAggregator createAggregatorInternal()
    +      throws SchemaChangeException, ClassTransformationException, IOException {
    +    container.clear();
    +
    +    List<LogicalExpression> keyExprs = Lists.newArrayList();
    +    List<LogicalExpression> valueExprs = Lists.newArrayList();
    +    List<TypedFieldId> keyOutputIds = Lists.newArrayList();
    +    GregorianCalendar calendar = new GregorianCalendar(TimeZone.getTimeZone("UTC"));
    +    calendar.setTimeInMillis(System.currentTimeMillis());
    +
    +    if (this.getPopConfig() instanceof StatisticsAggregate
    +        && (((StatisticsAggregate) this.getPopConfig()).getPhase() == OperatorPhase.PHASE_1of1
    +            || ((StatisticsAggregate) this.getPopConfig()).getPhase() == OperatorPhase.PHASE_2of2)) {
    +      createKeyColumn("schema",
    +          ValueExpressions.getBigInt(schema++),
    --- End diff --
    
    Does this work when parallelized? Suppose I have to files in a directory, or row groups in a file:
    
    ```
    Group 1:
       Schema 1: (INT a, VARCHAR b) for 100K records, then
       Schema 2: (DOUBLE a, VARCHAR b) for another 100K
    
    Group 2:
       Schema 1: (DOUBLE a, VARCHAR b) for 200K, then
       Schema 2: (INT a, VARCHAR b) for another 200K
    ```
    
    In this scenario, each fragment assigns schema numbers, but the numbers don't match up across groups.
    
    Given this, two questions:
    
    1. How are the group numbers used?
    2. How do we reconcile where "schema 1" is a different schema in different groups?


---
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 #729: Drill 1328: Support table statistics for Parquet

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

    https://github.com/apache/drill/pull/729#discussion_r100414466
  
    --- Diff: exec/java-exec/src/main/java/org/apache/drill/exec/planner/cost/DrillRelMdSelectivity.java ---
    @@ -0,0 +1,219 @@
    +/*******************************************************************************
    + * 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.planner.cost;
    +
    +import org.apache.calcite.plan.RelOptUtil;
    +import org.apache.calcite.plan.volcano.RelSubset;
    +import org.apache.calcite.rel.RelNode;
    +import org.apache.calcite.rel.SingleRel;
    +import org.apache.calcite.rel.core.JoinRelType;
    +import org.apache.calcite.rel.core.TableScan;
    +import org.apache.calcite.rel.metadata.ReflectiveRelMetadataProvider;
    +import org.apache.calcite.rel.metadata.RelMdSelectivity;
    +import org.apache.calcite.rel.metadata.RelMdUtil;
    +import org.apache.calcite.rel.metadata.RelMetadataProvider;
    +import org.apache.calcite.rel.metadata.RelMetadataQuery;
    +import org.apache.calcite.rel.type.RelDataType;
    +import org.apache.calcite.rex.RexBuilder;
    +import org.apache.calcite.rex.RexCall;
    +import org.apache.calcite.rex.RexInputRef;
    +import org.apache.calcite.rex.RexNode;
    +import org.apache.calcite.rex.RexUtil;
    +import org.apache.calcite.rex.RexVisitor;
    +import org.apache.calcite.rex.RexVisitorImpl;
    +import org.apache.calcite.sql.SqlKind;
    +import org.apache.calcite.util.BuiltInMethod;
    +import org.apache.calcite.util.Util;
    +import org.apache.drill.exec.planner.common.DrillJoinRelBase;
    +import org.apache.drill.exec.planner.common.DrillRelOptUtil;
    +import org.apache.drill.exec.planner.logical.DrillTable;
    +import org.apache.drill.exec.planner.logical.DrillTranslatableTable;
    +
    +import java.util.ArrayList;
    +import java.util.List;
    +
    +public class DrillRelMdSelectivity extends RelMdSelectivity {
    +  private static final org.slf4j.Logger logger =
    +      org.slf4j.LoggerFactory.getLogger(RelMdSelectivity.class);
    +
    +  private static final DrillRelMdSelectivity INSTANCE =
    +      new DrillRelMdSelectivity();
    +
    +  public static final RelMetadataProvider SOURCE =
    +      ReflectiveRelMetadataProvider.reflectiveSource(
    +          BuiltInMethod.SELECTIVITY.method, INSTANCE);
    +
    +  @Override
    +  public Double getSelectivity(RelNode rel, RexNode predicate) {
    +    if (rel instanceof TableScan) {
    +      return getScanSelectivity((TableScan) rel, predicate);
    +    } else if (rel instanceof DrillJoinRelBase) {
    +      return getJoinSelectivity(((DrillJoinRelBase) rel), predicate);
    +    } else if (rel instanceof SingleRel && !DrillRelOptUtil.guessRows(rel)) {
    +        return RelMetadataQuery.getSelectivity(((SingleRel) rel).getInput(), predicate);
    +    } else if (rel instanceof RelSubset && !DrillRelOptUtil.guessRows(rel)) {
    +      if (((RelSubset) rel).getBest() != null) {
    +        return RelMetadataQuery.getSelectivity(((RelSubset)rel).getBest(), predicate);
    +      } else if (((RelSubset)rel).getOriginal() != null) {
    +        return RelMetadataQuery.getSelectivity(((RelSubset)rel).getOriginal(), predicate);
    +      } else {
    +        return super.getSelectivity(rel, predicate);
    +      }
    +    } else {
    +      return super.getSelectivity(rel, predicate);
    +    }
    +  }
    +
    +  private Double getJoinSelectivity(DrillJoinRelBase rel, RexNode predicate) {
    --- End diff --
    
    I think proposing this as a Calcite change would be more involved and take more time. I will start a Calcite proposal a little while later and after the changes are committed, we can remove the Drill selectivity calculation.


---
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 #729: Drill 1328: Support table statistics for Parquet

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

    https://github.com/apache/drill/pull/729#discussion_r100705997
  
    --- Diff: exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/aggregate/InternalBatch.java ---
    @@ -88,4 +91,7 @@ public void clear() {
         return container.getValueAccessorById(clazz, fieldIds);
       }
     
    +  public FragmentContext getContext() {
    +    return incoming.getContext();
    --- End diff --
    
    Done. I use the incoming to get/store the context - not needed to pass it via the constructor.


---
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 #729: Drill 1328: Support table statistics for Parquet

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/729#discussion_r100680411
  
    --- Diff: exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/statistics/StatisticsMergeBatch.java ---
    @@ -0,0 +1,532 @@
    +/**
    + * 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.impl.statistics;
    +
    +import com.google.common.collect.Lists;
    +import com.google.common.collect.Maps;
    +import org.apache.drill.common.expression.LogicalExpression;
    +import org.apache.drill.common.expression.ErrorCollector;
    +import org.apache.drill.common.expression.ErrorCollectorImpl;
    +import org.apache.drill.common.expression.FunctionCallFactory;
    +import org.apache.drill.common.expression.SchemaPath;
    +import org.apache.drill.common.expression.ValueExpressions;
    +import org.apache.drill.common.types.TypeProtos;
    +import org.apache.drill.common.types.TypeProtos.MajorType;
    +import org.apache.drill.exec.exception.OutOfMemoryException;
    +import org.apache.drill.exec.exception.SchemaChangeException;
    +import org.apache.drill.exec.expr.ExpressionTreeMaterializer;
    +import org.apache.drill.exec.expr.TypeHelper;
    +import org.apache.drill.exec.expr.holders.BigIntHolder;
    +import org.apache.drill.exec.expr.holders.NullableFloat8Holder;
    +import org.apache.drill.exec.expr.holders.ObjectHolder;
    +import org.apache.drill.exec.expr.holders.ValueHolder;
    +import org.apache.drill.exec.ops.FragmentContext;
    +import org.apache.drill.exec.physical.config.StatisticsMerge;
    +import org.apache.drill.exec.record.AbstractSingleRecordBatch;
    +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.VectorContainer;
    +import org.apache.drill.exec.record.VectorWrapper;
    +import org.apache.drill.exec.vector.ValueVector;
    +import org.apache.drill.exec.vector.DateVector;
    +import org.apache.drill.exec.vector.BigIntVector;
    +import org.apache.drill.exec.vector.NullableBigIntVector;
    +import org.apache.drill.exec.vector.NullableFloat8Vector;
    +import org.apache.drill.exec.vector.NullableVarBinaryVector;
    +import org.apache.drill.exec.vector.VarCharVector;
    +import org.apache.drill.exec.vector.complex.MapVector;
    +
    +import java.io.ByteArrayInputStream;
    +import java.io.DataInputStream;
    +import java.io.IOException;
    +import java.util.List;
    +import java.util.GregorianCalendar;
    +import java.util.HashMap;
    +import java.util.Map;
    +import java.util.TimeZone;
    +
    +import com.clearspring.analytics.stream.cardinality.HyperLogLog;
    +
    +public class StatisticsMergeBatch extends AbstractSingleRecordBatch<StatisticsMerge> {
    +  private static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(StatisticsMergeBatch.class);
    +  private Map<String, String> functions;
    +  private boolean first = true;
    +  private boolean finished = false;
    +  private int schema = 0;
    +  private int recordCount = 0;
    +  private List<String> keyList = null;
    +  private Map<MaterializedField, ValueVector> dataSrcVecMap = null;
    +  // Map of non-map fields to VV in the incoming schema
    +  private Map<MaterializedField, ValueVector> copySrcVecMap = null;
    +  private Map<String, Map<String, ValueHolder>> aggregationMap = null;
    +  public StatisticsMergeBatch(StatisticsMerge popConfig, RecordBatch incoming,
    +                              FragmentContext context) throws OutOfMemoryException {
    +    super(popConfig, context, incoming);
    +    this.functions = new HashMap<>();
    +    this.aggregationMap = new HashMap<>();
    +
    +    /*for (String key : popConfig.getFunctions()) {
    +      aggregationMap.put(key, new HashMap<String, ValueHolder>());
    +      if (key.equalsIgnoreCase("statcount") || key.equalsIgnoreCase("nonnullstatcount")) {
    +        functions.put(key, "sum");
    +      } else if (key.equalsIgnoreCase("hll")) {
    +        functions.put(key, "hll_merge");
    +      } else if (key.equalsIgnoreCase("sum_width")) {
    +        functions.put(key, "avg_width");
    +      }
    +    }*/
    +    for (String key : popConfig.getFunctions()) {
    --- End diff --
    
    Is the idea that the set of functions is configurable or extendable? This seems to be how other operators are defined. If so, should there be a separate function definition that provides the information about the name of aggregation and summary functions? Else, seems awkward to hard-code these details in various places in 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 #729: Drill 1328: Support table statistics for Parquet

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

    https://github.com/apache/drill/pull/729#discussion_r101167425
  
    --- Diff: exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/statistics/StatisticsAggBatch.java ---
    @@ -0,0 +1,256 @@
    +/**
    + * 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.impl.statistics;
    +
    +import com.google.common.collect.Lists;
    +import com.sun.codemodel.JExpr;
    +import org.apache.drill.common.expression.ErrorCollector;
    +import org.apache.drill.common.expression.ErrorCollectorImpl;
    +import org.apache.drill.common.expression.FunctionCallFactory;
    +import org.apache.drill.common.expression.LogicalExpression;
    +import org.apache.drill.common.expression.SchemaPath;
    +import org.apache.drill.common.expression.ValueExpressions;
    +import org.apache.drill.exec.exception.ClassTransformationException;
    +import org.apache.drill.exec.exception.OutOfMemoryException;
    +import org.apache.drill.exec.exception.SchemaChangeException;
    +import org.apache.drill.exec.expr.ClassGenerator;
    +import org.apache.drill.exec.expr.CodeGenerator;
    +import org.apache.drill.exec.expr.ExpressionTreeMaterializer;
    +import org.apache.drill.exec.expr.TypeHelper;
    +import org.apache.drill.exec.expr.ValueVectorWriteExpression;
    +import org.apache.drill.exec.ops.FragmentContext;
    +import org.apache.drill.exec.physical.config.StatisticsAggregate;
    +import org.apache.drill.exec.physical.impl.aggregate.StreamingAggBatch;
    +import org.apache.drill.exec.physical.impl.aggregate.StreamingAggTemplate;
    +import org.apache.drill.exec.physical.impl.aggregate.StreamingAggregator;
    +import org.apache.drill.exec.planner.physical.StatsAggPrel.OperatorPhase;
    +import org.apache.drill.exec.record.BatchSchema.SelectionVectorMode;
    +import org.apache.drill.exec.record.MaterializedField;
    +import org.apache.drill.exec.record.RecordBatch;
    +import org.apache.drill.exec.record.TypedFieldId;
    +import org.apache.drill.exec.store.ImplicitColumnExplorer;
    +import org.apache.drill.exec.vector.ValueVector;
    +import org.apache.drill.exec.vector.complex.FieldIdUtil;
    +import org.apache.drill.exec.vector.complex.MapVector;
    +
    +import java.io.IOException;
    +import java.util.GregorianCalendar;
    +import java.util.List;
    +import java.util.TimeZone;
    +
    +/**
    + * TODO: This needs cleanup. Currently the key values are constants and we compare the constants
    + * for every record. Seems unnecessary.
    + *
    + * Example input and output:
    + * Schema of incoming batch: region_id (VARCHAR), sales_city (VARCHAR), cnt (BIGINT)
    + * Schema of output:
    + *    "schema" : BIGINT - Schema number. For each schema change this number is incremented.
    + *    "computed" : BIGINT - What time is it computed?
    + *    "columns"       : MAP - Column names
    + *       "region_id"  : VARCHAR
    + *       "sales_city" : VARCHAR
    + *       "cnt"        : VARCHAR
    + *    "statscount" : MAP
    + *       "region_id"  : BIGINT - statscount(region_id) - aggregation over all values of region_id
    + *                      in incoming batch
    + *       "sales_city" : BIGINT - statscount(sales_city)
    + *       "cnt"        : BIGINT - statscount(cnt)
    + *    "nonnullstatcount" : MAP
    + *       "region_id"  : BIGINT - nonnullstatcount(region_id)
    + *       "sales_city" : BIGINT - nonnullstatcount(sales_city)
    + *       "cnt"        : BIGINT - nonnullstatcount(cnt)
    + *   .... another map for next stats function ....
    + */
    +public class StatisticsAggBatch extends StreamingAggBatch {
    +  private List<String> functions;
    +  private int schema = 0;
    +
    +  public StatisticsAggBatch(StatisticsAggregate popConfig, RecordBatch incoming,
    +      FragmentContext context) throws OutOfMemoryException {
    +    super(popConfig, incoming, context);
    +    this.functions = popConfig.getFunctions();
    +  }
    +
    +  private void createKeyColumn(String name, LogicalExpression expr, List<LogicalExpression> keyExprs,
    +      List<TypedFieldId> keyOutputIds) throws SchemaChangeException {
    +    ErrorCollector collector = new ErrorCollectorImpl();
    +
    +    LogicalExpression mle = ExpressionTreeMaterializer.materialize(expr, incoming, collector,
    +        context.getFunctionRegistry());
    +
    +    MaterializedField outputField = MaterializedField.create(name, mle.getMajorType());
    +    ValueVector vector = TypeHelper.getNewVector(outputField, oContext.getAllocator());
    +
    +    keyExprs.add(mle);
    +    keyOutputIds.add(container.add(vector));
    +
    +    if (collector.hasErrors()) {
    +      throw new SchemaChangeException("Failure while materializing expression. "
    +          + collector.toErrorString());
    +    }
    +  }
    +
    +  private void createNestedKeyColumn(MapVector parent, String name, LogicalExpression expr,
    +      List<LogicalExpression> keyExprs, List<TypedFieldId> keyOutputIds)
    +          throws SchemaChangeException {
    +    ErrorCollector collector = new ErrorCollectorImpl();
    +
    +    LogicalExpression mle = ExpressionTreeMaterializer.materialize(expr, incoming, collector,
    +        context.getFunctionRegistry());
    +
    +    Class<? extends ValueVector> vvc =
    +        TypeHelper.getValueVectorClass(mle.getMajorType().getMinorType(),
    +            mle.getMajorType().getMode());
    +
    +    ValueVector vv = parent.addOrGet(name, mle.getMajorType(), vvc);
    +
    +    TypedFieldId pfid = container.getValueVectorId(SchemaPath.getSimplePath(parent.getField().getPath()));
    +    assert pfid.getFieldIds().length == 1;
    +    TypedFieldId.Builder builder = TypedFieldId.newBuilder();
    +    builder.addId(pfid.getFieldIds()[0]);
    +    TypedFieldId id =
    +        FieldIdUtil.getFieldIdIfMatches(parent, builder, true,
    +            SchemaPath.getSimplePath(vv.getField().getPath()).getRootSegment());
    +
    +    keyExprs.add(mle);
    +    keyOutputIds.add(id);
    +
    +    if (collector.hasErrors()) {
    +      throw new SchemaChangeException("Failure while materializing expression. "
    +          + collector.toErrorString());
    +    }
    +  }
    +
    +  private void addMapVector(String name, MapVector parent, LogicalExpression expr,
    +      List<LogicalExpression> valueExprs) throws SchemaChangeException {
    +    ErrorCollector collector = new ErrorCollectorImpl();
    --- End diff --
    
    Refactored the code - no code duplication


---
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 #729: Drill 1328: Support table statistics for Parquet

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/729#discussion_r102866471
  
    --- Diff: exec/java-exec/src/main/java/org/apache/drill/exec/ops/ContextInformation.java ---
    @@ -29,13 +29,15 @@
       private final long queryStartTime;
       private final int rootFragmentTimeZone;
       private final String sessionId;
    +  private final int hllAccuracy;
     
       public ContextInformation(final UserCredentials userCredentials, final QueryContextInformation queryContextInfo) {
         this.queryUser = userCredentials.getUserName();
         this.currentDefaultSchema = queryContextInfo.getDefaultSchemaName();
         this.queryStartTime = queryContextInfo.getQueryStartTime();
         this.rootFragmentTimeZone = queryContextInfo.getTimeZone();
         this.sessionId = queryContextInfo.getSessionId();
    +    this.hllAccuracy = queryContextInfo.getHllAccuracy();
    --- End diff --
    
    The query context is very general and is probably not the place to store specific options such as the hllAccuracy. The operator and/or factory can get the value directly from the option manager.
    
    Actually, any stats options should be set on the operator definition itself (`StatisticsAggregate`, etc.) so that all fragments use the same value: the one selected when creating the plan. This behavior would mimic how we set memory for sort operators, etc.


---
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 #729: Drill 1328: Support table statistics for Parquet

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/729#discussion_r102871257
  
    --- Diff: exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/statistics/AvgWidthMergedStatistic.java ---
    @@ -0,0 +1,135 @@
    +/*
    + * 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.impl.statistics;
    +
    +import org.apache.drill.common.types.TypeProtos;
    +import org.apache.drill.exec.expr.holders.NullableFloat8Holder;
    +import org.apache.drill.exec.expr.holders.ValueHolder;
    +import org.apache.drill.exec.vector.NullableFloat8Vector;
    +import org.apache.drill.exec.vector.ValueVector;
    +import org.apache.drill.exec.vector.complex.MapVector;
    +
    +import java.util.HashMap;
    +import java.util.List;
    +import java.util.Map;
    +
    +public class AvgWidthMergedStatistic extends AbstractMergedStatistic {
    +
    +  private String name;
    +  private String inputName;
    +  private boolean configureComplete = false;
    +  private boolean mergeComplete = false;
    +  private Map<String, ValueHolder> sumHolder;
    +  MergedStatistic types, nonNullStatCounts, statCounts;
    +
    +  public AvgWidthMergedStatistic (String name, String inputName) {
    +    this.name = name;
    +    this.inputName = inputName;
    +    this.sumHolder = new HashMap<>();
    +    types = nonNullStatCounts = statCounts = null;
    +  }
    +
    +  @Override
    +  public String getName() {
    +    return name;
    +  }
    +
    +  @Override
    +  public String getInput() {
    +    return inputName;
    +  }
    +
    +  @Override
    +  public void merge(ValueVector input) {
    +    // Check the input is a Map Vector
    +    assert (input.getField().getType().getMinorType() == TypeProtos.MinorType.MAP);
    +    MapVector inputMap = (MapVector) input;
    +    for (ValueVector vv : inputMap) {
    +      String colName = vv.getField().getLastName();
    +      NullableFloat8Holder colSumHolder;
    +      if (sumHolder.get(colName) != null) {
    +        colSumHolder = (NullableFloat8Holder) sumHolder.get(colName);
    +      } else {
    +        colSumHolder = new NullableFloat8Holder();
    +        sumHolder.put(colName, colSumHolder);
    +      }
    +      Object val = vv.getAccessor().getObject(0);
    +      if (val != null) {
    +        colSumHolder.value += (double) val;
    +        colSumHolder.isSet = 1;
    +      }
    +    }
    +  }
    +
    +  @Override
    +  public Object getStat(String colName) {
    +      if (mergeComplete != true) {
    +        throw new IllegalStateException(
    +            String.format("Statistic `%s` has not completed merging statistics", name));
    +      }
    +      NullableFloat8Holder colSumHolder = (NullableFloat8Holder) sumHolder.get(colName);
    +      return (long) (colSumHolder.value/ getRowCount(colName));
    +    }
    +
    +  @Override
    +  public void setOutput(ValueVector output) {
    --- End diff --
    
    ```
    setOutput(MapVector output)
    ```
    
    Does not make sense with any other vector type.


---
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 #729: Drill 1328 r4

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

    https://github.com/apache/drill/pull/729
  
    Please rename PR to standard format:
    ```
    DRILL-1328: Some textual description
    ```
    For us novices, "r4" is a bit of a terse description of the purpose of this PR...


---
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 #729: Drill 1328: Support table statistics for Parquet

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/729#discussion_r102875193
  
    --- Diff: exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/statistics/Statistic.java ---
    @@ -0,0 +1,38 @@
    +/**
    + * 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.impl.statistics;
    +
    +/*
    + * Base Statistics class - all statistics classes should extend this class
    + */
    +public abstract class Statistic {
    +  /*
    +   * List of statistics used in Drill.
    +   */
    +  public static final String COLNAME = "column";
    +  public static final String COLTYPE = "type";
    +  public static final String SCHEMA = "schema";
    +  public static final String COMPUTED = "computed";
    +  public static final String STATCOUNT = "statcount";
    +  public static final String NNSTATCOUNT = "nonnullstatcount";
    +  public static final String NDV = "ndv";
    +  public static final String HLL_MERGE = "hll_merge";
    +  public static final String HLL = "hll";
    +  public static final String AVG_WIDTH = "avg_width";
    +  public static final String SUM_WIDTH = "sum_width";
    --- End diff --
    
    Nice!


---
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 #729: Drill 1328: Support table statistics for Parquet

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

    https://github.com/apache/drill/pull/729#discussion_r100708172
  
    --- Diff: exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/statistics/StatisticsAggBatch.java ---
    @@ -0,0 +1,256 @@
    +/**
    + * 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.impl.statistics;
    +
    +import com.google.common.collect.Lists;
    +import com.sun.codemodel.JExpr;
    +import org.apache.drill.common.expression.ErrorCollector;
    +import org.apache.drill.common.expression.ErrorCollectorImpl;
    +import org.apache.drill.common.expression.FunctionCallFactory;
    +import org.apache.drill.common.expression.LogicalExpression;
    +import org.apache.drill.common.expression.SchemaPath;
    +import org.apache.drill.common.expression.ValueExpressions;
    +import org.apache.drill.exec.exception.ClassTransformationException;
    +import org.apache.drill.exec.exception.OutOfMemoryException;
    +import org.apache.drill.exec.exception.SchemaChangeException;
    +import org.apache.drill.exec.expr.ClassGenerator;
    +import org.apache.drill.exec.expr.CodeGenerator;
    +import org.apache.drill.exec.expr.ExpressionTreeMaterializer;
    +import org.apache.drill.exec.expr.TypeHelper;
    +import org.apache.drill.exec.expr.ValueVectorWriteExpression;
    +import org.apache.drill.exec.ops.FragmentContext;
    +import org.apache.drill.exec.physical.config.StatisticsAggregate;
    +import org.apache.drill.exec.physical.impl.aggregate.StreamingAggBatch;
    +import org.apache.drill.exec.physical.impl.aggregate.StreamingAggTemplate;
    +import org.apache.drill.exec.physical.impl.aggregate.StreamingAggregator;
    +import org.apache.drill.exec.planner.physical.StatsAggPrel.OperatorPhase;
    +import org.apache.drill.exec.record.BatchSchema.SelectionVectorMode;
    +import org.apache.drill.exec.record.MaterializedField;
    +import org.apache.drill.exec.record.RecordBatch;
    +import org.apache.drill.exec.record.TypedFieldId;
    +import org.apache.drill.exec.store.ImplicitColumnExplorer;
    +import org.apache.drill.exec.vector.ValueVector;
    +import org.apache.drill.exec.vector.complex.FieldIdUtil;
    +import org.apache.drill.exec.vector.complex.MapVector;
    +
    +import java.io.IOException;
    +import java.util.GregorianCalendar;
    +import java.util.List;
    +import java.util.TimeZone;
    +
    +/**
    + * TODO: This needs cleanup. Currently the key values are constants and we compare the constants
    + * for every record. Seems unnecessary.
    + *
    + * Example input and output:
    + * Schema of incoming batch: region_id (VARCHAR), sales_city (VARCHAR), cnt (BIGINT)
    + * Schema of output:
    + *    "schema" : BIGINT - Schema number. For each schema change this number is incremented.
    + *    "computed" : BIGINT - What time is it computed?
    + *    "columns"       : MAP - Column names
    + *       "region_id"  : VARCHAR
    + *       "sales_city" : VARCHAR
    + *       "cnt"        : VARCHAR
    + *    "statscount" : MAP
    + *       "region_id"  : BIGINT - statscount(region_id) - aggregation over all values of region_id
    + *                      in incoming batch
    + *       "sales_city" : BIGINT - statscount(sales_city)
    + *       "cnt"        : BIGINT - statscount(cnt)
    + *    "nonnullstatcount" : MAP
    + *       "region_id"  : BIGINT - nonnullstatcount(region_id)
    + *       "sales_city" : BIGINT - nonnullstatcount(sales_city)
    + *       "cnt"        : BIGINT - nonnullstatcount(cnt)
    + *   .... another map for next stats function ....
    + */
    +public class StatisticsAggBatch extends StreamingAggBatch {
    +  private List<String> functions;
    +  private int schema = 0;
    +
    +  public StatisticsAggBatch(StatisticsAggregate popConfig, RecordBatch incoming,
    +      FragmentContext context) throws OutOfMemoryException {
    +    super(popConfig, incoming, context);
    +    this.functions = popConfig.getFunctions();
    +  }
    +
    +  private void createKeyColumn(String name, LogicalExpression expr, List<LogicalExpression> keyExprs,
    +      List<TypedFieldId> keyOutputIds) throws SchemaChangeException {
    +    ErrorCollector collector = new ErrorCollectorImpl();
    +
    +    LogicalExpression mle = ExpressionTreeMaterializer.materialize(expr, incoming, collector,
    +        context.getFunctionRegistry());
    +
    +    MaterializedField outputField = MaterializedField.create(name, mle.getMajorType());
    +    ValueVector vector = TypeHelper.getNewVector(outputField, oContext.getAllocator());
    +
    +    keyExprs.add(mle);
    +    keyOutputIds.add(container.add(vector));
    +
    +    if (collector.hasErrors()) {
    +      throw new SchemaChangeException("Failure while materializing expression. "
    +          + collector.toErrorString());
    +    }
    +  }
    +
    +  private void createNestedKeyColumn(MapVector parent, String name, LogicalExpression expr,
    +      List<LogicalExpression> keyExprs, List<TypedFieldId> keyOutputIds)
    +          throws SchemaChangeException {
    +    ErrorCollector collector = new ErrorCollectorImpl();
    +
    +    LogicalExpression mle = ExpressionTreeMaterializer.materialize(expr, incoming, collector,
    +        context.getFunctionRegistry());
    +
    +    Class<? extends ValueVector> vvc =
    +        TypeHelper.getValueVectorClass(mle.getMajorType().getMinorType(),
    +            mle.getMajorType().getMode());
    +
    +    ValueVector vv = parent.addOrGet(name, mle.getMajorType(), vvc);
    +
    +    TypedFieldId pfid = container.getValueVectorId(SchemaPath.getSimplePath(parent.getField().getPath()));
    +    assert pfid.getFieldIds().length == 1;
    +    TypedFieldId.Builder builder = TypedFieldId.newBuilder();
    +    builder.addId(pfid.getFieldIds()[0]);
    +    TypedFieldId id =
    +        FieldIdUtil.getFieldIdIfMatches(parent, builder, true,
    +            SchemaPath.getSimplePath(vv.getField().getPath()).getRootSegment());
    +
    +    keyExprs.add(mle);
    +    keyOutputIds.add(id);
    +
    +    if (collector.hasErrors()) {
    --- 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 #729: Drill 1328: Support table statistics for Parquet

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/729#discussion_r100680426
  
    --- Diff: exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/statistics/StatisticsMergeBatch.java ---
    @@ -0,0 +1,532 @@
    +/**
    + * 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.impl.statistics;
    +
    +import com.google.common.collect.Lists;
    +import com.google.common.collect.Maps;
    +import org.apache.drill.common.expression.LogicalExpression;
    +import org.apache.drill.common.expression.ErrorCollector;
    +import org.apache.drill.common.expression.ErrorCollectorImpl;
    +import org.apache.drill.common.expression.FunctionCallFactory;
    +import org.apache.drill.common.expression.SchemaPath;
    +import org.apache.drill.common.expression.ValueExpressions;
    +import org.apache.drill.common.types.TypeProtos;
    +import org.apache.drill.common.types.TypeProtos.MajorType;
    +import org.apache.drill.exec.exception.OutOfMemoryException;
    +import org.apache.drill.exec.exception.SchemaChangeException;
    +import org.apache.drill.exec.expr.ExpressionTreeMaterializer;
    +import org.apache.drill.exec.expr.TypeHelper;
    +import org.apache.drill.exec.expr.holders.BigIntHolder;
    +import org.apache.drill.exec.expr.holders.NullableFloat8Holder;
    +import org.apache.drill.exec.expr.holders.ObjectHolder;
    +import org.apache.drill.exec.expr.holders.ValueHolder;
    +import org.apache.drill.exec.ops.FragmentContext;
    +import org.apache.drill.exec.physical.config.StatisticsMerge;
    +import org.apache.drill.exec.record.AbstractSingleRecordBatch;
    +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.VectorContainer;
    +import org.apache.drill.exec.record.VectorWrapper;
    +import org.apache.drill.exec.vector.ValueVector;
    +import org.apache.drill.exec.vector.DateVector;
    +import org.apache.drill.exec.vector.BigIntVector;
    +import org.apache.drill.exec.vector.NullableBigIntVector;
    +import org.apache.drill.exec.vector.NullableFloat8Vector;
    +import org.apache.drill.exec.vector.NullableVarBinaryVector;
    +import org.apache.drill.exec.vector.VarCharVector;
    +import org.apache.drill.exec.vector.complex.MapVector;
    +
    +import java.io.ByteArrayInputStream;
    +import java.io.DataInputStream;
    +import java.io.IOException;
    +import java.util.List;
    +import java.util.GregorianCalendar;
    +import java.util.HashMap;
    +import java.util.Map;
    +import java.util.TimeZone;
    +
    +import com.clearspring.analytics.stream.cardinality.HyperLogLog;
    +
    +public class StatisticsMergeBatch extends AbstractSingleRecordBatch<StatisticsMerge> {
    +  private static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(StatisticsMergeBatch.class);
    +  private Map<String, String> functions;
    +  private boolean first = true;
    +  private boolean finished = false;
    +  private int schema = 0;
    +  private int recordCount = 0;
    +  private List<String> keyList = null;
    +  private Map<MaterializedField, ValueVector> dataSrcVecMap = null;
    +  // Map of non-map fields to VV in the incoming schema
    +  private Map<MaterializedField, ValueVector> copySrcVecMap = null;
    +  private Map<String, Map<String, ValueHolder>> aggregationMap = null;
    +  public StatisticsMergeBatch(StatisticsMerge popConfig, RecordBatch incoming,
    +                              FragmentContext context) throws OutOfMemoryException {
    +    super(popConfig, context, incoming);
    +    this.functions = new HashMap<>();
    +    this.aggregationMap = new HashMap<>();
    +
    +    /*for (String key : popConfig.getFunctions()) {
    +      aggregationMap.put(key, new HashMap<String, ValueHolder>());
    +      if (key.equalsIgnoreCase("statcount") || key.equalsIgnoreCase("nonnullstatcount")) {
    +        functions.put(key, "sum");
    +      } else if (key.equalsIgnoreCase("hll")) {
    +        functions.put(key, "hll_merge");
    +      } else if (key.equalsIgnoreCase("sum_width")) {
    +        functions.put(key, "avg_width");
    +      }
    +    }*/
    +    for (String key : popConfig.getFunctions()) {
    +      if (key.equalsIgnoreCase("sum_width")) {
    +        functions.put(key, "avg_width");
    +      } else if (key.equalsIgnoreCase("hll")) {
    +        functions.put(key, "hll_merge");
    +      } else {
    +        functions.put(key, key);
    +      }
    +      aggregationMap.put(functions.get(key), new HashMap<String, ValueHolder>());
    +    }
    +  }
    +
    +  private void createKeyColumn(String name, LogicalExpression expr, Map<MaterializedField, ValueVector> parentMap)
    +      throws SchemaChangeException {
    +    ErrorCollector collector = new ErrorCollectorImpl();
    +
    +    LogicalExpression mle = ExpressionTreeMaterializer.materialize(expr, incoming, collector,
    +        context.getFunctionRegistry());
    +
    +    MaterializedField outputField = MaterializedField.create(name, mle.getMajorType());
    +    ValueVector vector = TypeHelper.getNewVector(outputField, oContext.getAllocator());
    +    container.add(vector);
    +
    +    if (collector.hasErrors()) {
    +      throw new SchemaChangeException("Failure while materializing expression. "
    +          + collector.toErrorString());
    +    }
    +    parentMap.put(outputField, vector);
    +  }
    +
    +  private ValueVector addMapVector(String name, MapVector parent, LogicalExpression expr) throws SchemaChangeException {
    +    ErrorCollector collector = new ErrorCollectorImpl();
    --- End diff --
    
    Lots of of duplication here and in similar places above.


---
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 #729: Drill 1328: Support table statistics for Parquet

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/729#discussion_r100680684
  
    --- Diff: exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/statistics/StatisticsMergeBatch.java ---
    @@ -0,0 +1,532 @@
    +/**
    + * 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.impl.statistics;
    +
    +import com.google.common.collect.Lists;
    +import com.google.common.collect.Maps;
    +import org.apache.drill.common.expression.LogicalExpression;
    +import org.apache.drill.common.expression.ErrorCollector;
    +import org.apache.drill.common.expression.ErrorCollectorImpl;
    +import org.apache.drill.common.expression.FunctionCallFactory;
    +import org.apache.drill.common.expression.SchemaPath;
    +import org.apache.drill.common.expression.ValueExpressions;
    +import org.apache.drill.common.types.TypeProtos;
    +import org.apache.drill.common.types.TypeProtos.MajorType;
    +import org.apache.drill.exec.exception.OutOfMemoryException;
    +import org.apache.drill.exec.exception.SchemaChangeException;
    +import org.apache.drill.exec.expr.ExpressionTreeMaterializer;
    +import org.apache.drill.exec.expr.TypeHelper;
    +import org.apache.drill.exec.expr.holders.BigIntHolder;
    +import org.apache.drill.exec.expr.holders.NullableFloat8Holder;
    +import org.apache.drill.exec.expr.holders.ObjectHolder;
    +import org.apache.drill.exec.expr.holders.ValueHolder;
    +import org.apache.drill.exec.ops.FragmentContext;
    +import org.apache.drill.exec.physical.config.StatisticsMerge;
    +import org.apache.drill.exec.record.AbstractSingleRecordBatch;
    +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.VectorContainer;
    +import org.apache.drill.exec.record.VectorWrapper;
    +import org.apache.drill.exec.vector.ValueVector;
    +import org.apache.drill.exec.vector.DateVector;
    +import org.apache.drill.exec.vector.BigIntVector;
    +import org.apache.drill.exec.vector.NullableBigIntVector;
    +import org.apache.drill.exec.vector.NullableFloat8Vector;
    +import org.apache.drill.exec.vector.NullableVarBinaryVector;
    +import org.apache.drill.exec.vector.VarCharVector;
    +import org.apache.drill.exec.vector.complex.MapVector;
    +
    +import java.io.ByteArrayInputStream;
    +import java.io.DataInputStream;
    +import java.io.IOException;
    +import java.util.List;
    +import java.util.GregorianCalendar;
    +import java.util.HashMap;
    +import java.util.Map;
    +import java.util.TimeZone;
    +
    +import com.clearspring.analytics.stream.cardinality.HyperLogLog;
    +
    +public class StatisticsMergeBatch extends AbstractSingleRecordBatch<StatisticsMerge> {
    +  private static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(StatisticsMergeBatch.class);
    +  private Map<String, String> functions;
    +  private boolean first = true;
    +  private boolean finished = false;
    +  private int schema = 0;
    +  private int recordCount = 0;
    +  private List<String> keyList = null;
    +  private Map<MaterializedField, ValueVector> dataSrcVecMap = null;
    +  // Map of non-map fields to VV in the incoming schema
    +  private Map<MaterializedField, ValueVector> copySrcVecMap = null;
    +  private Map<String, Map<String, ValueHolder>> aggregationMap = null;
    +  public StatisticsMergeBatch(StatisticsMerge popConfig, RecordBatch incoming,
    +                              FragmentContext context) throws OutOfMemoryException {
    +    super(popConfig, context, incoming);
    +    this.functions = new HashMap<>();
    +    this.aggregationMap = new HashMap<>();
    +
    +    /*for (String key : popConfig.getFunctions()) {
    +      aggregationMap.put(key, new HashMap<String, ValueHolder>());
    +      if (key.equalsIgnoreCase("statcount") || key.equalsIgnoreCase("nonnullstatcount")) {
    +        functions.put(key, "sum");
    +      } else if (key.equalsIgnoreCase("hll")) {
    +        functions.put(key, "hll_merge");
    +      } else if (key.equalsIgnoreCase("sum_width")) {
    +        functions.put(key, "avg_width");
    +      }
    +    }*/
    +    for (String key : popConfig.getFunctions()) {
    +      if (key.equalsIgnoreCase("sum_width")) {
    +        functions.put(key, "avg_width");
    +      } else if (key.equalsIgnoreCase("hll")) {
    +        functions.put(key, "hll_merge");
    +      } else {
    +        functions.put(key, key);
    +      }
    +      aggregationMap.put(functions.get(key), new HashMap<String, ValueHolder>());
    +    }
    +  }
    +
    +  private void createKeyColumn(String name, LogicalExpression expr, Map<MaterializedField, ValueVector> parentMap)
    +      throws SchemaChangeException {
    +    ErrorCollector collector = new ErrorCollectorImpl();
    +
    +    LogicalExpression mle = ExpressionTreeMaterializer.materialize(expr, incoming, collector,
    +        context.getFunctionRegistry());
    +
    +    MaterializedField outputField = MaterializedField.create(name, mle.getMajorType());
    +    ValueVector vector = TypeHelper.getNewVector(outputField, oContext.getAllocator());
    +    container.add(vector);
    +
    +    if (collector.hasErrors()) {
    +      throw new SchemaChangeException("Failure while materializing expression. "
    +          + collector.toErrorString());
    +    }
    +    parentMap.put(outputField, vector);
    +  }
    +
    +  private ValueVector addMapVector(String name, MapVector parent, LogicalExpression expr) throws SchemaChangeException {
    +    ErrorCollector collector = new ErrorCollectorImpl();
    +
    +    LogicalExpression mle = ExpressionTreeMaterializer.materialize(expr, incoming, collector,
    +        context.getFunctionRegistry());
    +
    +    Class<? extends ValueVector> vvc =
    +        TypeHelper.getValueVectorClass(mle.getMajorType().getMinorType(), mle.getMajorType().getMode());
    +    ValueVector vector = parent.addOrGet(name, mle.getMajorType(), vvc);
    +
    +    if (collector.hasErrors()) {
    +      throw new SchemaChangeException("Failure while materializing expression. "
    +          + collector.toErrorString());
    +    }
    +    return vector;
    +  }
    +
    +  /**
    +   * Identify the list of fields within a map which are unpivoted as columns in output
    +   */
    +  private void buildKeyList() {
    +    List<String> lastMapKeyList = null;
    +    for (VectorWrapper<?> vw : incoming) {
    +      if (vw.getField().getType().getMinorType() != TypeProtos.MinorType.MAP) {
    +        continue;
    +      }
    +
    +      keyList = Lists.newArrayList();
    +
    +      for (ValueVector vv : vw.getValueVector()) {
    +        keyList.add(vv.getField().getLastName());
    +      }
    +
    +      if (lastMapKeyList == null) {
    +        lastMapKeyList = keyList;
    +      } else {
    +        if (keyList.size() != lastMapKeyList.size() || !lastMapKeyList.containsAll(keyList)) {
    +          throw new UnsupportedOperationException("Maps have different fields");
    +        }
    +      }
    +    }
    +  }
    +
    +  private void buildOutputContainer() throws SchemaChangeException {
    +    dataSrcVecMap = Maps.newHashMap();
    +    copySrcVecMap = Maps.newHashMap();
    +    MajorType mt = null;
    +
    +    ErrorCollector collector = new ErrorCollectorImpl();
    +    GregorianCalendar calendar = new GregorianCalendar(TimeZone.getTimeZone("UTC"));
    +
    +    calendar.setTimeInMillis(System.currentTimeMillis());
    +    createKeyColumn("schema", ValueExpressions.getBigInt(schema++), copySrcVecMap);
    +    createKeyColumn("computed", ValueExpressions.getDate(calendar), copySrcVecMap);
    +
    +    for (VectorWrapper<?> vw : incoming) {
    +      addVectorToOutgoingContainer(vw.getField().getLastName(), vw, collector);
    +      /*MaterializedField ds = vw.getField();
    +      String field = vw.getField().getLastName();
    +      // Input map vector
    +      MapVector mapVector = (MapVector) vw.getValueVector();
    +      assert mapVector.getPrimitiveVectors().size() > 0;
    +      // Proceed to create output map vector with same name e.g. statcount etc.
    +      mt = mapVector.getField().getType();
    +      MaterializedField mf = MaterializedField.create(functions.get(field), mt);
    +      assert !dataSrcVecMap.containsKey(mf);
    +      ValueVector vector = TypeHelper.getNewVector(mf, oContext.getAllocator());
    +      container.add(vector);
    +      MapVector outputMapVector = (MapVector) vector;
    +
    +      for (ValueVector vv : mapVector) {
    +        String fieldName = vv.getField().getLastName();
    +        if (!keyList.contains(fieldName)) {
    +          throw new UnsupportedOperationException("Unpivot data vector " +
    +              ds + " contains key " + fieldName + " not contained in key source!");
    +        }
    +        if (vv.getField().getType().getMinorType() == TypeProtos.MinorType.MAP) {
    +          throw new UnsupportedOperationException("Unpivot of nested map is not supported!");
    +        }
    +        if (field.equals("column")) {
    +          outputMapVector.addOrGet(fieldName, vv.getField().getType(), vv.getClass());
    +        } else {
    +          List<LogicalExpression> args = Lists.newArrayList();
    +          //TODO: Something else to access value of col such as emp_id?
    +          args.add(SchemaPath.getSimplePath(vv.getField().getPath()));
    +          //TODO: Put in the mapVector
    +          LogicalExpression call = FunctionCallFactory.createExpression(functions.get(field), args);
    +          //TODO: Is this sufficient to add to new Map?
    +          ValueVector vector1 = addMapVector(fieldName, outputMapVector, call);
    +          if (collector.hasErrors()) {
    +            throw new SchemaChangeException("Failure while materializing expression. "
    +                + collector.toErrorString());
    +          }
    +        }
    +      }
    +      dataSrcVecMap.put(ds, outputMapVector);*/
    +    }
    +    //Now create NDV in the outgoing container which was not avaliable in the incoming
    +    for (VectorWrapper<?> vw : incoming) {
    +      if (vw.getField().getLastName().equalsIgnoreCase("sum_width")) {//NullableFloat8 type
    +        addVectorToOutgoingContainer("ndv", vw, collector);
    +        break;
    +      }
    +    }
    +    container.setRecordCount(0);
    +    recordCount = 0;
    +    container.buildSchema(incoming.getSchema().getSelectionVectorMode());
    +  }
    +
    +  private void addVectorToOutgoingContainer(String field, VectorWrapper vw, ErrorCollector collector)
    +     throws SchemaChangeException {
    +    // Input map vector
    +    MapVector mapVector = (MapVector) vw.getValueVector();
    +    MaterializedField mf;
    +    assert mapVector.getPrimitiveVectors().size() > 0;
    +    // Proceed to create output map vector with same name e.g. statcount etc.
    +    MajorType mt = mapVector.getField().getType();
    +    if (functions.get(field) != null) {
    +      mf = MaterializedField.create(functions.get(field), mt);
    +    } else {
    +      mf = MaterializedField.create(field, mt);
    +    }
    +    assert !dataSrcVecMap.containsKey(mf);
    +    ValueVector vector = TypeHelper.getNewVector(mf, oContext.getAllocator());
    +    container.add(vector);
    +    MapVector outputMapVector = (MapVector) vector;
    +
    +    for (ValueVector vv : mapVector) {
    +      String fieldName = vv.getField().getLastName();
    +      if (!keyList.contains(fieldName)) {
    +        throw new UnsupportedOperationException("Unpivot data vector " +
    +                field + " contains key " + fieldName + " not contained in key source!");
    +      }
    +      if (vv.getField().getType().getMinorType() == TypeProtos.MinorType.MAP) {
    +        throw new UnsupportedOperationException("Unpivot of nested map is not supported!");
    +      }
    +      if (field.equals("column")) {
    +        outputMapVector.addOrGet(fieldName, vv.getField().getType(), vv.getClass());
    +      } else {
    +        List<LogicalExpression> args = Lists.newArrayList();
    +        LogicalExpression call;
    +        //TODO: Something else to access value of col such as emp_id?
    +        args.add(SchemaPath.getSimplePath(vv.getField().getPath()));
    +        //TODO: Put in the mapVector
    +        if (functions.get(field) != null) {
    +          call = FunctionCallFactory.createExpression(functions.get(field), args);
    +        } else {
    +          call = FunctionCallFactory.createExpression(field, args);
    +        }
    +        //TODO: Is this sufficient to add to new Map?
    +        ValueVector vector1 = addMapVector(fieldName, outputMapVector, call);
    +        if (collector.hasErrors()) {
    +          throw new SchemaChangeException("Failure while materializing expression. "
    +                  + collector.toErrorString());
    +        }
    +      }
    +    }
    +  }
    +
    +  @Override
    +  protected boolean setupNewSchema() throws SchemaChangeException {
    +    container.clear();
    +    buildKeyList();
    +    buildOutputContainer();
    +    return true;
    +  }
    +
    +  @Override
    +  protected IterOutcome doWork() {
    +    int outRecordCount = incoming.getRecordCount();
    +    HashMap<String, Long> nonNullRows = new HashMap<>();
    +
    +    for (VectorWrapper<?> vw : incoming) {
    +      ValueVector vv = vw.getValueVector();
    +      if (vv.getField().getType().getMinorType() != TypeProtos.MinorType.MAP) {
    +        //We only expect Maps in the incoming. STOP, if this is not the case.
    +        return IterOutcome.STOP;
    +      }
    +      MapVector mapVec = (MapVector) vv;
    +      Map<String, ValueHolder> statMap = aggregationMap.get(vv.getField().getLastName());
    +
    +      for (ValueVector mapElt : mapVec) {
    +        if (vv.getField().getLastName().equalsIgnoreCase("column")) {
    --- End diff --
    
    First, how could it be that the name is a case other than what we defined?
    
    Second, to make this very clear, perhaps create constants for things such as special column names, function names and so on.


---
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 #729: Drill 1328: Support table statistics for Parquet

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

    https://github.com/apache/drill/pull/729#discussion_r102325216
  
    --- Diff: exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/statistics/StatisticsMergeBatch.java ---
    @@ -0,0 +1,532 @@
    +/**
    + * 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.impl.statistics;
    +
    +import com.google.common.collect.Lists;
    +import com.google.common.collect.Maps;
    +import org.apache.drill.common.expression.LogicalExpression;
    +import org.apache.drill.common.expression.ErrorCollector;
    +import org.apache.drill.common.expression.ErrorCollectorImpl;
    +import org.apache.drill.common.expression.FunctionCallFactory;
    +import org.apache.drill.common.expression.SchemaPath;
    +import org.apache.drill.common.expression.ValueExpressions;
    +import org.apache.drill.common.types.TypeProtos;
    +import org.apache.drill.common.types.TypeProtos.MajorType;
    +import org.apache.drill.exec.exception.OutOfMemoryException;
    +import org.apache.drill.exec.exception.SchemaChangeException;
    +import org.apache.drill.exec.expr.ExpressionTreeMaterializer;
    +import org.apache.drill.exec.expr.TypeHelper;
    +import org.apache.drill.exec.expr.holders.BigIntHolder;
    +import org.apache.drill.exec.expr.holders.NullableFloat8Holder;
    +import org.apache.drill.exec.expr.holders.ObjectHolder;
    +import org.apache.drill.exec.expr.holders.ValueHolder;
    +import org.apache.drill.exec.ops.FragmentContext;
    +import org.apache.drill.exec.physical.config.StatisticsMerge;
    +import org.apache.drill.exec.record.AbstractSingleRecordBatch;
    +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.VectorContainer;
    +import org.apache.drill.exec.record.VectorWrapper;
    +import org.apache.drill.exec.vector.ValueVector;
    +import org.apache.drill.exec.vector.DateVector;
    +import org.apache.drill.exec.vector.BigIntVector;
    +import org.apache.drill.exec.vector.NullableBigIntVector;
    +import org.apache.drill.exec.vector.NullableFloat8Vector;
    +import org.apache.drill.exec.vector.NullableVarBinaryVector;
    +import org.apache.drill.exec.vector.VarCharVector;
    +import org.apache.drill.exec.vector.complex.MapVector;
    +
    +import java.io.ByteArrayInputStream;
    +import java.io.DataInputStream;
    +import java.io.IOException;
    +import java.util.List;
    +import java.util.GregorianCalendar;
    +import java.util.HashMap;
    +import java.util.Map;
    +import java.util.TimeZone;
    +
    +import com.clearspring.analytics.stream.cardinality.HyperLogLog;
    +
    +public class StatisticsMergeBatch extends AbstractSingleRecordBatch<StatisticsMerge> {
    +  private static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(StatisticsMergeBatch.class);
    +  private Map<String, String> functions;
    +  private boolean first = true;
    +  private boolean finished = false;
    +  private int schema = 0;
    +  private int recordCount = 0;
    +  private List<String> keyList = null;
    +  private Map<MaterializedField, ValueVector> dataSrcVecMap = null;
    +  // Map of non-map fields to VV in the incoming schema
    +  private Map<MaterializedField, ValueVector> copySrcVecMap = null;
    +  private Map<String, Map<String, ValueHolder>> aggregationMap = null;
    +  public StatisticsMergeBatch(StatisticsMerge popConfig, RecordBatch incoming,
    +                              FragmentContext context) throws OutOfMemoryException {
    +    super(popConfig, context, incoming);
    +    this.functions = new HashMap<>();
    +    this.aggregationMap = new HashMap<>();
    +
    +    /*for (String key : popConfig.getFunctions()) {
    +      aggregationMap.put(key, new HashMap<String, ValueHolder>());
    +      if (key.equalsIgnoreCase("statcount") || key.equalsIgnoreCase("nonnullstatcount")) {
    +        functions.put(key, "sum");
    +      } else if (key.equalsIgnoreCase("hll")) {
    +        functions.put(key, "hll_merge");
    +      } else if (key.equalsIgnoreCase("sum_width")) {
    +        functions.put(key, "avg_width");
    +      }
    +    }*/
    +    for (String key : popConfig.getFunctions()) {
    --- End diff --
    
    Yes. The refactored code should allow us to do so. For stats agg, we use the passed in functions (in AnalyzePRule). We do the same now for stats merge instead of hard-coded functions inside the StatisticsMergeBatch.


---
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 #729: Drill 1328: Support table statistics for Parquet

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

    https://github.com/apache/drill/pull/729#discussion_r100414561
  
    --- Diff: exec/java-exec/src/main/java/org/apache/drill/exec/planner/cost/DrillRelMdRowCount.java ---
    @@ -17,32 +17,81 @@
      ******************************************************************************/
     package org.apache.drill.exec.planner.cost;
     
    +import java.io.IOException;
    +
    +import org.apache.calcite.rel.RelNode;
     import org.apache.calcite.rel.core.Aggregate;
    -import org.apache.calcite.rel.core.Filter;
    +import org.apache.calcite.rel.core.TableScan;
     import org.apache.calcite.rel.metadata.ReflectiveRelMetadataProvider;
     import org.apache.calcite.rel.metadata.RelMdRowCount;
     import org.apache.calcite.rel.metadata.RelMetadataProvider;
     import org.apache.calcite.util.BuiltInMethod;
     import org.apache.calcite.util.ImmutableBitSet;
    +import org.apache.drill.exec.planner.common.DrillFilterRelBase;
    +import org.apache.drill.exec.planner.common.DrillRelOptUtil;
    +import org.apache.drill.exec.planner.logical.DrillTable;
    +import org.apache.drill.exec.planner.logical.DrillTranslatableTable;
    +import org.apache.drill.exec.store.parquet.ParquetGroupScan;
     
     public class DrillRelMdRowCount extends RelMdRowCount{
       private static final DrillRelMdRowCount INSTANCE = new DrillRelMdRowCount();
     
       public static final RelMetadataProvider SOURCE = ReflectiveRelMetadataProvider.reflectiveSource(BuiltInMethod.ROW_COUNT.method, INSTANCE);
     
       @Override
    -  public Double getRowCount(Aggregate rel) {
    -    ImmutableBitSet groupKey = ImmutableBitSet.range(rel.getGroupCount());
    -
    -    if (groupKey.isEmpty()) {
    -      return 1.0;
    +  public Double getRowCount(RelNode rel) {
    +    if (rel instanceof TableScan) {
    +      return getRowCount((TableScan) rel);
    +    } else if (rel instanceof DrillFilterRelBase) {
    +      return getRowCount((DrillFilterRelBase) rel);
         } else {
           return super.getRowCount(rel);
         }
       }
     
    -  @Override
    -  public Double getRowCount(Filter rel) {
    +  private Double getRowCount(DrillFilterRelBase rel) {
    +    if (DrillRelOptUtil.guessRows(rel)) {
    +      return super.getRowCount(rel);
    +    }
    +    // Need capped selectivity estimates. See the Filter getRows() method
         return rel.getRows();
       }
    +
    +  private Double getRowCount(TableScan rel) {
    +    DrillTable table;
    +    if (DrillRelOptUtil.guessRows(rel)) {
    +      return super.getRowCount(rel);
    +    }
    +    table = rel.getTable().unwrap(DrillTable.class);
    +    if (table == null) {
    +      table = rel.getTable().unwrap(DrillTranslatableTable.class).getDrillTable();
    +    }
    +    // Return rowcount from statistics, if available. Otherwise, delegate to parent.
    +    try {
    +      if (table != null
    +          && table.getStatsTable() != null
    +          /* For ParquetGroupScan rely on accurate count from the scan instead of
    +           * statistics since partition pruning/filter pushdown might have occurred.
    +           * The other way would be to iterate over the rowgroups present in the
    +           * ParquetGroupScan to obtain the rowcount.
    +           */
    +          && !(table.getGroupScan() instanceof ParquetGroupScan)) {
    --- 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 #729: Drill 1328: Support table statistics for Parquet

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/729#discussion_r102872722
  
    --- Diff: exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/statistics/ColTypeMergedStatistic.java ---
    @@ -0,0 +1,93 @@
    +/**
    + * 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.impl.statistics;
    +
    +import org.apache.drill.common.types.TypeProtos;
    +import org.apache.drill.exec.expr.holders.IntHolder;
    +import org.apache.drill.exec.expr.holders.ValueHolder;
    +import org.apache.drill.exec.vector.IntVector;
    +import org.apache.drill.exec.vector.ValueVector;
    +import org.apache.drill.exec.vector.complex.MapVector;
    +
    +import java.util.HashMap;
    +import java.util.Map;
    +
    +public class ColTypeMergedStatistic extends AbstractMergedStatistic {
    +  private String name;
    +  private String inputName;
    +  private boolean mergeComplete = false;
    +  private Map<String, ValueHolder> typeHolder;
    +
    +
    +  public ColTypeMergedStatistic (String name, String inputName) {
    +    this.name = name;
    +    this.inputName = inputName;
    +    typeHolder = new HashMap<>();
    +  }
    +
    +  @Override
    +  public String getName() {
    +    return name;
    +  }
    +
    +  @Override
    +  public String getInput() {
    +    return inputName;
    +  }
    +
    +  @Override
    +  public void merge(ValueVector input) {
    +    // Check the input is a Map Vector
    +    assert (input.getField().getType().getMinorType() == TypeProtos.MinorType.MAP);
    +    MapVector inputMap = (MapVector) input;
    +    for (ValueVector vv : inputMap) {
    +      String colName = vv.getField().getLastName();
    +      if (typeHolder.get(colName) == null) {
    +        IntHolder colType = new IntHolder();
    +        ((IntVector) vv).getAccessor().get(0, colType);
    +        typeHolder.put(colName, colType);
    +      }
    +    }
    +  }
    +
    +  @Override
    +  public Object getStat(String colName) {
    --- End diff --
    
    I see. Some stats are longs, some are ints?
    
    Then, the getStat() should not be on the base class, it should be defined, with a specific return type, on each subclass. Why? I have to know the subclass to know how to interpret the returned Object. So, I might as well just use the subclass directly and get the unboxed primitive. That is, rather than:
    
    ```
    AbstractMergedStatistic stat = ...
    Object value = stat.getStat();
    if (stat instanceof ColTypeMergedStatistic) {
      int colType = (int) value;
    }
    ```
    Just do:
    
    ```
    ColTypeMergedStatistic colTypeStat = ...;
    int colType = colTypeStat.getType(colName);
    ```


---
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 #729: Drill 1328: Support table statistics for Parquet

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

    https://github.com/apache/drill/pull/729#discussion_r100867795
  
    --- Diff: exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/statistics/StatisticsMergeBatch.java ---
    @@ -0,0 +1,532 @@
    +/**
    + * 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.impl.statistics;
    +
    +import com.google.common.collect.Lists;
    +import com.google.common.collect.Maps;
    +import org.apache.drill.common.expression.LogicalExpression;
    +import org.apache.drill.common.expression.ErrorCollector;
    +import org.apache.drill.common.expression.ErrorCollectorImpl;
    +import org.apache.drill.common.expression.FunctionCallFactory;
    +import org.apache.drill.common.expression.SchemaPath;
    +import org.apache.drill.common.expression.ValueExpressions;
    +import org.apache.drill.common.types.TypeProtos;
    +import org.apache.drill.common.types.TypeProtos.MajorType;
    +import org.apache.drill.exec.exception.OutOfMemoryException;
    +import org.apache.drill.exec.exception.SchemaChangeException;
    +import org.apache.drill.exec.expr.ExpressionTreeMaterializer;
    +import org.apache.drill.exec.expr.TypeHelper;
    +import org.apache.drill.exec.expr.holders.BigIntHolder;
    +import org.apache.drill.exec.expr.holders.NullableFloat8Holder;
    +import org.apache.drill.exec.expr.holders.ObjectHolder;
    +import org.apache.drill.exec.expr.holders.ValueHolder;
    +import org.apache.drill.exec.ops.FragmentContext;
    +import org.apache.drill.exec.physical.config.StatisticsMerge;
    +import org.apache.drill.exec.record.AbstractSingleRecordBatch;
    +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.VectorContainer;
    +import org.apache.drill.exec.record.VectorWrapper;
    +import org.apache.drill.exec.vector.ValueVector;
    +import org.apache.drill.exec.vector.DateVector;
    +import org.apache.drill.exec.vector.BigIntVector;
    +import org.apache.drill.exec.vector.NullableBigIntVector;
    +import org.apache.drill.exec.vector.NullableFloat8Vector;
    +import org.apache.drill.exec.vector.NullableVarBinaryVector;
    +import org.apache.drill.exec.vector.VarCharVector;
    +import org.apache.drill.exec.vector.complex.MapVector;
    +
    +import java.io.ByteArrayInputStream;
    +import java.io.DataInputStream;
    +import java.io.IOException;
    +import java.util.List;
    +import java.util.GregorianCalendar;
    +import java.util.HashMap;
    +import java.util.Map;
    +import java.util.TimeZone;
    +
    +import com.clearspring.analytics.stream.cardinality.HyperLogLog;
    +
    --- 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 #729: Drill 1328: Support table statistics for Parquet

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/729#discussion_r100677171
  
    --- Diff: exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/statistics/StatisticsAggBatch.java ---
    @@ -0,0 +1,256 @@
    +/**
    + * 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.impl.statistics;
    +
    +import com.google.common.collect.Lists;
    +import com.sun.codemodel.JExpr;
    +import org.apache.drill.common.expression.ErrorCollector;
    +import org.apache.drill.common.expression.ErrorCollectorImpl;
    +import org.apache.drill.common.expression.FunctionCallFactory;
    +import org.apache.drill.common.expression.LogicalExpression;
    +import org.apache.drill.common.expression.SchemaPath;
    +import org.apache.drill.common.expression.ValueExpressions;
    +import org.apache.drill.exec.exception.ClassTransformationException;
    +import org.apache.drill.exec.exception.OutOfMemoryException;
    +import org.apache.drill.exec.exception.SchemaChangeException;
    +import org.apache.drill.exec.expr.ClassGenerator;
    +import org.apache.drill.exec.expr.CodeGenerator;
    +import org.apache.drill.exec.expr.ExpressionTreeMaterializer;
    +import org.apache.drill.exec.expr.TypeHelper;
    +import org.apache.drill.exec.expr.ValueVectorWriteExpression;
    +import org.apache.drill.exec.ops.FragmentContext;
    +import org.apache.drill.exec.physical.config.StatisticsAggregate;
    +import org.apache.drill.exec.physical.impl.aggregate.StreamingAggBatch;
    +import org.apache.drill.exec.physical.impl.aggregate.StreamingAggTemplate;
    +import org.apache.drill.exec.physical.impl.aggregate.StreamingAggregator;
    +import org.apache.drill.exec.planner.physical.StatsAggPrel.OperatorPhase;
    +import org.apache.drill.exec.record.BatchSchema.SelectionVectorMode;
    +import org.apache.drill.exec.record.MaterializedField;
    +import org.apache.drill.exec.record.RecordBatch;
    +import org.apache.drill.exec.record.TypedFieldId;
    +import org.apache.drill.exec.store.ImplicitColumnExplorer;
    +import org.apache.drill.exec.vector.ValueVector;
    +import org.apache.drill.exec.vector.complex.FieldIdUtil;
    +import org.apache.drill.exec.vector.complex.MapVector;
    +
    +import java.io.IOException;
    +import java.util.GregorianCalendar;
    +import java.util.List;
    +import java.util.TimeZone;
    +
    +/**
    + * TODO: This needs cleanup. Currently the key values are constants and we compare the constants
    + * for every record. Seems unnecessary.
    + *
    + * Example input and output:
    + * Schema of incoming batch: region_id (VARCHAR), sales_city (VARCHAR), cnt (BIGINT)
    + * Schema of output:
    + *    "schema" : BIGINT - Schema number. For each schema change this number is incremented.
    + *    "computed" : BIGINT - What time is it computed?
    + *    "columns"       : MAP - Column names
    + *       "region_id"  : VARCHAR
    + *       "sales_city" : VARCHAR
    + *       "cnt"        : VARCHAR
    + *    "statscount" : MAP
    + *       "region_id"  : BIGINT - statscount(region_id) - aggregation over all values of region_id
    + *                      in incoming batch
    + *       "sales_city" : BIGINT - statscount(sales_city)
    + *       "cnt"        : BIGINT - statscount(cnt)
    + *    "nonnullstatcount" : MAP
    + *       "region_id"  : BIGINT - nonnullstatcount(region_id)
    + *       "sales_city" : BIGINT - nonnullstatcount(sales_city)
    + *       "cnt"        : BIGINT - nonnullstatcount(cnt)
    + *   .... another map for next stats function ....
    + */
    +public class StatisticsAggBatch extends StreamingAggBatch {
    +  private List<String> functions;
    +  private int schema = 0;
    +
    +  public StatisticsAggBatch(StatisticsAggregate popConfig, RecordBatch incoming,
    +      FragmentContext context) throws OutOfMemoryException {
    +    super(popConfig, incoming, context);
    +    this.functions = popConfig.getFunctions();
    +  }
    +
    +  private void createKeyColumn(String name, LogicalExpression expr, List<LogicalExpression> keyExprs,
    +      List<TypedFieldId> keyOutputIds) throws SchemaChangeException {
    +    ErrorCollector collector = new ErrorCollectorImpl();
    +
    +    LogicalExpression mle = ExpressionTreeMaterializer.materialize(expr, incoming, collector,
    +        context.getFunctionRegistry());
    +
    +    MaterializedField outputField = MaterializedField.create(name, mle.getMajorType());
    +    ValueVector vector = TypeHelper.getNewVector(outputField, oContext.getAllocator());
    +
    +    keyExprs.add(mle);
    +    keyOutputIds.add(container.add(vector));
    +
    +    if (collector.hasErrors()) {
    --- End diff --
    
    Shouldn't this check be done just after the call to materialize? Else, we are proceeding along doing work in an error state, only later abandoning the work just done with an exception...
    
    Also, we seem to discard the actual errors. Perhaps rethrow the first error? Or, find an exception class that can hold onto a list of exceptions? Just rethrowing the fist is probably fine, most people will fix just the first error and retry.


---
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 #729: Drill 1328: Support table statistics for Parquet

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/729#discussion_r100677643
  
    --- Diff: exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/statistics/StatisticsAggBatch.java ---
    @@ -0,0 +1,256 @@
    +/**
    + * 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.impl.statistics;
    +
    +import com.google.common.collect.Lists;
    +import com.sun.codemodel.JExpr;
    +import org.apache.drill.common.expression.ErrorCollector;
    +import org.apache.drill.common.expression.ErrorCollectorImpl;
    +import org.apache.drill.common.expression.FunctionCallFactory;
    +import org.apache.drill.common.expression.LogicalExpression;
    +import org.apache.drill.common.expression.SchemaPath;
    +import org.apache.drill.common.expression.ValueExpressions;
    +import org.apache.drill.exec.exception.ClassTransformationException;
    +import org.apache.drill.exec.exception.OutOfMemoryException;
    +import org.apache.drill.exec.exception.SchemaChangeException;
    +import org.apache.drill.exec.expr.ClassGenerator;
    +import org.apache.drill.exec.expr.CodeGenerator;
    +import org.apache.drill.exec.expr.ExpressionTreeMaterializer;
    +import org.apache.drill.exec.expr.TypeHelper;
    +import org.apache.drill.exec.expr.ValueVectorWriteExpression;
    +import org.apache.drill.exec.ops.FragmentContext;
    +import org.apache.drill.exec.physical.config.StatisticsAggregate;
    +import org.apache.drill.exec.physical.impl.aggregate.StreamingAggBatch;
    +import org.apache.drill.exec.physical.impl.aggregate.StreamingAggTemplate;
    +import org.apache.drill.exec.physical.impl.aggregate.StreamingAggregator;
    +import org.apache.drill.exec.planner.physical.StatsAggPrel.OperatorPhase;
    +import org.apache.drill.exec.record.BatchSchema.SelectionVectorMode;
    +import org.apache.drill.exec.record.MaterializedField;
    +import org.apache.drill.exec.record.RecordBatch;
    +import org.apache.drill.exec.record.TypedFieldId;
    +import org.apache.drill.exec.store.ImplicitColumnExplorer;
    +import org.apache.drill.exec.vector.ValueVector;
    +import org.apache.drill.exec.vector.complex.FieldIdUtil;
    +import org.apache.drill.exec.vector.complex.MapVector;
    +
    +import java.io.IOException;
    +import java.util.GregorianCalendar;
    +import java.util.List;
    +import java.util.TimeZone;
    +
    +/**
    + * TODO: This needs cleanup. Currently the key values are constants and we compare the constants
    + * for every record. Seems unnecessary.
    + *
    + * Example input and output:
    + * Schema of incoming batch: region_id (VARCHAR), sales_city (VARCHAR), cnt (BIGINT)
    + * Schema of output:
    + *    "schema" : BIGINT - Schema number. For each schema change this number is incremented.
    + *    "computed" : BIGINT - What time is it computed?
    + *    "columns"       : MAP - Column names
    + *       "region_id"  : VARCHAR
    + *       "sales_city" : VARCHAR
    + *       "cnt"        : VARCHAR
    + *    "statscount" : MAP
    + *       "region_id"  : BIGINT - statscount(region_id) - aggregation over all values of region_id
    + *                      in incoming batch
    + *       "sales_city" : BIGINT - statscount(sales_city)
    + *       "cnt"        : BIGINT - statscount(cnt)
    + *    "nonnullstatcount" : MAP
    + *       "region_id"  : BIGINT - nonnullstatcount(region_id)
    + *       "sales_city" : BIGINT - nonnullstatcount(sales_city)
    + *       "cnt"        : BIGINT - nonnullstatcount(cnt)
    + *   .... another map for next stats function ....
    + */
    +public class StatisticsAggBatch extends StreamingAggBatch {
    +  private List<String> functions;
    +  private int schema = 0;
    +
    +  public StatisticsAggBatch(StatisticsAggregate popConfig, RecordBatch incoming,
    +      FragmentContext context) throws OutOfMemoryException {
    +    super(popConfig, incoming, context);
    +    this.functions = popConfig.getFunctions();
    +  }
    +
    +  private void createKeyColumn(String name, LogicalExpression expr, List<LogicalExpression> keyExprs,
    +      List<TypedFieldId> keyOutputIds) throws SchemaChangeException {
    +    ErrorCollector collector = new ErrorCollectorImpl();
    +
    +    LogicalExpression mle = ExpressionTreeMaterializer.materialize(expr, incoming, collector,
    +        context.getFunctionRegistry());
    +
    +    MaterializedField outputField = MaterializedField.create(name, mle.getMajorType());
    +    ValueVector vector = TypeHelper.getNewVector(outputField, oContext.getAllocator());
    +
    +    keyExprs.add(mle);
    +    keyOutputIds.add(container.add(vector));
    +
    +    if (collector.hasErrors()) {
    +      throw new SchemaChangeException("Failure while materializing expression. "
    +          + collector.toErrorString());
    +    }
    +  }
    +
    +  private void createNestedKeyColumn(MapVector parent, String name, LogicalExpression expr,
    +      List<LogicalExpression> keyExprs, List<TypedFieldId> keyOutputIds)
    +          throws SchemaChangeException {
    +    ErrorCollector collector = new ErrorCollectorImpl();
    +
    +    LogicalExpression mle = ExpressionTreeMaterializer.materialize(expr, incoming, collector,
    +        context.getFunctionRegistry());
    +
    +    Class<? extends ValueVector> vvc =
    +        TypeHelper.getValueVectorClass(mle.getMajorType().getMinorType(),
    +            mle.getMajorType().getMode());
    +
    +    ValueVector vv = parent.addOrGet(name, mle.getMajorType(), vvc);
    +
    +    TypedFieldId pfid = container.getValueVectorId(SchemaPath.getSimplePath(parent.getField().getPath()));
    +    assert pfid.getFieldIds().length == 1;
    +    TypedFieldId.Builder builder = TypedFieldId.newBuilder();
    +    builder.addId(pfid.getFieldIds()[0]);
    +    TypedFieldId id =
    +        FieldIdUtil.getFieldIdIfMatches(parent, builder, true,
    +            SchemaPath.getSimplePath(vv.getField().getPath()).getRootSegment());
    +
    +    keyExprs.add(mle);
    +    keyOutputIds.add(id);
    +
    +    if (collector.hasErrors()) {
    +      throw new SchemaChangeException("Failure while materializing expression. "
    +          + collector.toErrorString());
    +    }
    +  }
    +
    +  private void addMapVector(String name, MapVector parent, LogicalExpression expr,
    +      List<LogicalExpression> valueExprs) throws SchemaChangeException {
    +    ErrorCollector collector = new ErrorCollectorImpl();
    +
    +    LogicalExpression mle = ExpressionTreeMaterializer.materialize(expr, incoming, collector,
    +        context.getFunctionRegistry());
    +
    +    Class<? extends ValueVector> vvc =
    +        TypeHelper.getValueVectorClass(mle.getMajorType().getMinorType(), mle.getMajorType().getMode());
    +    ValueVector vv = parent.addOrGet(name, mle.getMajorType(), vvc);
    +
    +    TypedFieldId pfid = container.getValueVectorId(SchemaPath.getSimplePath(parent.getField().getPath()));
    +    assert pfid.getFieldIds().length == 1;
    +    TypedFieldId.Builder builder = TypedFieldId.newBuilder();
    +    builder.addId(pfid.getFieldIds()[0]);
    +    TypedFieldId id = FieldIdUtil.getFieldIdIfMatches(parent, builder, true,
    +        SchemaPath.getSimplePath(vv.getField().getPath()).getRootSegment());
    +
    +    valueExprs.add(new ValueVectorWriteExpression(id, mle, true));
    +
    +    if (collector.hasErrors()) {
    +      throw new SchemaChangeException("Failure while materializing expression. "
    +          + collector.toErrorString());
    +    }
    +  }
    +
    +  private StreamingAggregator codegenAggregator(List<LogicalExpression> keyExprs,
    +      List<LogicalExpression> valueExprs, List<TypedFieldId> keyOutputIds)
    +          throws SchemaChangeException, ClassTransformationException, IOException {
    +    ClassGenerator<StreamingAggregator> cg =
    +        CodeGenerator.getRoot(StreamingAggTemplate.TEMPLATE_DEFINITION, context.getFunctionRegistry(),
    +            context.getOptions());
    +
    +    LogicalExpression[] keyExprsArray = new LogicalExpression[keyExprs.size()];
    +    LogicalExpression[] valueExprsArray = new LogicalExpression[valueExprs.size()];
    +    TypedFieldId[] keyOutputIdsArray = new TypedFieldId[keyOutputIds.size()];
    +
    +    keyExprs.toArray(keyExprsArray);
    +    valueExprs.toArray(valueExprsArray);
    +    keyOutputIds.toArray(keyOutputIdsArray);
    +
    +    setupIsSame(cg, keyExprsArray);
    +    setupIsSameApart(cg, keyExprsArray);
    +    addRecordValues(cg, valueExprsArray);
    +    outputRecordKeys(cg, keyOutputIdsArray, keyExprsArray);
    +    outputRecordKeysPrev(cg, keyOutputIdsArray, keyExprsArray);
    +
    +    cg.getBlock("resetValues")._return(JExpr.TRUE);
    +    getIndex(cg);
    +
    +    container.buildSchema(SelectionVectorMode.NONE);
    +    StreamingAggregator agg = context.getImplementationClass(cg);
    +    agg.setup(oContext, incoming, this);
    +    return agg;
    +  }
    +
    +  private boolean isImplicitFileColumn(MaterializedField mf) {
    +    return ImplicitColumnExplorer.initImplicitFileColumns(context.getOptions()).get(mf.getName()) != null;
    +  }
    +
    +  protected StreamingAggregator createAggregatorInternal()
    +      throws SchemaChangeException, ClassTransformationException, IOException {
    +    container.clear();
    +
    +    List<LogicalExpression> keyExprs = Lists.newArrayList();
    +    List<LogicalExpression> valueExprs = Lists.newArrayList();
    +    List<TypedFieldId> keyOutputIds = Lists.newArrayList();
    +    GregorianCalendar calendar = new GregorianCalendar(TimeZone.getTimeZone("UTC"));
    +    calendar.setTimeInMillis(System.currentTimeMillis());
    +
    +    if (this.getPopConfig() instanceof StatisticsAggregate
    +        && (((StatisticsAggregate) this.getPopConfig()).getPhase() == OperatorPhase.PHASE_1of1
    +            || ((StatisticsAggregate) this.getPopConfig()).getPhase() == OperatorPhase.PHASE_2of2)) {
    +      createKeyColumn("schema",
    +          ValueExpressions.getBigInt(schema++),
    +          keyExprs,
    +          keyOutputIds
    +      );
    +      createKeyColumn("computed",
    +          ValueExpressions.getDate(calendar),
    +          keyExprs,
    +          keyOutputIds
    +      );
    +    }
    +
    --- End diff --
    
    Each code block is a bit complex. Maybe move each block to a function for greater clarity?


---
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 #729: Drill 1328: Support table statistics for Parquet

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

    https://github.com/apache/drill/pull/729#discussion_r100455272
  
    --- Diff: exec/java-exec/src/main/java/org/apache/drill/exec/ops/QueryContext.java ---
    @@ -283,4 +286,18 @@ public void close() throws Exception {
           closed = true;
         }
       }
    +
    +  /**
    +  * @param stmtType : The type {@link StatementType} e.g. CTAS, ANALYZE of the statement
    +  */
    +  public void setStatementType(StatementType stmtType) {
    --- End diff --
    
    I have changed the code to allow the statement type to be set only once. Please suggest if there is a better approach.


---
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 #729: Drill 1328 r4

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

    https://github.com/apache/drill/pull/729#discussion_r97910777
  
    --- Diff: exec/java-exec/src/main/java/org/apache/drill/exec/planner/cost/DrillRelMdSelectivity.java ---
    @@ -0,0 +1,219 @@
    +/*******************************************************************************
    + * 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.planner.cost;
    +
    +import org.apache.calcite.plan.RelOptUtil;
    +import org.apache.calcite.plan.volcano.RelSubset;
    +import org.apache.calcite.rel.RelNode;
    +import org.apache.calcite.rel.SingleRel;
    +import org.apache.calcite.rel.core.JoinRelType;
    +import org.apache.calcite.rel.core.TableScan;
    +import org.apache.calcite.rel.metadata.ReflectiveRelMetadataProvider;
    +import org.apache.calcite.rel.metadata.RelMdSelectivity;
    +import org.apache.calcite.rel.metadata.RelMdUtil;
    +import org.apache.calcite.rel.metadata.RelMetadataProvider;
    +import org.apache.calcite.rel.metadata.RelMetadataQuery;
    +import org.apache.calcite.rel.type.RelDataType;
    +import org.apache.calcite.rex.RexBuilder;
    +import org.apache.calcite.rex.RexCall;
    +import org.apache.calcite.rex.RexInputRef;
    +import org.apache.calcite.rex.RexNode;
    +import org.apache.calcite.rex.RexUtil;
    +import org.apache.calcite.rex.RexVisitor;
    +import org.apache.calcite.rex.RexVisitorImpl;
    +import org.apache.calcite.sql.SqlKind;
    +import org.apache.calcite.util.BuiltInMethod;
    +import org.apache.calcite.util.Util;
    +import org.apache.drill.exec.planner.common.DrillJoinRelBase;
    +import org.apache.drill.exec.planner.common.DrillRelOptUtil;
    +import org.apache.drill.exec.planner.logical.DrillTable;
    +import org.apache.drill.exec.planner.logical.DrillTranslatableTable;
    +
    +import java.util.ArrayList;
    +import java.util.List;
    +
    +public class DrillRelMdSelectivity extends RelMdSelectivity {
    +  private static final org.slf4j.Logger logger =
    +      org.slf4j.LoggerFactory.getLogger(RelMdSelectivity.class);
    +
    +  private static final DrillRelMdSelectivity INSTANCE =
    +      new DrillRelMdSelectivity();
    +
    +  public static final RelMetadataProvider SOURCE =
    +      ReflectiveRelMetadataProvider.reflectiveSource(
    +          BuiltInMethod.SELECTIVITY.method, INSTANCE);
    +
    +  @Override
    +  public Double getSelectivity(RelNode rel, RexNode predicate) {
    +    if (rel instanceof TableScan) {
    +      return getScanSelectivity((TableScan) rel, predicate);
    +    } else if (rel instanceof DrillJoinRelBase) {
    +      return getJoinSelectivity(((DrillJoinRelBase) rel), predicate);
    +    } else if (rel instanceof SingleRel && !DrillRelOptUtil.guessRows(rel)) {
    +        return RelMetadataQuery.getSelectivity(((SingleRel) rel).getInput(), predicate);
    +    } else if (rel instanceof RelSubset && !DrillRelOptUtil.guessRows(rel)) {
    +      if (((RelSubset) rel).getBest() != null) {
    +        return RelMetadataQuery.getSelectivity(((RelSubset)rel).getBest(), predicate);
    +      } else if (((RelSubset)rel).getOriginal() != null) {
    +        return RelMetadataQuery.getSelectivity(((RelSubset)rel).getOriginal(), predicate);
    +      } else {
    +        return super.getSelectivity(rel, predicate);
    +      }
    +    } else {
    +      return super.getSelectivity(rel, predicate);
    +    }
    +  }
    +
    +  private Double getJoinSelectivity(DrillJoinRelBase rel, RexNode predicate) {
    --- End diff --
    
    Let's explore whether this can be done in Calcite instead of Drill since it is a generic function, not Drill specific. 


---
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 #729: Drill 1328: Support table statistics for Parquet

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/729#discussion_r100676700
  
    --- Diff: exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/aggregate/InternalBatch.java ---
    @@ -54,6 +56,7 @@ public InternalBatch(RecordBatch incoming, VectorWrapper[] ignoreWrappers, Opera
           this.sv2 = null;
         }
         this.schema = incoming.getSchema();
    +    this.incoming = incoming;
    --- End diff --
    
    Are we sure we want to do this? It seems the semantics are that this is a, well, "internal batch" a batch used to hold temporary results. By adding the incoming we are binding this batch to the output of another operator. That seems to defeat the purpose of an internal batch.


---
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 #729: Drill 1328: Support table statistics for Parquet

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/729#discussion_r100677712
  
    --- Diff: exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/statistics/StatisticsAggBatch.java ---
    @@ -0,0 +1,256 @@
    +/**
    + * 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.impl.statistics;
    +
    +import com.google.common.collect.Lists;
    +import com.sun.codemodel.JExpr;
    +import org.apache.drill.common.expression.ErrorCollector;
    +import org.apache.drill.common.expression.ErrorCollectorImpl;
    +import org.apache.drill.common.expression.FunctionCallFactory;
    +import org.apache.drill.common.expression.LogicalExpression;
    +import org.apache.drill.common.expression.SchemaPath;
    +import org.apache.drill.common.expression.ValueExpressions;
    +import org.apache.drill.exec.exception.ClassTransformationException;
    +import org.apache.drill.exec.exception.OutOfMemoryException;
    +import org.apache.drill.exec.exception.SchemaChangeException;
    +import org.apache.drill.exec.expr.ClassGenerator;
    +import org.apache.drill.exec.expr.CodeGenerator;
    +import org.apache.drill.exec.expr.ExpressionTreeMaterializer;
    +import org.apache.drill.exec.expr.TypeHelper;
    +import org.apache.drill.exec.expr.ValueVectorWriteExpression;
    +import org.apache.drill.exec.ops.FragmentContext;
    +import org.apache.drill.exec.physical.config.StatisticsAggregate;
    +import org.apache.drill.exec.physical.impl.aggregate.StreamingAggBatch;
    +import org.apache.drill.exec.physical.impl.aggregate.StreamingAggTemplate;
    +import org.apache.drill.exec.physical.impl.aggregate.StreamingAggregator;
    +import org.apache.drill.exec.planner.physical.StatsAggPrel.OperatorPhase;
    +import org.apache.drill.exec.record.BatchSchema.SelectionVectorMode;
    +import org.apache.drill.exec.record.MaterializedField;
    +import org.apache.drill.exec.record.RecordBatch;
    +import org.apache.drill.exec.record.TypedFieldId;
    +import org.apache.drill.exec.store.ImplicitColumnExplorer;
    +import org.apache.drill.exec.vector.ValueVector;
    +import org.apache.drill.exec.vector.complex.FieldIdUtil;
    +import org.apache.drill.exec.vector.complex.MapVector;
    +
    +import java.io.IOException;
    +import java.util.GregorianCalendar;
    +import java.util.List;
    +import java.util.TimeZone;
    +
    +/**
    + * TODO: This needs cleanup. Currently the key values are constants and we compare the constants
    + * for every record. Seems unnecessary.
    + *
    + * Example input and output:
    + * Schema of incoming batch: region_id (VARCHAR), sales_city (VARCHAR), cnt (BIGINT)
    + * Schema of output:
    + *    "schema" : BIGINT - Schema number. For each schema change this number is incremented.
    + *    "computed" : BIGINT - What time is it computed?
    + *    "columns"       : MAP - Column names
    + *       "region_id"  : VARCHAR
    + *       "sales_city" : VARCHAR
    + *       "cnt"        : VARCHAR
    + *    "statscount" : MAP
    + *       "region_id"  : BIGINT - statscount(region_id) - aggregation over all values of region_id
    + *                      in incoming batch
    + *       "sales_city" : BIGINT - statscount(sales_city)
    + *       "cnt"        : BIGINT - statscount(cnt)
    + *    "nonnullstatcount" : MAP
    + *       "region_id"  : BIGINT - nonnullstatcount(region_id)
    + *       "sales_city" : BIGINT - nonnullstatcount(sales_city)
    + *       "cnt"        : BIGINT - nonnullstatcount(cnt)
    + *   .... another map for next stats function ....
    + */
    +public class StatisticsAggBatch extends StreamingAggBatch {
    +  private List<String> functions;
    +  private int schema = 0;
    +
    +  public StatisticsAggBatch(StatisticsAggregate popConfig, RecordBatch incoming,
    +      FragmentContext context) throws OutOfMemoryException {
    +    super(popConfig, incoming, context);
    +    this.functions = popConfig.getFunctions();
    +  }
    +
    +  private void createKeyColumn(String name, LogicalExpression expr, List<LogicalExpression> keyExprs,
    +      List<TypedFieldId> keyOutputIds) throws SchemaChangeException {
    +    ErrorCollector collector = new ErrorCollectorImpl();
    +
    +    LogicalExpression mle = ExpressionTreeMaterializer.materialize(expr, incoming, collector,
    +        context.getFunctionRegistry());
    +
    +    MaterializedField outputField = MaterializedField.create(name, mle.getMajorType());
    +    ValueVector vector = TypeHelper.getNewVector(outputField, oContext.getAllocator());
    +
    +    keyExprs.add(mle);
    +    keyOutputIds.add(container.add(vector));
    +
    +    if (collector.hasErrors()) {
    +      throw new SchemaChangeException("Failure while materializing expression. "
    +          + collector.toErrorString());
    +    }
    +  }
    +
    +  private void createNestedKeyColumn(MapVector parent, String name, LogicalExpression expr,
    +      List<LogicalExpression> keyExprs, List<TypedFieldId> keyOutputIds)
    +          throws SchemaChangeException {
    +    ErrorCollector collector = new ErrorCollectorImpl();
    +
    +    LogicalExpression mle = ExpressionTreeMaterializer.materialize(expr, incoming, collector,
    +        context.getFunctionRegistry());
    +
    +    Class<? extends ValueVector> vvc =
    +        TypeHelper.getValueVectorClass(mle.getMajorType().getMinorType(),
    +            mle.getMajorType().getMode());
    +
    +    ValueVector vv = parent.addOrGet(name, mle.getMajorType(), vvc);
    +
    +    TypedFieldId pfid = container.getValueVectorId(SchemaPath.getSimplePath(parent.getField().getPath()));
    +    assert pfid.getFieldIds().length == 1;
    +    TypedFieldId.Builder builder = TypedFieldId.newBuilder();
    +    builder.addId(pfid.getFieldIds()[0]);
    +    TypedFieldId id =
    +        FieldIdUtil.getFieldIdIfMatches(parent, builder, true,
    +            SchemaPath.getSimplePath(vv.getField().getPath()).getRootSegment());
    +
    +    keyExprs.add(mle);
    +    keyOutputIds.add(id);
    +
    +    if (collector.hasErrors()) {
    +      throw new SchemaChangeException("Failure while materializing expression. "
    +          + collector.toErrorString());
    +    }
    +  }
    +
    +  private void addMapVector(String name, MapVector parent, LogicalExpression expr,
    +      List<LogicalExpression> valueExprs) throws SchemaChangeException {
    +    ErrorCollector collector = new ErrorCollectorImpl();
    +
    +    LogicalExpression mle = ExpressionTreeMaterializer.materialize(expr, incoming, collector,
    +        context.getFunctionRegistry());
    +
    +    Class<? extends ValueVector> vvc =
    +        TypeHelper.getValueVectorClass(mle.getMajorType().getMinorType(), mle.getMajorType().getMode());
    +    ValueVector vv = parent.addOrGet(name, mle.getMajorType(), vvc);
    +
    +    TypedFieldId pfid = container.getValueVectorId(SchemaPath.getSimplePath(parent.getField().getPath()));
    +    assert pfid.getFieldIds().length == 1;
    +    TypedFieldId.Builder builder = TypedFieldId.newBuilder();
    +    builder.addId(pfid.getFieldIds()[0]);
    +    TypedFieldId id = FieldIdUtil.getFieldIdIfMatches(parent, builder, true,
    +        SchemaPath.getSimplePath(vv.getField().getPath()).getRootSegment());
    +
    +    valueExprs.add(new ValueVectorWriteExpression(id, mle, true));
    +
    +    if (collector.hasErrors()) {
    +      throw new SchemaChangeException("Failure while materializing expression. "
    +          + collector.toErrorString());
    +    }
    +  }
    +
    +  private StreamingAggregator codegenAggregator(List<LogicalExpression> keyExprs,
    +      List<LogicalExpression> valueExprs, List<TypedFieldId> keyOutputIds)
    +          throws SchemaChangeException, ClassTransformationException, IOException {
    +    ClassGenerator<StreamingAggregator> cg =
    +        CodeGenerator.getRoot(StreamingAggTemplate.TEMPLATE_DEFINITION, context.getFunctionRegistry(),
    +            context.getOptions());
    +
    +    LogicalExpression[] keyExprsArray = new LogicalExpression[keyExprs.size()];
    +    LogicalExpression[] valueExprsArray = new LogicalExpression[valueExprs.size()];
    +    TypedFieldId[] keyOutputIdsArray = new TypedFieldId[keyOutputIds.size()];
    +
    +    keyExprs.toArray(keyExprsArray);
    +    valueExprs.toArray(valueExprsArray);
    +    keyOutputIds.toArray(keyOutputIdsArray);
    +
    +    setupIsSame(cg, keyExprsArray);
    +    setupIsSameApart(cg, keyExprsArray);
    +    addRecordValues(cg, valueExprsArray);
    +    outputRecordKeys(cg, keyOutputIdsArray, keyExprsArray);
    +    outputRecordKeysPrev(cg, keyOutputIdsArray, keyExprsArray);
    +
    +    cg.getBlock("resetValues")._return(JExpr.TRUE);
    +    getIndex(cg);
    +
    +    container.buildSchema(SelectionVectorMode.NONE);
    +    StreamingAggregator agg = context.getImplementationClass(cg);
    +    agg.setup(oContext, incoming, this);
    +    return agg;
    +  }
    +
    +  private boolean isImplicitFileColumn(MaterializedField mf) {
    +    return ImplicitColumnExplorer.initImplicitFileColumns(context.getOptions()).get(mf.getName()) != null;
    +  }
    +
    +  protected StreamingAggregator createAggregatorInternal()
    +      throws SchemaChangeException, ClassTransformationException, IOException {
    +    container.clear();
    +
    +    List<LogicalExpression> keyExprs = Lists.newArrayList();
    +    List<LogicalExpression> valueExprs = Lists.newArrayList();
    +    List<TypedFieldId> keyOutputIds = Lists.newArrayList();
    +    GregorianCalendar calendar = new GregorianCalendar(TimeZone.getTimeZone("UTC"));
    +    calendar.setTimeInMillis(System.currentTimeMillis());
    +
    +    if (this.getPopConfig() instanceof StatisticsAggregate
    +        && (((StatisticsAggregate) this.getPopConfig()).getPhase() == OperatorPhase.PHASE_1of1
    +            || ((StatisticsAggregate) this.getPopConfig()).getPhase() == OperatorPhase.PHASE_2of2)) {
    +      createKeyColumn("schema",
    +          ValueExpressions.getBigInt(schema++),
    +          keyExprs,
    +          keyOutputIds
    +      );
    +      createKeyColumn("computed",
    +          ValueExpressions.getDate(calendar),
    +          keyExprs,
    +          keyOutputIds
    +      );
    +    }
    +
    +    MapVector cparent = new MapVector("column", oContext.getAllocator(), null);
    +    container.add(cparent);
    +    for (MaterializedField mf : incoming.getSchema()) {
    +      // Ignore implicit columns
    +      if (!isImplicitFileColumn(mf)) {
    +        createNestedKeyColumn(
    +            cparent,
    +            mf.getLastName(),
    --- End diff --
    
    Is this a bug? Suppose my incoming is JSON:
    
    ```
    { a: { x: "foo", y: "bar" }, b: { x: 10, y: 20 } }
    ```
    
    (Indeed, this is the very structure that this operator creates!)
    
    If we use only the last name, then under our "column" map we try to create two "x"s and two "y"s. Does the last one win? Does that cause a problem later as we try to store values from two columns into a single summary column?


---
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 #729: Drill 1328: Support table statistics for Parquet

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/729#discussion_r100680828
  
    --- Diff: exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/statistics/StatisticsAggBatch.java ---
    @@ -0,0 +1,256 @@
    +/**
    + * 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.impl.statistics;
    +
    +import com.google.common.collect.Lists;
    +import com.sun.codemodel.JExpr;
    +import org.apache.drill.common.expression.ErrorCollector;
    +import org.apache.drill.common.expression.ErrorCollectorImpl;
    +import org.apache.drill.common.expression.FunctionCallFactory;
    +import org.apache.drill.common.expression.LogicalExpression;
    +import org.apache.drill.common.expression.SchemaPath;
    +import org.apache.drill.common.expression.ValueExpressions;
    +import org.apache.drill.exec.exception.ClassTransformationException;
    +import org.apache.drill.exec.exception.OutOfMemoryException;
    +import org.apache.drill.exec.exception.SchemaChangeException;
    +import org.apache.drill.exec.expr.ClassGenerator;
    +import org.apache.drill.exec.expr.CodeGenerator;
    +import org.apache.drill.exec.expr.ExpressionTreeMaterializer;
    +import org.apache.drill.exec.expr.TypeHelper;
    +import org.apache.drill.exec.expr.ValueVectorWriteExpression;
    +import org.apache.drill.exec.ops.FragmentContext;
    +import org.apache.drill.exec.physical.config.StatisticsAggregate;
    +import org.apache.drill.exec.physical.impl.aggregate.StreamingAggBatch;
    +import org.apache.drill.exec.physical.impl.aggregate.StreamingAggTemplate;
    +import org.apache.drill.exec.physical.impl.aggregate.StreamingAggregator;
    +import org.apache.drill.exec.planner.physical.StatsAggPrel.OperatorPhase;
    +import org.apache.drill.exec.record.BatchSchema.SelectionVectorMode;
    +import org.apache.drill.exec.record.MaterializedField;
    +import org.apache.drill.exec.record.RecordBatch;
    +import org.apache.drill.exec.record.TypedFieldId;
    +import org.apache.drill.exec.store.ImplicitColumnExplorer;
    +import org.apache.drill.exec.vector.ValueVector;
    +import org.apache.drill.exec.vector.complex.FieldIdUtil;
    +import org.apache.drill.exec.vector.complex.MapVector;
    +
    +import java.io.IOException;
    +import java.util.GregorianCalendar;
    +import java.util.List;
    +import java.util.TimeZone;
    +
    +/**
    + * TODO: This needs cleanup. Currently the key values are constants and we compare the constants
    + * for every record. Seems unnecessary.
    + *
    + * Example input and output:
    + * Schema of incoming batch: region_id (VARCHAR), sales_city (VARCHAR), cnt (BIGINT)
    + * Schema of output:
    + *    "schema" : BIGINT - Schema number. For each schema change this number is incremented.
    + *    "computed" : BIGINT - What time is it computed?
    + *    "columns"       : MAP - Column names
    + *       "region_id"  : VARCHAR
    + *       "sales_city" : VARCHAR
    + *       "cnt"        : VARCHAR
    + *    "statscount" : MAP
    + *       "region_id"  : BIGINT - statscount(region_id) - aggregation over all values of region_id
    + *                      in incoming batch
    + *       "sales_city" : BIGINT - statscount(sales_city)
    + *       "cnt"        : BIGINT - statscount(cnt)
    + *    "nonnullstatcount" : MAP
    + *       "region_id"  : BIGINT - nonnullstatcount(region_id)
    + *       "sales_city" : BIGINT - nonnullstatcount(sales_city)
    + *       "cnt"        : BIGINT - nonnullstatcount(cnt)
    + *   .... another map for next stats function ....
    --- End diff --
    
    Note that Drill is columnar; there is overhead in allocating a vector. It seems here we are creating very wide records that are not very deep. This means we create 3, 4 or more value vectors in the stats record for every column in the original data. And, we store just one record in each?
    
    This will likely be tremendously inefficient. Vectors have very large amount of overhead. They can also be memory intensive when allocated incorrectly (pre-allocating a vector of a given size.)
    
    Would it be better to implement the record as a set of a very few lists (repeated columns.)
    
    * schema: Integer
    * timestamp: BigInt
    * column_names: Repeated Varchar (index of name is important)
    * fn1: Repeated BigInt (index of value matches index of column name)
    * fn2: ...
    
    The above makes far more efficient use of vector memory than do the large maps.
    
    Plus, unlike the maps, the above can easily handle nested fields. "a.b.c" is a fine VarChar value in a list, it does not work as a key in a map (since the dots are interpreted as creating nested maps.)



---
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 #729: Drill 1328: Support table statistics for Parquet

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

    https://github.com/apache/drill/pull/729#discussion_r102320614
  
    --- Diff: exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/statistics/StatisticsMergeBatch.java ---
    @@ -0,0 +1,532 @@
    +/**
    + * 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.impl.statistics;
    +
    +import com.google.common.collect.Lists;
    +import com.google.common.collect.Maps;
    +import org.apache.drill.common.expression.LogicalExpression;
    +import org.apache.drill.common.expression.ErrorCollector;
    +import org.apache.drill.common.expression.ErrorCollectorImpl;
    +import org.apache.drill.common.expression.FunctionCallFactory;
    +import org.apache.drill.common.expression.SchemaPath;
    +import org.apache.drill.common.expression.ValueExpressions;
    +import org.apache.drill.common.types.TypeProtos;
    +import org.apache.drill.common.types.TypeProtos.MajorType;
    +import org.apache.drill.exec.exception.OutOfMemoryException;
    +import org.apache.drill.exec.exception.SchemaChangeException;
    +import org.apache.drill.exec.expr.ExpressionTreeMaterializer;
    +import org.apache.drill.exec.expr.TypeHelper;
    +import org.apache.drill.exec.expr.holders.BigIntHolder;
    +import org.apache.drill.exec.expr.holders.NullableFloat8Holder;
    +import org.apache.drill.exec.expr.holders.ObjectHolder;
    +import org.apache.drill.exec.expr.holders.ValueHolder;
    +import org.apache.drill.exec.ops.FragmentContext;
    +import org.apache.drill.exec.physical.config.StatisticsMerge;
    +import org.apache.drill.exec.record.AbstractSingleRecordBatch;
    +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.VectorContainer;
    +import org.apache.drill.exec.record.VectorWrapper;
    +import org.apache.drill.exec.vector.ValueVector;
    +import org.apache.drill.exec.vector.DateVector;
    +import org.apache.drill.exec.vector.BigIntVector;
    +import org.apache.drill.exec.vector.NullableBigIntVector;
    +import org.apache.drill.exec.vector.NullableFloat8Vector;
    +import org.apache.drill.exec.vector.NullableVarBinaryVector;
    +import org.apache.drill.exec.vector.VarCharVector;
    +import org.apache.drill.exec.vector.complex.MapVector;
    +
    +import java.io.ByteArrayInputStream;
    +import java.io.DataInputStream;
    +import java.io.IOException;
    +import java.util.List;
    +import java.util.GregorianCalendar;
    +import java.util.HashMap;
    +import java.util.Map;
    +import java.util.TimeZone;
    +
    +import com.clearspring.analytics.stream.cardinality.HyperLogLog;
    +
    +public class StatisticsMergeBatch extends AbstractSingleRecordBatch<StatisticsMerge> {
    +  private static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(StatisticsMergeBatch.class);
    +  private Map<String, String> functions;
    +  private boolean first = true;
    +  private boolean finished = false;
    +  private int schema = 0;
    +  private int recordCount = 0;
    +  private List<String> keyList = null;
    +  private Map<MaterializedField, ValueVector> dataSrcVecMap = null;
    +  // Map of non-map fields to VV in the incoming schema
    +  private Map<MaterializedField, ValueVector> copySrcVecMap = null;
    +  private Map<String, Map<String, ValueHolder>> aggregationMap = null;
    +  public StatisticsMergeBatch(StatisticsMerge popConfig, RecordBatch incoming,
    +                              FragmentContext context) throws OutOfMemoryException {
    +    super(popConfig, context, incoming);
    +    this.functions = new HashMap<>();
    +    this.aggregationMap = new HashMap<>();
    +
    +    /*for (String key : popConfig.getFunctions()) {
    +      aggregationMap.put(key, new HashMap<String, ValueHolder>());
    +      if (key.equalsIgnoreCase("statcount") || key.equalsIgnoreCase("nonnullstatcount")) {
    +        functions.put(key, "sum");
    +      } else if (key.equalsIgnoreCase("hll")) {
    +        functions.put(key, "hll_merge");
    +      } else if (key.equalsIgnoreCase("sum_width")) {
    +        functions.put(key, "avg_width");
    +      }
    +    }*/
    +    for (String key : popConfig.getFunctions()) {
    +      if (key.equalsIgnoreCase("sum_width")) {
    +        functions.put(key, "avg_width");
    +      } else if (key.equalsIgnoreCase("hll")) {
    +        functions.put(key, "hll_merge");
    +      } else {
    +        functions.put(key, key);
    +      }
    +      aggregationMap.put(functions.get(key), new HashMap<String, ValueHolder>());
    +    }
    +  }
    +
    +  private void createKeyColumn(String name, LogicalExpression expr, Map<MaterializedField, ValueVector> parentMap)
    +      throws SchemaChangeException {
    +    ErrorCollector collector = new ErrorCollectorImpl();
    +
    +    LogicalExpression mle = ExpressionTreeMaterializer.materialize(expr, incoming, collector,
    +        context.getFunctionRegistry());
    +
    +    MaterializedField outputField = MaterializedField.create(name, mle.getMajorType());
    +    ValueVector vector = TypeHelper.getNewVector(outputField, oContext.getAllocator());
    +    container.add(vector);
    +
    +    if (collector.hasErrors()) {
    +      throw new SchemaChangeException("Failure while materializing expression. "
    +          + collector.toErrorString());
    +    }
    +    parentMap.put(outputField, vector);
    +  }
    +
    +  private ValueVector addMapVector(String name, MapVector parent, LogicalExpression expr) throws SchemaChangeException {
    +    ErrorCollector collector = new ErrorCollectorImpl();
    +
    +    LogicalExpression mle = ExpressionTreeMaterializer.materialize(expr, incoming, collector,
    +        context.getFunctionRegistry());
    +
    +    Class<? extends ValueVector> vvc =
    +        TypeHelper.getValueVectorClass(mle.getMajorType().getMinorType(), mle.getMajorType().getMode());
    +    ValueVector vector = parent.addOrGet(name, mle.getMajorType(), vvc);
    +
    +    if (collector.hasErrors()) {
    +      throw new SchemaChangeException("Failure while materializing expression. "
    +          + collector.toErrorString());
    +    }
    +    return vector;
    +  }
    +
    +  /**
    +   * Identify the list of fields within a map which are unpivoted as columns in output
    +   */
    +  private void buildKeyList() {
    +    List<String> lastMapKeyList = null;
    +    for (VectorWrapper<?> vw : incoming) {
    +      if (vw.getField().getType().getMinorType() != TypeProtos.MinorType.MAP) {
    +        continue;
    +      }
    +
    +      keyList = Lists.newArrayList();
    +
    +      for (ValueVector vv : vw.getValueVector()) {
    +        keyList.add(vv.getField().getLastName());
    +      }
    +
    +      if (lastMapKeyList == null) {
    +        lastMapKeyList = keyList;
    +      } else {
    +        if (keyList.size() != lastMapKeyList.size() || !lastMapKeyList.containsAll(keyList)) {
    +          throw new UnsupportedOperationException("Maps have different fields");
    +        }
    +      }
    +    }
    +  }
    +
    +  private void buildOutputContainer() throws SchemaChangeException {
    +    dataSrcVecMap = Maps.newHashMap();
    +    copySrcVecMap = Maps.newHashMap();
    +    MajorType mt = null;
    +
    +    ErrorCollector collector = new ErrorCollectorImpl();
    +    GregorianCalendar calendar = new GregorianCalendar(TimeZone.getTimeZone("UTC"));
    +
    +    calendar.setTimeInMillis(System.currentTimeMillis());
    +    createKeyColumn("schema", ValueExpressions.getBigInt(schema++), copySrcVecMap);
    +    createKeyColumn("computed", ValueExpressions.getDate(calendar), copySrcVecMap);
    +
    +    for (VectorWrapper<?> vw : incoming) {
    +      addVectorToOutgoingContainer(vw.getField().getLastName(), vw, collector);
    +      /*MaterializedField ds = vw.getField();
    +      String field = vw.getField().getLastName();
    +      // Input map vector
    +      MapVector mapVector = (MapVector) vw.getValueVector();
    +      assert mapVector.getPrimitiveVectors().size() > 0;
    +      // Proceed to create output map vector with same name e.g. statcount etc.
    +      mt = mapVector.getField().getType();
    +      MaterializedField mf = MaterializedField.create(functions.get(field), mt);
    +      assert !dataSrcVecMap.containsKey(mf);
    +      ValueVector vector = TypeHelper.getNewVector(mf, oContext.getAllocator());
    +      container.add(vector);
    +      MapVector outputMapVector = (MapVector) vector;
    +
    +      for (ValueVector vv : mapVector) {
    +        String fieldName = vv.getField().getLastName();
    +        if (!keyList.contains(fieldName)) {
    +          throw new UnsupportedOperationException("Unpivot data vector " +
    +              ds + " contains key " + fieldName + " not contained in key source!");
    +        }
    +        if (vv.getField().getType().getMinorType() == TypeProtos.MinorType.MAP) {
    +          throw new UnsupportedOperationException("Unpivot of nested map is not supported!");
    +        }
    +        if (field.equals("column")) {
    +          outputMapVector.addOrGet(fieldName, vv.getField().getType(), vv.getClass());
    +        } else {
    +          List<LogicalExpression> args = Lists.newArrayList();
    +          //TODO: Something else to access value of col such as emp_id?
    +          args.add(SchemaPath.getSimplePath(vv.getField().getPath()));
    +          //TODO: Put in the mapVector
    +          LogicalExpression call = FunctionCallFactory.createExpression(functions.get(field), args);
    +          //TODO: Is this sufficient to add to new Map?
    +          ValueVector vector1 = addMapVector(fieldName, outputMapVector, call);
    +          if (collector.hasErrors()) {
    +            throw new SchemaChangeException("Failure while materializing expression. "
    +                + collector.toErrorString());
    +          }
    +        }
    +      }
    +      dataSrcVecMap.put(ds, outputMapVector);*/
    +    }
    +    //Now create NDV in the outgoing container which was not avaliable in the incoming
    +    for (VectorWrapper<?> vw : incoming) {
    +      if (vw.getField().getLastName().equalsIgnoreCase("sum_width")) {//NullableFloat8 type
    +        addVectorToOutgoingContainer("ndv", vw, collector);
    +        break;
    +      }
    +    }
    +    container.setRecordCount(0);
    +    recordCount = 0;
    +    container.buildSchema(incoming.getSchema().getSelectionVectorMode());
    +  }
    +
    +  private void addVectorToOutgoingContainer(String field, VectorWrapper vw, ErrorCollector collector)
    +     throws SchemaChangeException {
    +    // Input map vector
    +    MapVector mapVector = (MapVector) vw.getValueVector();
    +    MaterializedField mf;
    +    assert mapVector.getPrimitiveVectors().size() > 0;
    +    // Proceed to create output map vector with same name e.g. statcount etc.
    +    MajorType mt = mapVector.getField().getType();
    +    if (functions.get(field) != null) {
    +      mf = MaterializedField.create(functions.get(field), mt);
    +    } else {
    +      mf = MaterializedField.create(field, mt);
    +    }
    +    assert !dataSrcVecMap.containsKey(mf);
    +    ValueVector vector = TypeHelper.getNewVector(mf, oContext.getAllocator());
    +    container.add(vector);
    +    MapVector outputMapVector = (MapVector) vector;
    +
    +    for (ValueVector vv : mapVector) {
    +      String fieldName = vv.getField().getLastName();
    +      if (!keyList.contains(fieldName)) {
    +        throw new UnsupportedOperationException("Unpivot data vector " +
    +                field + " contains key " + fieldName + " not contained in key source!");
    +      }
    +      if (vv.getField().getType().getMinorType() == TypeProtos.MinorType.MAP) {
    +        throw new UnsupportedOperationException("Unpivot of nested map is not supported!");
    +      }
    +      if (field.equals("column")) {
    +        outputMapVector.addOrGet(fieldName, vv.getField().getType(), vv.getClass());
    +      } else {
    +        List<LogicalExpression> args = Lists.newArrayList();
    +        LogicalExpression call;
    +        //TODO: Something else to access value of col such as emp_id?
    +        args.add(SchemaPath.getSimplePath(vv.getField().getPath()));
    +        //TODO: Put in the mapVector
    +        if (functions.get(field) != null) {
    +          call = FunctionCallFactory.createExpression(functions.get(field), args);
    +        } else {
    +          call = FunctionCallFactory.createExpression(field, args);
    +        }
    +        //TODO: Is this sufficient to add to new Map?
    +        ValueVector vector1 = addMapVector(fieldName, outputMapVector, call);
    +        if (collector.hasErrors()) {
    +          throw new SchemaChangeException("Failure while materializing expression. "
    +                  + collector.toErrorString());
    +        }
    +      }
    +    }
    +  }
    +
    +  @Override
    +  protected boolean setupNewSchema() throws SchemaChangeException {
    +    container.clear();
    +    buildKeyList();
    +    buildOutputContainer();
    +    return true;
    +  }
    +
    +  @Override
    +  protected IterOutcome doWork() {
    +    int outRecordCount = incoming.getRecordCount();
    +    HashMap<String, Long> nonNullRows = new HashMap<>();
    +
    +    for (VectorWrapper<?> vw : incoming) {
    +      ValueVector vv = vw.getValueVector();
    +      if (vv.getField().getType().getMinorType() != TypeProtos.MinorType.MAP) {
    +        //We only expect Maps in the incoming. STOP, if this is not the case.
    +        return IterOutcome.STOP;
    +      }
    +      MapVector mapVec = (MapVector) vv;
    +      Map<String, ValueHolder> statMap = aggregationMap.get(vv.getField().getLastName());
    +
    +      for (ValueVector mapElt : mapVec) {
    +        if (vv.getField().getLastName().equalsIgnoreCase("column")) {
    +          BigIntHolder nameHolder;
    +          if (statMap.get(mapElt.getField().getLastName()) != null) {
    +            nameHolder = (BigIntHolder) statMap.get(mapElt.getField().getLastName());
    +          } else {
    +            nameHolder = new BigIntHolder();
    +            statMap.put(mapElt.getField().getLastName(), nameHolder);
    +          }
    +          nameHolder.value = 1;
    +        } else if (vv.getField().getLastName().equalsIgnoreCase("statcount")
    +            || vv.getField().getLastName().equalsIgnoreCase("nonnullstatcount")) {
    +          BigIntHolder sumHolder;
    +          String colName = mapElt.getField().getLastName();
    +          if (statMap.get(colName) != null) {
    +            sumHolder = (BigIntHolder) statMap.get(colName);
    +          } else {
    +            sumHolder = new BigIntHolder();
    +            statMap.put(colName, sumHolder);
    +          }
    +          //TODO: assert size = 1
    +          //TODO: logger
    +          Object val = mapElt.getAccessor().getObject(0);
    +          if (val != null) {
    +            sumHolder.value += (long)val;
    +          }
    +        } else if (vv.getField().getLastName().equalsIgnoreCase("sum_width")) {
    +          NullableFloat8Holder sumHolder;
    +          String colName = mapElt.getField().getLastName();
    +          if (statMap == null) {
    +            statMap = aggregationMap.get(functions.get(vv.getField().getLastName()));
    +          }
    +          if (statMap.get(colName) != null) {
    +            sumHolder = (NullableFloat8Holder) statMap.get(colName);
    +          } else {
    +            sumHolder = new NullableFloat8Holder();
    +            statMap.put(colName, sumHolder);
    +          }
    +          //TODO: assert size = 1
    +          //TODO: logger
    +          Object val = mapElt.getAccessor().getObject(0);
    +          if (val != null) {
    +            sumHolder.value += (double) val;
    +            sumHolder.isSet = 1;
    +          }
    +        } else if (vv.getField().getLastName().equalsIgnoreCase("hll")) {
    +          ObjectHolder hllHolder;
    +          String colName = mapElt.getField().getLastName();
    +          if (statMap == null) {
    +            statMap = aggregationMap.get(functions.get(vv.getField().getLastName()));
    +          }
    +          if (statMap.get(colName) != null) {
    +            hllHolder = (ObjectHolder) statMap.get(colName);
    +          } else {
    +            hllHolder = new ObjectHolder();
    +            hllHolder.obj = new HyperLogLog(context.getContextInformation().getHllAccuracy());
    +            statMap.put(colName, hllHolder);
    +          }
    +          NullableVarBinaryVector hllVector = (NullableVarBinaryVector) mapElt;
    +          try {
    +            if (hllVector.getAccessor().isSet(0) == 1) {
    +              ByteArrayInputStream bais = new ByteArrayInputStream(hllVector.getAccessor().getObject(0), 0,
    +                  mapElt.getBufferSize());
    +              HyperLogLog other = HyperLogLog.Builder.build(new DataInputStream(bais));
    +              ((HyperLogLog) hllHolder.obj).addAll(other);
    +            }
    +          } catch (Exception ex) {
    +            //TODO: Catch IOException/CardinalityMergeException
    +            //TODO: logger
    +            return IterOutcome.STOP;
    +          }
    +        }
    +      }
    +      // Add NDV value vector map using HLL map (since the NDV map is directly generated from HLL and not produced by the underlying
    +      // Statistics Agg)
    +      Map<String, ValueHolder> hllMap = aggregationMap.get("hll");
    +      if (hllMap != null) {
    +        aggregationMap.put("ndv", hllMap);
    +      }
    +    }
    +    return IterOutcome.OK;
    +  }
    +
    +  public VectorContainer getOutgoingContainer() {
    +    return this.container;
    +  }
    +
    +  @Override
    +  public IterOutcome innerNext() {
    +    IterOutcome outcome;
    +    boolean didSomeWork = false;
    +    if (finished) {
    +      return IterOutcome.NONE;
    +    }
    +    try {
    +      outer: while (true) {
    +        outcome = next(incoming);
    +        switch (outcome) {
    +          case NONE:
    +            break outer;
    +          case OUT_OF_MEMORY:
    +          case NOT_YET:
    +          case STOP:
    +            return outcome;
    +          case OK_NEW_SCHEMA:
    +            if (first) {
    +              first =false;
    --- End diff --
    
    This is borrowed from the existing state management code in other operators. We should change them across the board rather than doing it for a few operators in a separate commit.


---
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 #729: Drill 1328: Support table statistics for Parquet

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

    https://github.com/apache/drill/pull/729#discussion_r100448803
  
    --- Diff: exec/java-exec/src/main/java/org/apache/drill/exec/ops/QueryContext.java ---
    @@ -77,6 +78,7 @@
       private final SchemaTreeProvider schemaTreeProvider;
       /** Stores constants and their holders by type */
       private final Map<String, Map<MinorType, ValueHolder>> constantValueHolderCache;
    +  private StatementType stmtType = StatementType.UNKNOWN;
    --- End diff --
    
    Passing it in the constructor seems infeasible because the query context is created much earlier. However, the statement type is determined after parsing.


---
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 #729: Drill 1328: Support table statistics for Parquet

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

    https://github.com/apache/drill/pull/729#discussion_r102286549
  
    --- Diff: exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/statistics/StatisticsMergeBatch.java ---
    @@ -0,0 +1,532 @@
    +/**
    + * 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.impl.statistics;
    +
    +import com.google.common.collect.Lists;
    +import com.google.common.collect.Maps;
    +import org.apache.drill.common.expression.LogicalExpression;
    +import org.apache.drill.common.expression.ErrorCollector;
    +import org.apache.drill.common.expression.ErrorCollectorImpl;
    +import org.apache.drill.common.expression.FunctionCallFactory;
    +import org.apache.drill.common.expression.SchemaPath;
    +import org.apache.drill.common.expression.ValueExpressions;
    +import org.apache.drill.common.types.TypeProtos;
    +import org.apache.drill.common.types.TypeProtos.MajorType;
    +import org.apache.drill.exec.exception.OutOfMemoryException;
    +import org.apache.drill.exec.exception.SchemaChangeException;
    +import org.apache.drill.exec.expr.ExpressionTreeMaterializer;
    +import org.apache.drill.exec.expr.TypeHelper;
    +import org.apache.drill.exec.expr.holders.BigIntHolder;
    +import org.apache.drill.exec.expr.holders.NullableFloat8Holder;
    +import org.apache.drill.exec.expr.holders.ObjectHolder;
    +import org.apache.drill.exec.expr.holders.ValueHolder;
    +import org.apache.drill.exec.ops.FragmentContext;
    +import org.apache.drill.exec.physical.config.StatisticsMerge;
    +import org.apache.drill.exec.record.AbstractSingleRecordBatch;
    +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.VectorContainer;
    +import org.apache.drill.exec.record.VectorWrapper;
    +import org.apache.drill.exec.vector.ValueVector;
    +import org.apache.drill.exec.vector.DateVector;
    +import org.apache.drill.exec.vector.BigIntVector;
    +import org.apache.drill.exec.vector.NullableBigIntVector;
    +import org.apache.drill.exec.vector.NullableFloat8Vector;
    +import org.apache.drill.exec.vector.NullableVarBinaryVector;
    +import org.apache.drill.exec.vector.VarCharVector;
    +import org.apache.drill.exec.vector.complex.MapVector;
    +
    +import java.io.ByteArrayInputStream;
    +import java.io.DataInputStream;
    +import java.io.IOException;
    +import java.util.List;
    +import java.util.GregorianCalendar;
    +import java.util.HashMap;
    +import java.util.Map;
    +import java.util.TimeZone;
    +
    +import com.clearspring.analytics.stream.cardinality.HyperLogLog;
    +
    +public class StatisticsMergeBatch extends AbstractSingleRecordBatch<StatisticsMerge> {
    +  private static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(StatisticsMergeBatch.class);
    +  private Map<String, String> functions;
    +  private boolean first = true;
    +  private boolean finished = false;
    +  private int schema = 0;
    +  private int recordCount = 0;
    +  private List<String> keyList = null;
    +  private Map<MaterializedField, ValueVector> dataSrcVecMap = null;
    +  // Map of non-map fields to VV in the incoming schema
    +  private Map<MaterializedField, ValueVector> copySrcVecMap = null;
    +  private Map<String, Map<String, ValueHolder>> aggregationMap = null;
    +  public StatisticsMergeBatch(StatisticsMerge popConfig, RecordBatch incoming,
    +                              FragmentContext context) throws OutOfMemoryException {
    +    super(popConfig, context, incoming);
    +    this.functions = new HashMap<>();
    +    this.aggregationMap = new HashMap<>();
    +
    +    /*for (String key : popConfig.getFunctions()) {
    +      aggregationMap.put(key, new HashMap<String, ValueHolder>());
    +      if (key.equalsIgnoreCase("statcount") || key.equalsIgnoreCase("nonnullstatcount")) {
    +        functions.put(key, "sum");
    +      } else if (key.equalsIgnoreCase("hll")) {
    +        functions.put(key, "hll_merge");
    +      } else if (key.equalsIgnoreCase("sum_width")) {
    +        functions.put(key, "avg_width");
    +      }
    +    }*/
    +    for (String key : popConfig.getFunctions()) {
    +      if (key.equalsIgnoreCase("sum_width")) {
    +        functions.put(key, "avg_width");
    +      } else if (key.equalsIgnoreCase("hll")) {
    +        functions.put(key, "hll_merge");
    +      } else {
    +        functions.put(key, key);
    +      }
    +      aggregationMap.put(functions.get(key), new HashMap<String, ValueHolder>());
    +    }
    +  }
    +
    +  private void createKeyColumn(String name, LogicalExpression expr, Map<MaterializedField, ValueVector> parentMap)
    +      throws SchemaChangeException {
    +    ErrorCollector collector = new ErrorCollectorImpl();
    +
    +    LogicalExpression mle = ExpressionTreeMaterializer.materialize(expr, incoming, collector,
    +        context.getFunctionRegistry());
    +
    +    MaterializedField outputField = MaterializedField.create(name, mle.getMajorType());
    +    ValueVector vector = TypeHelper.getNewVector(outputField, oContext.getAllocator());
    +    container.add(vector);
    +
    +    if (collector.hasErrors()) {
    +      throw new SchemaChangeException("Failure while materializing expression. "
    +          + collector.toErrorString());
    +    }
    +    parentMap.put(outputField, vector);
    +  }
    +
    +  private ValueVector addMapVector(String name, MapVector parent, LogicalExpression expr) throws SchemaChangeException {
    +    ErrorCollector collector = new ErrorCollectorImpl();
    +
    +    LogicalExpression mle = ExpressionTreeMaterializer.materialize(expr, incoming, collector,
    +        context.getFunctionRegistry());
    +
    +    Class<? extends ValueVector> vvc =
    +        TypeHelper.getValueVectorClass(mle.getMajorType().getMinorType(), mle.getMajorType().getMode());
    +    ValueVector vector = parent.addOrGet(name, mle.getMajorType(), vvc);
    +
    +    if (collector.hasErrors()) {
    +      throw new SchemaChangeException("Failure while materializing expression. "
    +          + collector.toErrorString());
    +    }
    +    return vector;
    +  }
    +
    +  /**
    +   * Identify the list of fields within a map which are unpivoted as columns in output
    +   */
    +  private void buildKeyList() {
    +    List<String> lastMapKeyList = null;
    +    for (VectorWrapper<?> vw : incoming) {
    +      if (vw.getField().getType().getMinorType() != TypeProtos.MinorType.MAP) {
    +        continue;
    +      }
    +
    +      keyList = Lists.newArrayList();
    +
    +      for (ValueVector vv : vw.getValueVector()) {
    +        keyList.add(vv.getField().getLastName());
    +      }
    +
    +      if (lastMapKeyList == null) {
    +        lastMapKeyList = keyList;
    +      } else {
    +        if (keyList.size() != lastMapKeyList.size() || !lastMapKeyList.containsAll(keyList)) {
    +          throw new UnsupportedOperationException("Maps have different fields");
    +        }
    +      }
    +    }
    +  }
    +
    +  private void buildOutputContainer() throws SchemaChangeException {
    +    dataSrcVecMap = Maps.newHashMap();
    +    copySrcVecMap = Maps.newHashMap();
    +    MajorType mt = null;
    +
    +    ErrorCollector collector = new ErrorCollectorImpl();
    +    GregorianCalendar calendar = new GregorianCalendar(TimeZone.getTimeZone("UTC"));
    +
    +    calendar.setTimeInMillis(System.currentTimeMillis());
    +    createKeyColumn("schema", ValueExpressions.getBigInt(schema++), copySrcVecMap);
    +    createKeyColumn("computed", ValueExpressions.getDate(calendar), copySrcVecMap);
    +
    +    for (VectorWrapper<?> vw : incoming) {
    +      addVectorToOutgoingContainer(vw.getField().getLastName(), vw, collector);
    +      /*MaterializedField ds = vw.getField();
    +      String field = vw.getField().getLastName();
    +      // Input map vector
    +      MapVector mapVector = (MapVector) vw.getValueVector();
    +      assert mapVector.getPrimitiveVectors().size() > 0;
    +      // Proceed to create output map vector with same name e.g. statcount etc.
    +      mt = mapVector.getField().getType();
    +      MaterializedField mf = MaterializedField.create(functions.get(field), mt);
    +      assert !dataSrcVecMap.containsKey(mf);
    +      ValueVector vector = TypeHelper.getNewVector(mf, oContext.getAllocator());
    +      container.add(vector);
    +      MapVector outputMapVector = (MapVector) vector;
    +
    +      for (ValueVector vv : mapVector) {
    +        String fieldName = vv.getField().getLastName();
    +        if (!keyList.contains(fieldName)) {
    +          throw new UnsupportedOperationException("Unpivot data vector " +
    +              ds + " contains key " + fieldName + " not contained in key source!");
    +        }
    +        if (vv.getField().getType().getMinorType() == TypeProtos.MinorType.MAP) {
    +          throw new UnsupportedOperationException("Unpivot of nested map is not supported!");
    +        }
    +        if (field.equals("column")) {
    +          outputMapVector.addOrGet(fieldName, vv.getField().getType(), vv.getClass());
    +        } else {
    +          List<LogicalExpression> args = Lists.newArrayList();
    +          //TODO: Something else to access value of col such as emp_id?
    +          args.add(SchemaPath.getSimplePath(vv.getField().getPath()));
    +          //TODO: Put in the mapVector
    +          LogicalExpression call = FunctionCallFactory.createExpression(functions.get(field), args);
    +          //TODO: Is this sufficient to add to new Map?
    +          ValueVector vector1 = addMapVector(fieldName, outputMapVector, call);
    +          if (collector.hasErrors()) {
    +            throw new SchemaChangeException("Failure while materializing expression. "
    +                + collector.toErrorString());
    +          }
    +        }
    +      }
    +      dataSrcVecMap.put(ds, outputMapVector);*/
    +    }
    +    //Now create NDV in the outgoing container which was not avaliable in the incoming
    +    for (VectorWrapper<?> vw : incoming) {
    +      if (vw.getField().getLastName().equalsIgnoreCase("sum_width")) {//NullableFloat8 type
    +        addVectorToOutgoingContainer("ndv", vw, collector);
    +        break;
    +      }
    +    }
    +    container.setRecordCount(0);
    +    recordCount = 0;
    +    container.buildSchema(incoming.getSchema().getSelectionVectorMode());
    +  }
    +
    +  private void addVectorToOutgoingContainer(String field, VectorWrapper vw, ErrorCollector collector)
    +     throws SchemaChangeException {
    +    // Input map vector
    +    MapVector mapVector = (MapVector) vw.getValueVector();
    +    MaterializedField mf;
    +    assert mapVector.getPrimitiveVectors().size() > 0;
    +    // Proceed to create output map vector with same name e.g. statcount etc.
    +    MajorType mt = mapVector.getField().getType();
    +    if (functions.get(field) != null) {
    --- End diff --
    
    Refactored out.


---
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 #729: Drill 1328: Support table statistics for Parquet

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/729#discussion_r100680549
  
    --- Diff: exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/statistics/StatisticsMergeBatch.java ---
    @@ -0,0 +1,532 @@
    +/**
    + * 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.impl.statistics;
    +
    +import com.google.common.collect.Lists;
    +import com.google.common.collect.Maps;
    +import org.apache.drill.common.expression.LogicalExpression;
    +import org.apache.drill.common.expression.ErrorCollector;
    +import org.apache.drill.common.expression.ErrorCollectorImpl;
    +import org.apache.drill.common.expression.FunctionCallFactory;
    +import org.apache.drill.common.expression.SchemaPath;
    +import org.apache.drill.common.expression.ValueExpressions;
    +import org.apache.drill.common.types.TypeProtos;
    +import org.apache.drill.common.types.TypeProtos.MajorType;
    +import org.apache.drill.exec.exception.OutOfMemoryException;
    +import org.apache.drill.exec.exception.SchemaChangeException;
    +import org.apache.drill.exec.expr.ExpressionTreeMaterializer;
    +import org.apache.drill.exec.expr.TypeHelper;
    +import org.apache.drill.exec.expr.holders.BigIntHolder;
    +import org.apache.drill.exec.expr.holders.NullableFloat8Holder;
    +import org.apache.drill.exec.expr.holders.ObjectHolder;
    +import org.apache.drill.exec.expr.holders.ValueHolder;
    +import org.apache.drill.exec.ops.FragmentContext;
    +import org.apache.drill.exec.physical.config.StatisticsMerge;
    +import org.apache.drill.exec.record.AbstractSingleRecordBatch;
    +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.VectorContainer;
    +import org.apache.drill.exec.record.VectorWrapper;
    +import org.apache.drill.exec.vector.ValueVector;
    +import org.apache.drill.exec.vector.DateVector;
    +import org.apache.drill.exec.vector.BigIntVector;
    +import org.apache.drill.exec.vector.NullableBigIntVector;
    +import org.apache.drill.exec.vector.NullableFloat8Vector;
    +import org.apache.drill.exec.vector.NullableVarBinaryVector;
    +import org.apache.drill.exec.vector.VarCharVector;
    +import org.apache.drill.exec.vector.complex.MapVector;
    +
    +import java.io.ByteArrayInputStream;
    +import java.io.DataInputStream;
    +import java.io.IOException;
    +import java.util.List;
    +import java.util.GregorianCalendar;
    +import java.util.HashMap;
    +import java.util.Map;
    +import java.util.TimeZone;
    +
    +import com.clearspring.analytics.stream.cardinality.HyperLogLog;
    +
    +public class StatisticsMergeBatch extends AbstractSingleRecordBatch<StatisticsMerge> {
    +  private static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(StatisticsMergeBatch.class);
    +  private Map<String, String> functions;
    +  private boolean first = true;
    +  private boolean finished = false;
    +  private int schema = 0;
    +  private int recordCount = 0;
    +  private List<String> keyList = null;
    +  private Map<MaterializedField, ValueVector> dataSrcVecMap = null;
    +  // Map of non-map fields to VV in the incoming schema
    +  private Map<MaterializedField, ValueVector> copySrcVecMap = null;
    +  private Map<String, Map<String, ValueHolder>> aggregationMap = null;
    +  public StatisticsMergeBatch(StatisticsMerge popConfig, RecordBatch incoming,
    +                              FragmentContext context) throws OutOfMemoryException {
    +    super(popConfig, context, incoming);
    +    this.functions = new HashMap<>();
    +    this.aggregationMap = new HashMap<>();
    +
    +    /*for (String key : popConfig.getFunctions()) {
    +      aggregationMap.put(key, new HashMap<String, ValueHolder>());
    +      if (key.equalsIgnoreCase("statcount") || key.equalsIgnoreCase("nonnullstatcount")) {
    +        functions.put(key, "sum");
    +      } else if (key.equalsIgnoreCase("hll")) {
    +        functions.put(key, "hll_merge");
    +      } else if (key.equalsIgnoreCase("sum_width")) {
    +        functions.put(key, "avg_width");
    +      }
    +    }*/
    +    for (String key : popConfig.getFunctions()) {
    +      if (key.equalsIgnoreCase("sum_width")) {
    +        functions.put(key, "avg_width");
    +      } else if (key.equalsIgnoreCase("hll")) {
    +        functions.put(key, "hll_merge");
    +      } else {
    +        functions.put(key, key);
    +      }
    +      aggregationMap.put(functions.get(key), new HashMap<String, ValueHolder>());
    +    }
    +  }
    +
    +  private void createKeyColumn(String name, LogicalExpression expr, Map<MaterializedField, ValueVector> parentMap)
    +      throws SchemaChangeException {
    +    ErrorCollector collector = new ErrorCollectorImpl();
    +
    +    LogicalExpression mle = ExpressionTreeMaterializer.materialize(expr, incoming, collector,
    +        context.getFunctionRegistry());
    +
    +    MaterializedField outputField = MaterializedField.create(name, mle.getMajorType());
    +    ValueVector vector = TypeHelper.getNewVector(outputField, oContext.getAllocator());
    +    container.add(vector);
    +
    +    if (collector.hasErrors()) {
    +      throw new SchemaChangeException("Failure while materializing expression. "
    +          + collector.toErrorString());
    +    }
    +    parentMap.put(outputField, vector);
    +  }
    +
    +  private ValueVector addMapVector(String name, MapVector parent, LogicalExpression expr) throws SchemaChangeException {
    +    ErrorCollector collector = new ErrorCollectorImpl();
    +
    +    LogicalExpression mle = ExpressionTreeMaterializer.materialize(expr, incoming, collector,
    +        context.getFunctionRegistry());
    +
    +    Class<? extends ValueVector> vvc =
    +        TypeHelper.getValueVectorClass(mle.getMajorType().getMinorType(), mle.getMajorType().getMode());
    +    ValueVector vector = parent.addOrGet(name, mle.getMajorType(), vvc);
    +
    +    if (collector.hasErrors()) {
    +      throw new SchemaChangeException("Failure while materializing expression. "
    +          + collector.toErrorString());
    +    }
    +    return vector;
    +  }
    +
    +  /**
    +   * Identify the list of fields within a map which are unpivoted as columns in output
    +   */
    +  private void buildKeyList() {
    +    List<String> lastMapKeyList = null;
    +    for (VectorWrapper<?> vw : incoming) {
    +      if (vw.getField().getType().getMinorType() != TypeProtos.MinorType.MAP) {
    +        continue;
    +      }
    +
    +      keyList = Lists.newArrayList();
    +
    +      for (ValueVector vv : vw.getValueVector()) {
    +        keyList.add(vv.getField().getLastName());
    +      }
    +
    +      if (lastMapKeyList == null) {
    +        lastMapKeyList = keyList;
    +      } else {
    +        if (keyList.size() != lastMapKeyList.size() || !lastMapKeyList.containsAll(keyList)) {
    +          throw new UnsupportedOperationException("Maps have different fields");
    +        }
    +      }
    +    }
    +  }
    +
    +  private void buildOutputContainer() throws SchemaChangeException {
    +    dataSrcVecMap = Maps.newHashMap();
    +    copySrcVecMap = Maps.newHashMap();
    +    MajorType mt = null;
    +
    +    ErrorCollector collector = new ErrorCollectorImpl();
    +    GregorianCalendar calendar = new GregorianCalendar(TimeZone.getTimeZone("UTC"));
    +
    +    calendar.setTimeInMillis(System.currentTimeMillis());
    +    createKeyColumn("schema", ValueExpressions.getBigInt(schema++), copySrcVecMap);
    +    createKeyColumn("computed", ValueExpressions.getDate(calendar), copySrcVecMap);
    +
    +    for (VectorWrapper<?> vw : incoming) {
    +      addVectorToOutgoingContainer(vw.getField().getLastName(), vw, collector);
    --- End diff --
    
    Do we really want to add the same vectors to the output? Columns make sense. But values? If we have, for example a list of distinct values on input, might we want a count of those values on output?


---
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 #729: Drill 1328: Support table statistics for Parquet

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/729#discussion_r100681062
  
    --- Diff: exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/statistics/StatisticsMergeBatch.java ---
    @@ -0,0 +1,532 @@
    +/**
    + * 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.impl.statistics;
    +
    +import com.google.common.collect.Lists;
    +import com.google.common.collect.Maps;
    +import org.apache.drill.common.expression.LogicalExpression;
    +import org.apache.drill.common.expression.ErrorCollector;
    +import org.apache.drill.common.expression.ErrorCollectorImpl;
    +import org.apache.drill.common.expression.FunctionCallFactory;
    +import org.apache.drill.common.expression.SchemaPath;
    +import org.apache.drill.common.expression.ValueExpressions;
    +import org.apache.drill.common.types.TypeProtos;
    +import org.apache.drill.common.types.TypeProtos.MajorType;
    +import org.apache.drill.exec.exception.OutOfMemoryException;
    +import org.apache.drill.exec.exception.SchemaChangeException;
    +import org.apache.drill.exec.expr.ExpressionTreeMaterializer;
    +import org.apache.drill.exec.expr.TypeHelper;
    +import org.apache.drill.exec.expr.holders.BigIntHolder;
    +import org.apache.drill.exec.expr.holders.NullableFloat8Holder;
    +import org.apache.drill.exec.expr.holders.ObjectHolder;
    +import org.apache.drill.exec.expr.holders.ValueHolder;
    +import org.apache.drill.exec.ops.FragmentContext;
    +import org.apache.drill.exec.physical.config.StatisticsMerge;
    +import org.apache.drill.exec.record.AbstractSingleRecordBatch;
    +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.VectorContainer;
    +import org.apache.drill.exec.record.VectorWrapper;
    +import org.apache.drill.exec.vector.ValueVector;
    +import org.apache.drill.exec.vector.DateVector;
    +import org.apache.drill.exec.vector.BigIntVector;
    +import org.apache.drill.exec.vector.NullableBigIntVector;
    +import org.apache.drill.exec.vector.NullableFloat8Vector;
    +import org.apache.drill.exec.vector.NullableVarBinaryVector;
    +import org.apache.drill.exec.vector.VarCharVector;
    +import org.apache.drill.exec.vector.complex.MapVector;
    +
    +import java.io.ByteArrayInputStream;
    +import java.io.DataInputStream;
    +import java.io.IOException;
    +import java.util.List;
    +import java.util.GregorianCalendar;
    +import java.util.HashMap;
    +import java.util.Map;
    +import java.util.TimeZone;
    +
    +import com.clearspring.analytics.stream.cardinality.HyperLogLog;
    +
    +public class StatisticsMergeBatch extends AbstractSingleRecordBatch<StatisticsMerge> {
    +  private static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(StatisticsMergeBatch.class);
    +  private Map<String, String> functions;
    +  private boolean first = true;
    +  private boolean finished = false;
    +  private int schema = 0;
    +  private int recordCount = 0;
    +  private List<String> keyList = null;
    +  private Map<MaterializedField, ValueVector> dataSrcVecMap = null;
    +  // Map of non-map fields to VV in the incoming schema
    +  private Map<MaterializedField, ValueVector> copySrcVecMap = null;
    +  private Map<String, Map<String, ValueHolder>> aggregationMap = null;
    +  public StatisticsMergeBatch(StatisticsMerge popConfig, RecordBatch incoming,
    +                              FragmentContext context) throws OutOfMemoryException {
    +    super(popConfig, context, incoming);
    +    this.functions = new HashMap<>();
    +    this.aggregationMap = new HashMap<>();
    +
    +    /*for (String key : popConfig.getFunctions()) {
    +      aggregationMap.put(key, new HashMap<String, ValueHolder>());
    +      if (key.equalsIgnoreCase("statcount") || key.equalsIgnoreCase("nonnullstatcount")) {
    +        functions.put(key, "sum");
    +      } else if (key.equalsIgnoreCase("hll")) {
    +        functions.put(key, "hll_merge");
    +      } else if (key.equalsIgnoreCase("sum_width")) {
    +        functions.put(key, "avg_width");
    +      }
    +    }*/
    +    for (String key : popConfig.getFunctions()) {
    +      if (key.equalsIgnoreCase("sum_width")) {
    +        functions.put(key, "avg_width");
    +      } else if (key.equalsIgnoreCase("hll")) {
    +        functions.put(key, "hll_merge");
    +      } else {
    +        functions.put(key, key);
    +      }
    +      aggregationMap.put(functions.get(key), new HashMap<String, ValueHolder>());
    +    }
    +  }
    +
    +  private void createKeyColumn(String name, LogicalExpression expr, Map<MaterializedField, ValueVector> parentMap)
    +      throws SchemaChangeException {
    +    ErrorCollector collector = new ErrorCollectorImpl();
    +
    +    LogicalExpression mle = ExpressionTreeMaterializer.materialize(expr, incoming, collector,
    +        context.getFunctionRegistry());
    +
    +    MaterializedField outputField = MaterializedField.create(name, mle.getMajorType());
    +    ValueVector vector = TypeHelper.getNewVector(outputField, oContext.getAllocator());
    +    container.add(vector);
    +
    +    if (collector.hasErrors()) {
    +      throw new SchemaChangeException("Failure while materializing expression. "
    +          + collector.toErrorString());
    +    }
    +    parentMap.put(outputField, vector);
    +  }
    +
    +  private ValueVector addMapVector(String name, MapVector parent, LogicalExpression expr) throws SchemaChangeException {
    +    ErrorCollector collector = new ErrorCollectorImpl();
    +
    +    LogicalExpression mle = ExpressionTreeMaterializer.materialize(expr, incoming, collector,
    +        context.getFunctionRegistry());
    +
    +    Class<? extends ValueVector> vvc =
    +        TypeHelper.getValueVectorClass(mle.getMajorType().getMinorType(), mle.getMajorType().getMode());
    +    ValueVector vector = parent.addOrGet(name, mle.getMajorType(), vvc);
    +
    +    if (collector.hasErrors()) {
    +      throw new SchemaChangeException("Failure while materializing expression. "
    +          + collector.toErrorString());
    +    }
    +    return vector;
    +  }
    +
    +  /**
    +   * Identify the list of fields within a map which are unpivoted as columns in output
    +   */
    +  private void buildKeyList() {
    +    List<String> lastMapKeyList = null;
    +    for (VectorWrapper<?> vw : incoming) {
    +      if (vw.getField().getType().getMinorType() != TypeProtos.MinorType.MAP) {
    +        continue;
    +      }
    +
    +      keyList = Lists.newArrayList();
    +
    +      for (ValueVector vv : vw.getValueVector()) {
    +        keyList.add(vv.getField().getLastName());
    +      }
    +
    +      if (lastMapKeyList == null) {
    +        lastMapKeyList = keyList;
    +      } else {
    +        if (keyList.size() != lastMapKeyList.size() || !lastMapKeyList.containsAll(keyList)) {
    +          throw new UnsupportedOperationException("Maps have different fields");
    +        }
    +      }
    +    }
    +  }
    +
    +  private void buildOutputContainer() throws SchemaChangeException {
    +    dataSrcVecMap = Maps.newHashMap();
    +    copySrcVecMap = Maps.newHashMap();
    +    MajorType mt = null;
    +
    +    ErrorCollector collector = new ErrorCollectorImpl();
    +    GregorianCalendar calendar = new GregorianCalendar(TimeZone.getTimeZone("UTC"));
    +
    +    calendar.setTimeInMillis(System.currentTimeMillis());
    +    createKeyColumn("schema", ValueExpressions.getBigInt(schema++), copySrcVecMap);
    +    createKeyColumn("computed", ValueExpressions.getDate(calendar), copySrcVecMap);
    +
    +    for (VectorWrapper<?> vw : incoming) {
    +      addVectorToOutgoingContainer(vw.getField().getLastName(), vw, collector);
    +      /*MaterializedField ds = vw.getField();
    +      String field = vw.getField().getLastName();
    +      // Input map vector
    +      MapVector mapVector = (MapVector) vw.getValueVector();
    +      assert mapVector.getPrimitiveVectors().size() > 0;
    +      // Proceed to create output map vector with same name e.g. statcount etc.
    +      mt = mapVector.getField().getType();
    +      MaterializedField mf = MaterializedField.create(functions.get(field), mt);
    +      assert !dataSrcVecMap.containsKey(mf);
    +      ValueVector vector = TypeHelper.getNewVector(mf, oContext.getAllocator());
    +      container.add(vector);
    +      MapVector outputMapVector = (MapVector) vector;
    +
    +      for (ValueVector vv : mapVector) {
    +        String fieldName = vv.getField().getLastName();
    +        if (!keyList.contains(fieldName)) {
    +          throw new UnsupportedOperationException("Unpivot data vector " +
    +              ds + " contains key " + fieldName + " not contained in key source!");
    +        }
    +        if (vv.getField().getType().getMinorType() == TypeProtos.MinorType.MAP) {
    +          throw new UnsupportedOperationException("Unpivot of nested map is not supported!");
    +        }
    +        if (field.equals("column")) {
    +          outputMapVector.addOrGet(fieldName, vv.getField().getType(), vv.getClass());
    +        } else {
    +          List<LogicalExpression> args = Lists.newArrayList();
    +          //TODO: Something else to access value of col such as emp_id?
    +          args.add(SchemaPath.getSimplePath(vv.getField().getPath()));
    +          //TODO: Put in the mapVector
    +          LogicalExpression call = FunctionCallFactory.createExpression(functions.get(field), args);
    +          //TODO: Is this sufficient to add to new Map?
    +          ValueVector vector1 = addMapVector(fieldName, outputMapVector, call);
    +          if (collector.hasErrors()) {
    +            throw new SchemaChangeException("Failure while materializing expression. "
    +                + collector.toErrorString());
    +          }
    +        }
    +      }
    +      dataSrcVecMap.put(ds, outputMapVector);*/
    +    }
    +    //Now create NDV in the outgoing container which was not avaliable in the incoming
    +    for (VectorWrapper<?> vw : incoming) {
    +      if (vw.getField().getLastName().equalsIgnoreCase("sum_width")) {//NullableFloat8 type
    +        addVectorToOutgoingContainer("ndv", vw, collector);
    +        break;
    +      }
    +    }
    +    container.setRecordCount(0);
    +    recordCount = 0;
    +    container.buildSchema(incoming.getSchema().getSelectionVectorMode());
    +  }
    +
    +  private void addVectorToOutgoingContainer(String field, VectorWrapper vw, ErrorCollector collector)
    +     throws SchemaChangeException {
    +    // Input map vector
    +    MapVector mapVector = (MapVector) vw.getValueVector();
    +    MaterializedField mf;
    +    assert mapVector.getPrimitiveVectors().size() > 0;
    +    // Proceed to create output map vector with same name e.g. statcount etc.
    +    MajorType mt = mapVector.getField().getType();
    +    if (functions.get(field) != null) {
    +      mf = MaterializedField.create(functions.get(field), mt);
    +    } else {
    +      mf = MaterializedField.create(field, mt);
    +    }
    +    assert !dataSrcVecMap.containsKey(mf);
    +    ValueVector vector = TypeHelper.getNewVector(mf, oContext.getAllocator());
    +    container.add(vector);
    +    MapVector outputMapVector = (MapVector) vector;
    +
    +    for (ValueVector vv : mapVector) {
    +      String fieldName = vv.getField().getLastName();
    +      if (!keyList.contains(fieldName)) {
    +        throw new UnsupportedOperationException("Unpivot data vector " +
    +                field + " contains key " + fieldName + " not contained in key source!");
    +      }
    +      if (vv.getField().getType().getMinorType() == TypeProtos.MinorType.MAP) {
    +        throw new UnsupportedOperationException("Unpivot of nested map is not supported!");
    +      }
    +      if (field.equals("column")) {
    +        outputMapVector.addOrGet(fieldName, vv.getField().getType(), vv.getClass());
    +      } else {
    +        List<LogicalExpression> args = Lists.newArrayList();
    +        LogicalExpression call;
    +        //TODO: Something else to access value of col such as emp_id?
    +        args.add(SchemaPath.getSimplePath(vv.getField().getPath()));
    +        //TODO: Put in the mapVector
    +        if (functions.get(field) != null) {
    +          call = FunctionCallFactory.createExpression(functions.get(field), args);
    +        } else {
    +          call = FunctionCallFactory.createExpression(field, args);
    +        }
    +        //TODO: Is this sufficient to add to new Map?
    +        ValueVector vector1 = addMapVector(fieldName, outputMapVector, call);
    +        if (collector.hasErrors()) {
    +          throw new SchemaChangeException("Failure while materializing expression. "
    +                  + collector.toErrorString());
    +        }
    +      }
    +    }
    +  }
    +
    +  @Override
    +  protected boolean setupNewSchema() throws SchemaChangeException {
    +    container.clear();
    +    buildKeyList();
    +    buildOutputContainer();
    +    return true;
    +  }
    +
    +  @Override
    +  protected IterOutcome doWork() {
    +    int outRecordCount = incoming.getRecordCount();
    +    HashMap<String, Long> nonNullRows = new HashMap<>();
    +
    +    for (VectorWrapper<?> vw : incoming) {
    +      ValueVector vv = vw.getValueVector();
    +      if (vv.getField().getType().getMinorType() != TypeProtos.MinorType.MAP) {
    +        //We only expect Maps in the incoming. STOP, if this is not the case.
    +        return IterOutcome.STOP;
    +      }
    +      MapVector mapVec = (MapVector) vv;
    +      Map<String, ValueHolder> statMap = aggregationMap.get(vv.getField().getLastName());
    +
    +      for (ValueVector mapElt : mapVec) {
    +        if (vv.getField().getLastName().equalsIgnoreCase("column")) {
    +          BigIntHolder nameHolder;
    +          if (statMap.get(mapElt.getField().getLastName()) != null) {
    +            nameHolder = (BigIntHolder) statMap.get(mapElt.getField().getLastName());
    +          } else {
    +            nameHolder = new BigIntHolder();
    +            statMap.put(mapElt.getField().getLastName(), nameHolder);
    +          }
    +          nameHolder.value = 1;
    +        } else if (vv.getField().getLastName().equalsIgnoreCase("statcount")
    +            || vv.getField().getLastName().equalsIgnoreCase("nonnullstatcount")) {
    +          BigIntHolder sumHolder;
    +          String colName = mapElt.getField().getLastName();
    +          if (statMap.get(colName) != null) {
    +            sumHolder = (BigIntHolder) statMap.get(colName);
    +          } else {
    +            sumHolder = new BigIntHolder();
    +            statMap.put(colName, sumHolder);
    +          }
    +          //TODO: assert size = 1
    +          //TODO: logger
    +          Object val = mapElt.getAccessor().getObject(0);
    +          if (val != null) {
    +            sumHolder.value += (long)val;
    +          }
    +        } else if (vv.getField().getLastName().equalsIgnoreCase("sum_width")) {
    +          NullableFloat8Holder sumHolder;
    +          String colName = mapElt.getField().getLastName();
    +          if (statMap == null) {
    +            statMap = aggregationMap.get(functions.get(vv.getField().getLastName()));
    +          }
    +          if (statMap.get(colName) != null) {
    +            sumHolder = (NullableFloat8Holder) statMap.get(colName);
    +          } else {
    +            sumHolder = new NullableFloat8Holder();
    +            statMap.put(colName, sumHolder);
    +          }
    +          //TODO: assert size = 1
    +          //TODO: logger
    +          Object val = mapElt.getAccessor().getObject(0);
    +          if (val != null) {
    +            sumHolder.value += (double) val;
    +            sumHolder.isSet = 1;
    +          }
    +        } else if (vv.getField().getLastName().equalsIgnoreCase("hll")) {
    +          ObjectHolder hllHolder;
    +          String colName = mapElt.getField().getLastName();
    +          if (statMap == null) {
    +            statMap = aggregationMap.get(functions.get(vv.getField().getLastName()));
    +          }
    +          if (statMap.get(colName) != null) {
    +            hllHolder = (ObjectHolder) statMap.get(colName);
    +          } else {
    +            hllHolder = new ObjectHolder();
    +            hllHolder.obj = new HyperLogLog(context.getContextInformation().getHllAccuracy());
    +            statMap.put(colName, hllHolder);
    +          }
    +          NullableVarBinaryVector hllVector = (NullableVarBinaryVector) mapElt;
    +          try {
    +            if (hllVector.getAccessor().isSet(0) == 1) {
    +              ByteArrayInputStream bais = new ByteArrayInputStream(hllVector.getAccessor().getObject(0), 0,
    +                  mapElt.getBufferSize());
    +              HyperLogLog other = HyperLogLog.Builder.build(new DataInputStream(bais));
    +              ((HyperLogLog) hllHolder.obj).addAll(other);
    +            }
    +          } catch (Exception ex) {
    +            //TODO: Catch IOException/CardinalityMergeException
    +            //TODO: logger
    +            return IterOutcome.STOP;
    +          }
    +        }
    +      }
    +      // Add NDV value vector map using HLL map (since the NDV map is directly generated from HLL and not produced by the underlying
    +      // Statistics Agg)
    +      Map<String, ValueHolder> hllMap = aggregationMap.get("hll");
    +      if (hllMap != null) {
    +        aggregationMap.put("ndv", hllMap);
    +      }
    +    }
    +    return IterOutcome.OK;
    +  }
    +
    +  public VectorContainer getOutgoingContainer() {
    +    return this.container;
    +  }
    +
    +  @Override
    +  public IterOutcome innerNext() {
    +    IterOutcome outcome;
    +    boolean didSomeWork = false;
    +    if (finished) {
    +      return IterOutcome.NONE;
    +    }
    +    try {
    +      outer: while (true) {
    +        outcome = next(incoming);
    +        switch (outcome) {
    +          case NONE:
    +            break outer;
    +          case OUT_OF_MEMORY:
    +          case NOT_YET:
    +          case STOP:
    +            return outcome;
    +          case OK_NEW_SCHEMA:
    +            if (first) {
    +              first =false;
    +              if (!setupNewSchema()) {
    +                outcome = IterOutcome.OK;
    +              }
    +              return outcome;
    +            }
    +            //fall through
    +          case OK:
    +            assert first == false : "First batch should be OK_NEW_SCHEMA";
    +            IterOutcome out = doWork();
    +            didSomeWork = true;
    +            if (out != IterOutcome.OK) {
    +              return out;
    +            }
    +            break;
    +          default:
    +            throw new UnsupportedOperationException("Unsupported upstream state " + outcome);
    +        }
    +      }
    +    } catch (SchemaChangeException ex) {
    +      kill(false);
    +      logger.error("Failure during query", ex);
    +      context.fail(ex);
    +      return IterOutcome.STOP;
    +    }
    +
    +    // We can only get here if upstream is NONE i.e. no more batches. If we did some work prior to
    +    // exhausting all upstream, then return OK. Otherwise, return NONE.
    +    if (didSomeWork) {
    +      IterOutcome out = buildOutgoingRecordBatch();
    --- End diff --
    
    Can we get so many different schema changes on input that we need more than one batch to return the outputs?


---
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 #729: Drill 1328: Support table statistics for Parquet

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/729#discussion_r100680561
  
    --- Diff: exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/statistics/StatisticsMergeBatch.java ---
    @@ -0,0 +1,532 @@
    +/**
    + * 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.impl.statistics;
    +
    +import com.google.common.collect.Lists;
    +import com.google.common.collect.Maps;
    +import org.apache.drill.common.expression.LogicalExpression;
    +import org.apache.drill.common.expression.ErrorCollector;
    +import org.apache.drill.common.expression.ErrorCollectorImpl;
    +import org.apache.drill.common.expression.FunctionCallFactory;
    +import org.apache.drill.common.expression.SchemaPath;
    +import org.apache.drill.common.expression.ValueExpressions;
    +import org.apache.drill.common.types.TypeProtos;
    +import org.apache.drill.common.types.TypeProtos.MajorType;
    +import org.apache.drill.exec.exception.OutOfMemoryException;
    +import org.apache.drill.exec.exception.SchemaChangeException;
    +import org.apache.drill.exec.expr.ExpressionTreeMaterializer;
    +import org.apache.drill.exec.expr.TypeHelper;
    +import org.apache.drill.exec.expr.holders.BigIntHolder;
    +import org.apache.drill.exec.expr.holders.NullableFloat8Holder;
    +import org.apache.drill.exec.expr.holders.ObjectHolder;
    +import org.apache.drill.exec.expr.holders.ValueHolder;
    +import org.apache.drill.exec.ops.FragmentContext;
    +import org.apache.drill.exec.physical.config.StatisticsMerge;
    +import org.apache.drill.exec.record.AbstractSingleRecordBatch;
    +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.VectorContainer;
    +import org.apache.drill.exec.record.VectorWrapper;
    +import org.apache.drill.exec.vector.ValueVector;
    +import org.apache.drill.exec.vector.DateVector;
    +import org.apache.drill.exec.vector.BigIntVector;
    +import org.apache.drill.exec.vector.NullableBigIntVector;
    +import org.apache.drill.exec.vector.NullableFloat8Vector;
    +import org.apache.drill.exec.vector.NullableVarBinaryVector;
    +import org.apache.drill.exec.vector.VarCharVector;
    +import org.apache.drill.exec.vector.complex.MapVector;
    +
    +import java.io.ByteArrayInputStream;
    +import java.io.DataInputStream;
    +import java.io.IOException;
    +import java.util.List;
    +import java.util.GregorianCalendar;
    +import java.util.HashMap;
    +import java.util.Map;
    +import java.util.TimeZone;
    +
    +import com.clearspring.analytics.stream.cardinality.HyperLogLog;
    +
    +public class StatisticsMergeBatch extends AbstractSingleRecordBatch<StatisticsMerge> {
    +  private static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(StatisticsMergeBatch.class);
    +  private Map<String, String> functions;
    +  private boolean first = true;
    +  private boolean finished = false;
    +  private int schema = 0;
    +  private int recordCount = 0;
    +  private List<String> keyList = null;
    +  private Map<MaterializedField, ValueVector> dataSrcVecMap = null;
    +  // Map of non-map fields to VV in the incoming schema
    +  private Map<MaterializedField, ValueVector> copySrcVecMap = null;
    +  private Map<String, Map<String, ValueHolder>> aggregationMap = null;
    +  public StatisticsMergeBatch(StatisticsMerge popConfig, RecordBatch incoming,
    +                              FragmentContext context) throws OutOfMemoryException {
    +    super(popConfig, context, incoming);
    +    this.functions = new HashMap<>();
    +    this.aggregationMap = new HashMap<>();
    +
    +    /*for (String key : popConfig.getFunctions()) {
    +      aggregationMap.put(key, new HashMap<String, ValueHolder>());
    +      if (key.equalsIgnoreCase("statcount") || key.equalsIgnoreCase("nonnullstatcount")) {
    +        functions.put(key, "sum");
    +      } else if (key.equalsIgnoreCase("hll")) {
    +        functions.put(key, "hll_merge");
    +      } else if (key.equalsIgnoreCase("sum_width")) {
    +        functions.put(key, "avg_width");
    +      }
    +    }*/
    +    for (String key : popConfig.getFunctions()) {
    +      if (key.equalsIgnoreCase("sum_width")) {
    +        functions.put(key, "avg_width");
    +      } else if (key.equalsIgnoreCase("hll")) {
    +        functions.put(key, "hll_merge");
    +      } else {
    +        functions.put(key, key);
    +      }
    +      aggregationMap.put(functions.get(key), new HashMap<String, ValueHolder>());
    +    }
    +  }
    +
    +  private void createKeyColumn(String name, LogicalExpression expr, Map<MaterializedField, ValueVector> parentMap)
    +      throws SchemaChangeException {
    +    ErrorCollector collector = new ErrorCollectorImpl();
    +
    +    LogicalExpression mle = ExpressionTreeMaterializer.materialize(expr, incoming, collector,
    +        context.getFunctionRegistry());
    +
    +    MaterializedField outputField = MaterializedField.create(name, mle.getMajorType());
    +    ValueVector vector = TypeHelper.getNewVector(outputField, oContext.getAllocator());
    +    container.add(vector);
    +
    +    if (collector.hasErrors()) {
    +      throw new SchemaChangeException("Failure while materializing expression. "
    +          + collector.toErrorString());
    +    }
    +    parentMap.put(outputField, vector);
    +  }
    +
    +  private ValueVector addMapVector(String name, MapVector parent, LogicalExpression expr) throws SchemaChangeException {
    +    ErrorCollector collector = new ErrorCollectorImpl();
    +
    +    LogicalExpression mle = ExpressionTreeMaterializer.materialize(expr, incoming, collector,
    +        context.getFunctionRegistry());
    +
    +    Class<? extends ValueVector> vvc =
    +        TypeHelper.getValueVectorClass(mle.getMajorType().getMinorType(), mle.getMajorType().getMode());
    +    ValueVector vector = parent.addOrGet(name, mle.getMajorType(), vvc);
    +
    +    if (collector.hasErrors()) {
    +      throw new SchemaChangeException("Failure while materializing expression. "
    +          + collector.toErrorString());
    +    }
    +    return vector;
    +  }
    +
    +  /**
    +   * Identify the list of fields within a map which are unpivoted as columns in output
    +   */
    +  private void buildKeyList() {
    +    List<String> lastMapKeyList = null;
    +    for (VectorWrapper<?> vw : incoming) {
    +      if (vw.getField().getType().getMinorType() != TypeProtos.MinorType.MAP) {
    +        continue;
    +      }
    +
    +      keyList = Lists.newArrayList();
    +
    +      for (ValueVector vv : vw.getValueVector()) {
    +        keyList.add(vv.getField().getLastName());
    +      }
    +
    +      if (lastMapKeyList == null) {
    +        lastMapKeyList = keyList;
    +      } else {
    +        if (keyList.size() != lastMapKeyList.size() || !lastMapKeyList.containsAll(keyList)) {
    +          throw new UnsupportedOperationException("Maps have different fields");
    +        }
    +      }
    +    }
    +  }
    +
    +  private void buildOutputContainer() throws SchemaChangeException {
    +    dataSrcVecMap = Maps.newHashMap();
    +    copySrcVecMap = Maps.newHashMap();
    +    MajorType mt = null;
    +
    +    ErrorCollector collector = new ErrorCollectorImpl();
    +    GregorianCalendar calendar = new GregorianCalendar(TimeZone.getTimeZone("UTC"));
    +
    +    calendar.setTimeInMillis(System.currentTimeMillis());
    +    createKeyColumn("schema", ValueExpressions.getBigInt(schema++), copySrcVecMap);
    +    createKeyColumn("computed", ValueExpressions.getDate(calendar), copySrcVecMap);
    +
    +    for (VectorWrapper<?> vw : incoming) {
    +      addVectorToOutgoingContainer(vw.getField().getLastName(), vw, collector);
    +      /*MaterializedField ds = vw.getField();
    --- End diff --
    
    Remove obsolete code before the PR? If you want it, keep it in your working branch. But, remove it in the PR branch.


---
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 #729: Drill 1328: Support table statistics for Parquet

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/729#discussion_r100681030
  
    --- Diff: exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/statistics/StatisticsMergeBatch.java ---
    @@ -0,0 +1,532 @@
    +/**
    + * 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.impl.statistics;
    +
    +import com.google.common.collect.Lists;
    +import com.google.common.collect.Maps;
    +import org.apache.drill.common.expression.LogicalExpression;
    +import org.apache.drill.common.expression.ErrorCollector;
    +import org.apache.drill.common.expression.ErrorCollectorImpl;
    +import org.apache.drill.common.expression.FunctionCallFactory;
    +import org.apache.drill.common.expression.SchemaPath;
    +import org.apache.drill.common.expression.ValueExpressions;
    +import org.apache.drill.common.types.TypeProtos;
    +import org.apache.drill.common.types.TypeProtos.MajorType;
    +import org.apache.drill.exec.exception.OutOfMemoryException;
    +import org.apache.drill.exec.exception.SchemaChangeException;
    +import org.apache.drill.exec.expr.ExpressionTreeMaterializer;
    +import org.apache.drill.exec.expr.TypeHelper;
    +import org.apache.drill.exec.expr.holders.BigIntHolder;
    +import org.apache.drill.exec.expr.holders.NullableFloat8Holder;
    +import org.apache.drill.exec.expr.holders.ObjectHolder;
    +import org.apache.drill.exec.expr.holders.ValueHolder;
    +import org.apache.drill.exec.ops.FragmentContext;
    +import org.apache.drill.exec.physical.config.StatisticsMerge;
    +import org.apache.drill.exec.record.AbstractSingleRecordBatch;
    +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.VectorContainer;
    +import org.apache.drill.exec.record.VectorWrapper;
    +import org.apache.drill.exec.vector.ValueVector;
    +import org.apache.drill.exec.vector.DateVector;
    +import org.apache.drill.exec.vector.BigIntVector;
    +import org.apache.drill.exec.vector.NullableBigIntVector;
    +import org.apache.drill.exec.vector.NullableFloat8Vector;
    +import org.apache.drill.exec.vector.NullableVarBinaryVector;
    +import org.apache.drill.exec.vector.VarCharVector;
    +import org.apache.drill.exec.vector.complex.MapVector;
    +
    +import java.io.ByteArrayInputStream;
    +import java.io.DataInputStream;
    +import java.io.IOException;
    +import java.util.List;
    +import java.util.GregorianCalendar;
    +import java.util.HashMap;
    +import java.util.Map;
    +import java.util.TimeZone;
    +
    +import com.clearspring.analytics.stream.cardinality.HyperLogLog;
    +
    +public class StatisticsMergeBatch extends AbstractSingleRecordBatch<StatisticsMerge> {
    +  private static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(StatisticsMergeBatch.class);
    +  private Map<String, String> functions;
    +  private boolean first = true;
    +  private boolean finished = false;
    +  private int schema = 0;
    +  private int recordCount = 0;
    +  private List<String> keyList = null;
    +  private Map<MaterializedField, ValueVector> dataSrcVecMap = null;
    +  // Map of non-map fields to VV in the incoming schema
    +  private Map<MaterializedField, ValueVector> copySrcVecMap = null;
    +  private Map<String, Map<String, ValueHolder>> aggregationMap = null;
    +  public StatisticsMergeBatch(StatisticsMerge popConfig, RecordBatch incoming,
    +                              FragmentContext context) throws OutOfMemoryException {
    +    super(popConfig, context, incoming);
    +    this.functions = new HashMap<>();
    +    this.aggregationMap = new HashMap<>();
    +
    +    /*for (String key : popConfig.getFunctions()) {
    +      aggregationMap.put(key, new HashMap<String, ValueHolder>());
    +      if (key.equalsIgnoreCase("statcount") || key.equalsIgnoreCase("nonnullstatcount")) {
    +        functions.put(key, "sum");
    +      } else if (key.equalsIgnoreCase("hll")) {
    +        functions.put(key, "hll_merge");
    +      } else if (key.equalsIgnoreCase("sum_width")) {
    +        functions.put(key, "avg_width");
    +      }
    +    }*/
    +    for (String key : popConfig.getFunctions()) {
    +      if (key.equalsIgnoreCase("sum_width")) {
    +        functions.put(key, "avg_width");
    +      } else if (key.equalsIgnoreCase("hll")) {
    +        functions.put(key, "hll_merge");
    +      } else {
    +        functions.put(key, key);
    +      }
    +      aggregationMap.put(functions.get(key), new HashMap<String, ValueHolder>());
    +    }
    +  }
    +
    +  private void createKeyColumn(String name, LogicalExpression expr, Map<MaterializedField, ValueVector> parentMap)
    +      throws SchemaChangeException {
    +    ErrorCollector collector = new ErrorCollectorImpl();
    +
    +    LogicalExpression mle = ExpressionTreeMaterializer.materialize(expr, incoming, collector,
    +        context.getFunctionRegistry());
    +
    +    MaterializedField outputField = MaterializedField.create(name, mle.getMajorType());
    +    ValueVector vector = TypeHelper.getNewVector(outputField, oContext.getAllocator());
    +    container.add(vector);
    +
    +    if (collector.hasErrors()) {
    +      throw new SchemaChangeException("Failure while materializing expression. "
    +          + collector.toErrorString());
    +    }
    +    parentMap.put(outputField, vector);
    +  }
    +
    +  private ValueVector addMapVector(String name, MapVector parent, LogicalExpression expr) throws SchemaChangeException {
    +    ErrorCollector collector = new ErrorCollectorImpl();
    +
    +    LogicalExpression mle = ExpressionTreeMaterializer.materialize(expr, incoming, collector,
    +        context.getFunctionRegistry());
    +
    +    Class<? extends ValueVector> vvc =
    +        TypeHelper.getValueVectorClass(mle.getMajorType().getMinorType(), mle.getMajorType().getMode());
    +    ValueVector vector = parent.addOrGet(name, mle.getMajorType(), vvc);
    +
    +    if (collector.hasErrors()) {
    +      throw new SchemaChangeException("Failure while materializing expression. "
    +          + collector.toErrorString());
    +    }
    +    return vector;
    +  }
    +
    +  /**
    +   * Identify the list of fields within a map which are unpivoted as columns in output
    +   */
    +  private void buildKeyList() {
    +    List<String> lastMapKeyList = null;
    +    for (VectorWrapper<?> vw : incoming) {
    +      if (vw.getField().getType().getMinorType() != TypeProtos.MinorType.MAP) {
    +        continue;
    +      }
    +
    +      keyList = Lists.newArrayList();
    +
    +      for (ValueVector vv : vw.getValueVector()) {
    +        keyList.add(vv.getField().getLastName());
    +      }
    +
    +      if (lastMapKeyList == null) {
    +        lastMapKeyList = keyList;
    +      } else {
    +        if (keyList.size() != lastMapKeyList.size() || !lastMapKeyList.containsAll(keyList)) {
    +          throw new UnsupportedOperationException("Maps have different fields");
    +        }
    +      }
    +    }
    +  }
    +
    +  private void buildOutputContainer() throws SchemaChangeException {
    +    dataSrcVecMap = Maps.newHashMap();
    +    copySrcVecMap = Maps.newHashMap();
    +    MajorType mt = null;
    +
    +    ErrorCollector collector = new ErrorCollectorImpl();
    +    GregorianCalendar calendar = new GregorianCalendar(TimeZone.getTimeZone("UTC"));
    +
    +    calendar.setTimeInMillis(System.currentTimeMillis());
    +    createKeyColumn("schema", ValueExpressions.getBigInt(schema++), copySrcVecMap);
    +    createKeyColumn("computed", ValueExpressions.getDate(calendar), copySrcVecMap);
    +
    +    for (VectorWrapper<?> vw : incoming) {
    +      addVectorToOutgoingContainer(vw.getField().getLastName(), vw, collector);
    +      /*MaterializedField ds = vw.getField();
    +      String field = vw.getField().getLastName();
    +      // Input map vector
    +      MapVector mapVector = (MapVector) vw.getValueVector();
    +      assert mapVector.getPrimitiveVectors().size() > 0;
    +      // Proceed to create output map vector with same name e.g. statcount etc.
    +      mt = mapVector.getField().getType();
    +      MaterializedField mf = MaterializedField.create(functions.get(field), mt);
    +      assert !dataSrcVecMap.containsKey(mf);
    +      ValueVector vector = TypeHelper.getNewVector(mf, oContext.getAllocator());
    +      container.add(vector);
    +      MapVector outputMapVector = (MapVector) vector;
    +
    +      for (ValueVector vv : mapVector) {
    +        String fieldName = vv.getField().getLastName();
    +        if (!keyList.contains(fieldName)) {
    +          throw new UnsupportedOperationException("Unpivot data vector " +
    +              ds + " contains key " + fieldName + " not contained in key source!");
    +        }
    +        if (vv.getField().getType().getMinorType() == TypeProtos.MinorType.MAP) {
    +          throw new UnsupportedOperationException("Unpivot of nested map is not supported!");
    +        }
    +        if (field.equals("column")) {
    +          outputMapVector.addOrGet(fieldName, vv.getField().getType(), vv.getClass());
    +        } else {
    +          List<LogicalExpression> args = Lists.newArrayList();
    +          //TODO: Something else to access value of col such as emp_id?
    +          args.add(SchemaPath.getSimplePath(vv.getField().getPath()));
    +          //TODO: Put in the mapVector
    +          LogicalExpression call = FunctionCallFactory.createExpression(functions.get(field), args);
    +          //TODO: Is this sufficient to add to new Map?
    +          ValueVector vector1 = addMapVector(fieldName, outputMapVector, call);
    +          if (collector.hasErrors()) {
    +            throw new SchemaChangeException("Failure while materializing expression. "
    +                + collector.toErrorString());
    +          }
    +        }
    +      }
    +      dataSrcVecMap.put(ds, outputMapVector);*/
    +    }
    +    //Now create NDV in the outgoing container which was not avaliable in the incoming
    +    for (VectorWrapper<?> vw : incoming) {
    +      if (vw.getField().getLastName().equalsIgnoreCase("sum_width")) {//NullableFloat8 type
    +        addVectorToOutgoingContainer("ndv", vw, collector);
    +        break;
    +      }
    +    }
    +    container.setRecordCount(0);
    +    recordCount = 0;
    +    container.buildSchema(incoming.getSchema().getSelectionVectorMode());
    +  }
    +
    +  private void addVectorToOutgoingContainer(String field, VectorWrapper vw, ErrorCollector collector)
    +     throws SchemaChangeException {
    +    // Input map vector
    +    MapVector mapVector = (MapVector) vw.getValueVector();
    +    MaterializedField mf;
    +    assert mapVector.getPrimitiveVectors().size() > 0;
    +    // Proceed to create output map vector with same name e.g. statcount etc.
    +    MajorType mt = mapVector.getField().getType();
    +    if (functions.get(field) != null) {
    +      mf = MaterializedField.create(functions.get(field), mt);
    +    } else {
    +      mf = MaterializedField.create(field, mt);
    +    }
    +    assert !dataSrcVecMap.containsKey(mf);
    +    ValueVector vector = TypeHelper.getNewVector(mf, oContext.getAllocator());
    +    container.add(vector);
    +    MapVector outputMapVector = (MapVector) vector;
    +
    +    for (ValueVector vv : mapVector) {
    +      String fieldName = vv.getField().getLastName();
    +      if (!keyList.contains(fieldName)) {
    +        throw new UnsupportedOperationException("Unpivot data vector " +
    +                field + " contains key " + fieldName + " not contained in key source!");
    +      }
    +      if (vv.getField().getType().getMinorType() == TypeProtos.MinorType.MAP) {
    +        throw new UnsupportedOperationException("Unpivot of nested map is not supported!");
    +      }
    +      if (field.equals("column")) {
    +        outputMapVector.addOrGet(fieldName, vv.getField().getType(), vv.getClass());
    +      } else {
    +        List<LogicalExpression> args = Lists.newArrayList();
    +        LogicalExpression call;
    +        //TODO: Something else to access value of col such as emp_id?
    +        args.add(SchemaPath.getSimplePath(vv.getField().getPath()));
    +        //TODO: Put in the mapVector
    +        if (functions.get(field) != null) {
    +          call = FunctionCallFactory.createExpression(functions.get(field), args);
    +        } else {
    +          call = FunctionCallFactory.createExpression(field, args);
    +        }
    +        //TODO: Is this sufficient to add to new Map?
    +        ValueVector vector1 = addMapVector(fieldName, outputMapVector, call);
    +        if (collector.hasErrors()) {
    +          throw new SchemaChangeException("Failure while materializing expression. "
    +                  + collector.toErrorString());
    +        }
    +      }
    +    }
    +  }
    +
    +  @Override
    +  protected boolean setupNewSchema() throws SchemaChangeException {
    +    container.clear();
    +    buildKeyList();
    +    buildOutputContainer();
    +    return true;
    +  }
    +
    +  @Override
    +  protected IterOutcome doWork() {
    +    int outRecordCount = incoming.getRecordCount();
    +    HashMap<String, Long> nonNullRows = new HashMap<>();
    +
    +    for (VectorWrapper<?> vw : incoming) {
    +      ValueVector vv = vw.getValueVector();
    +      if (vv.getField().getType().getMinorType() != TypeProtos.MinorType.MAP) {
    +        //We only expect Maps in the incoming. STOP, if this is not the case.
    +        return IterOutcome.STOP;
    +      }
    +      MapVector mapVec = (MapVector) vv;
    +      Map<String, ValueHolder> statMap = aggregationMap.get(vv.getField().getLastName());
    +
    +      for (ValueVector mapElt : mapVec) {
    +        if (vv.getField().getLastName().equalsIgnoreCase("column")) {
    +          BigIntHolder nameHolder;
    +          if (statMap.get(mapElt.getField().getLastName()) != null) {
    +            nameHolder = (BigIntHolder) statMap.get(mapElt.getField().getLastName());
    +          } else {
    +            nameHolder = new BigIntHolder();
    +            statMap.put(mapElt.getField().getLastName(), nameHolder);
    +          }
    +          nameHolder.value = 1;
    +        } else if (vv.getField().getLastName().equalsIgnoreCase("statcount")
    +            || vv.getField().getLastName().equalsIgnoreCase("nonnullstatcount")) {
    +          BigIntHolder sumHolder;
    +          String colName = mapElt.getField().getLastName();
    +          if (statMap.get(colName) != null) {
    +            sumHolder = (BigIntHolder) statMap.get(colName);
    +          } else {
    +            sumHolder = new BigIntHolder();
    +            statMap.put(colName, sumHolder);
    +          }
    +          //TODO: assert size = 1
    +          //TODO: logger
    +          Object val = mapElt.getAccessor().getObject(0);
    +          if (val != null) {
    +            sumHolder.value += (long)val;
    +          }
    +        } else if (vv.getField().getLastName().equalsIgnoreCase("sum_width")) {
    +          NullableFloat8Holder sumHolder;
    +          String colName = mapElt.getField().getLastName();
    +          if (statMap == null) {
    +            statMap = aggregationMap.get(functions.get(vv.getField().getLastName()));
    +          }
    +          if (statMap.get(colName) != null) {
    +            sumHolder = (NullableFloat8Holder) statMap.get(colName);
    +          } else {
    +            sumHolder = new NullableFloat8Holder();
    +            statMap.put(colName, sumHolder);
    +          }
    +          //TODO: assert size = 1
    +          //TODO: logger
    +          Object val = mapElt.getAccessor().getObject(0);
    +          if (val != null) {
    +            sumHolder.value += (double) val;
    +            sumHolder.isSet = 1;
    +          }
    +        } else if (vv.getField().getLastName().equalsIgnoreCase("hll")) {
    +          ObjectHolder hllHolder;
    +          String colName = mapElt.getField().getLastName();
    +          if (statMap == null) {
    +            statMap = aggregationMap.get(functions.get(vv.getField().getLastName()));
    +          }
    +          if (statMap.get(colName) != null) {
    +            hllHolder = (ObjectHolder) statMap.get(colName);
    +          } else {
    +            hllHolder = new ObjectHolder();
    +            hllHolder.obj = new HyperLogLog(context.getContextInformation().getHllAccuracy());
    +            statMap.put(colName, hllHolder);
    +          }
    +          NullableVarBinaryVector hllVector = (NullableVarBinaryVector) mapElt;
    +          try {
    +            if (hllVector.getAccessor().isSet(0) == 1) {
    +              ByteArrayInputStream bais = new ByteArrayInputStream(hllVector.getAccessor().getObject(0), 0,
    +                  mapElt.getBufferSize());
    +              HyperLogLog other = HyperLogLog.Builder.build(new DataInputStream(bais));
    +              ((HyperLogLog) hllHolder.obj).addAll(other);
    +            }
    +          } catch (Exception ex) {
    +            //TODO: Catch IOException/CardinalityMergeException
    +            //TODO: logger
    +            return IterOutcome.STOP;
    +          }
    +        }
    +      }
    +      // Add NDV value vector map using HLL map (since the NDV map is directly generated from HLL and not produced by the underlying
    +      // Statistics Agg)
    +      Map<String, ValueHolder> hllMap = aggregationMap.get("hll");
    +      if (hllMap != null) {
    +        aggregationMap.put("ndv", hllMap);
    +      }
    +    }
    +    return IterOutcome.OK;
    +  }
    +
    +  public VectorContainer getOutgoingContainer() {
    +    return this.container;
    +  }
    +
    +  @Override
    +  public IterOutcome innerNext() {
    +    IterOutcome outcome;
    +    boolean didSomeWork = false;
    +    if (finished) {
    +      return IterOutcome.NONE;
    +    }
    +    try {
    +      outer: while (true) {
    +        outcome = next(incoming);
    +        switch (outcome) {
    +          case NONE:
    +            break outer;
    +          case OUT_OF_MEMORY:
    +          case NOT_YET:
    +          case STOP:
    +            return outcome;
    +          case OK_NEW_SCHEMA:
    +            if (first) {
    +              first =false;
    --- End diff --
    
    The multiple-fag approach to state management is awkward. Maybe define an enum with states:
    ```
    enum OperatorState { START, RUN, END };
    ```


---
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 #729: Drill 1328: Support table statistics for Parquet

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

    https://github.com/apache/drill/pull/729#discussion_r102295174
  
    --- Diff: exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/statistics/StatisticsMergeBatch.java ---
    @@ -0,0 +1,532 @@
    +/**
    + * 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.impl.statistics;
    +
    +import com.google.common.collect.Lists;
    +import com.google.common.collect.Maps;
    +import org.apache.drill.common.expression.LogicalExpression;
    +import org.apache.drill.common.expression.ErrorCollector;
    +import org.apache.drill.common.expression.ErrorCollectorImpl;
    +import org.apache.drill.common.expression.FunctionCallFactory;
    +import org.apache.drill.common.expression.SchemaPath;
    +import org.apache.drill.common.expression.ValueExpressions;
    +import org.apache.drill.common.types.TypeProtos;
    +import org.apache.drill.common.types.TypeProtos.MajorType;
    +import org.apache.drill.exec.exception.OutOfMemoryException;
    +import org.apache.drill.exec.exception.SchemaChangeException;
    +import org.apache.drill.exec.expr.ExpressionTreeMaterializer;
    +import org.apache.drill.exec.expr.TypeHelper;
    +import org.apache.drill.exec.expr.holders.BigIntHolder;
    +import org.apache.drill.exec.expr.holders.NullableFloat8Holder;
    +import org.apache.drill.exec.expr.holders.ObjectHolder;
    +import org.apache.drill.exec.expr.holders.ValueHolder;
    +import org.apache.drill.exec.ops.FragmentContext;
    +import org.apache.drill.exec.physical.config.StatisticsMerge;
    +import org.apache.drill.exec.record.AbstractSingleRecordBatch;
    +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.VectorContainer;
    +import org.apache.drill.exec.record.VectorWrapper;
    +import org.apache.drill.exec.vector.ValueVector;
    +import org.apache.drill.exec.vector.DateVector;
    +import org.apache.drill.exec.vector.BigIntVector;
    +import org.apache.drill.exec.vector.NullableBigIntVector;
    +import org.apache.drill.exec.vector.NullableFloat8Vector;
    +import org.apache.drill.exec.vector.NullableVarBinaryVector;
    +import org.apache.drill.exec.vector.VarCharVector;
    +import org.apache.drill.exec.vector.complex.MapVector;
    +
    +import java.io.ByteArrayInputStream;
    +import java.io.DataInputStream;
    +import java.io.IOException;
    +import java.util.List;
    +import java.util.GregorianCalendar;
    +import java.util.HashMap;
    +import java.util.Map;
    +import java.util.TimeZone;
    +
    +import com.clearspring.analytics.stream.cardinality.HyperLogLog;
    +
    +public class StatisticsMergeBatch extends AbstractSingleRecordBatch<StatisticsMerge> {
    +  private static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(StatisticsMergeBatch.class);
    +  private Map<String, String> functions;
    +  private boolean first = true;
    +  private boolean finished = false;
    +  private int schema = 0;
    +  private int recordCount = 0;
    +  private List<String> keyList = null;
    +  private Map<MaterializedField, ValueVector> dataSrcVecMap = null;
    +  // Map of non-map fields to VV in the incoming schema
    +  private Map<MaterializedField, ValueVector> copySrcVecMap = null;
    +  private Map<String, Map<String, ValueHolder>> aggregationMap = null;
    +  public StatisticsMergeBatch(StatisticsMerge popConfig, RecordBatch incoming,
    +                              FragmentContext context) throws OutOfMemoryException {
    +    super(popConfig, context, incoming);
    +    this.functions = new HashMap<>();
    +    this.aggregationMap = new HashMap<>();
    +
    +    /*for (String key : popConfig.getFunctions()) {
    +      aggregationMap.put(key, new HashMap<String, ValueHolder>());
    +      if (key.equalsIgnoreCase("statcount") || key.equalsIgnoreCase("nonnullstatcount")) {
    +        functions.put(key, "sum");
    +      } else if (key.equalsIgnoreCase("hll")) {
    +        functions.put(key, "hll_merge");
    +      } else if (key.equalsIgnoreCase("sum_width")) {
    +        functions.put(key, "avg_width");
    +      }
    +    }*/
    +    for (String key : popConfig.getFunctions()) {
    +      if (key.equalsIgnoreCase("sum_width")) {
    +        functions.put(key, "avg_width");
    +      } else if (key.equalsIgnoreCase("hll")) {
    +        functions.put(key, "hll_merge");
    +      } else {
    +        functions.put(key, key);
    +      }
    +      aggregationMap.put(functions.get(key), new HashMap<String, ValueHolder>());
    +    }
    +  }
    +
    +  private void createKeyColumn(String name, LogicalExpression expr, Map<MaterializedField, ValueVector> parentMap)
    +      throws SchemaChangeException {
    +    ErrorCollector collector = new ErrorCollectorImpl();
    +
    +    LogicalExpression mle = ExpressionTreeMaterializer.materialize(expr, incoming, collector,
    +        context.getFunctionRegistry());
    +
    +    MaterializedField outputField = MaterializedField.create(name, mle.getMajorType());
    +    ValueVector vector = TypeHelper.getNewVector(outputField, oContext.getAllocator());
    +    container.add(vector);
    +
    +    if (collector.hasErrors()) {
    +      throw new SchemaChangeException("Failure while materializing expression. "
    +          + collector.toErrorString());
    +    }
    +    parentMap.put(outputField, vector);
    +  }
    +
    +  private ValueVector addMapVector(String name, MapVector parent, LogicalExpression expr) throws SchemaChangeException {
    +    ErrorCollector collector = new ErrorCollectorImpl();
    +
    +    LogicalExpression mle = ExpressionTreeMaterializer.materialize(expr, incoming, collector,
    +        context.getFunctionRegistry());
    +
    +    Class<? extends ValueVector> vvc =
    +        TypeHelper.getValueVectorClass(mle.getMajorType().getMinorType(), mle.getMajorType().getMode());
    +    ValueVector vector = parent.addOrGet(name, mle.getMajorType(), vvc);
    +
    +    if (collector.hasErrors()) {
    +      throw new SchemaChangeException("Failure while materializing expression. "
    +          + collector.toErrorString());
    +    }
    +    return vector;
    +  }
    +
    +  /**
    +   * Identify the list of fields within a map which are unpivoted as columns in output
    +   */
    +  private void buildKeyList() {
    +    List<String> lastMapKeyList = null;
    +    for (VectorWrapper<?> vw : incoming) {
    +      if (vw.getField().getType().getMinorType() != TypeProtos.MinorType.MAP) {
    +        continue;
    +      }
    +
    +      keyList = Lists.newArrayList();
    +
    +      for (ValueVector vv : vw.getValueVector()) {
    +        keyList.add(vv.getField().getLastName());
    +      }
    +
    +      if (lastMapKeyList == null) {
    +        lastMapKeyList = keyList;
    +      } else {
    +        if (keyList.size() != lastMapKeyList.size() || !lastMapKeyList.containsAll(keyList)) {
    +          throw new UnsupportedOperationException("Maps have different fields");
    +        }
    +      }
    +    }
    +  }
    +
    +  private void buildOutputContainer() throws SchemaChangeException {
    +    dataSrcVecMap = Maps.newHashMap();
    +    copySrcVecMap = Maps.newHashMap();
    +    MajorType mt = null;
    +
    +    ErrorCollector collector = new ErrorCollectorImpl();
    +    GregorianCalendar calendar = new GregorianCalendar(TimeZone.getTimeZone("UTC"));
    +
    +    calendar.setTimeInMillis(System.currentTimeMillis());
    +    createKeyColumn("schema", ValueExpressions.getBigInt(schema++), copySrcVecMap);
    +    createKeyColumn("computed", ValueExpressions.getDate(calendar), copySrcVecMap);
    +
    +    for (VectorWrapper<?> vw : incoming) {
    +      addVectorToOutgoingContainer(vw.getField().getLastName(), vw, collector);
    +      /*MaterializedField ds = vw.getField();
    +      String field = vw.getField().getLastName();
    +      // Input map vector
    +      MapVector mapVector = (MapVector) vw.getValueVector();
    +      assert mapVector.getPrimitiveVectors().size() > 0;
    +      // Proceed to create output map vector with same name e.g. statcount etc.
    +      mt = mapVector.getField().getType();
    +      MaterializedField mf = MaterializedField.create(functions.get(field), mt);
    +      assert !dataSrcVecMap.containsKey(mf);
    +      ValueVector vector = TypeHelper.getNewVector(mf, oContext.getAllocator());
    +      container.add(vector);
    +      MapVector outputMapVector = (MapVector) vector;
    +
    +      for (ValueVector vv : mapVector) {
    +        String fieldName = vv.getField().getLastName();
    +        if (!keyList.contains(fieldName)) {
    +          throw new UnsupportedOperationException("Unpivot data vector " +
    +              ds + " contains key " + fieldName + " not contained in key source!");
    +        }
    +        if (vv.getField().getType().getMinorType() == TypeProtos.MinorType.MAP) {
    +          throw new UnsupportedOperationException("Unpivot of nested map is not supported!");
    +        }
    +        if (field.equals("column")) {
    +          outputMapVector.addOrGet(fieldName, vv.getField().getType(), vv.getClass());
    +        } else {
    +          List<LogicalExpression> args = Lists.newArrayList();
    +          //TODO: Something else to access value of col such as emp_id?
    +          args.add(SchemaPath.getSimplePath(vv.getField().getPath()));
    +          //TODO: Put in the mapVector
    +          LogicalExpression call = FunctionCallFactory.createExpression(functions.get(field), args);
    +          //TODO: Is this sufficient to add to new Map?
    +          ValueVector vector1 = addMapVector(fieldName, outputMapVector, call);
    +          if (collector.hasErrors()) {
    +            throw new SchemaChangeException("Failure while materializing expression. "
    +                + collector.toErrorString());
    +          }
    +        }
    +      }
    +      dataSrcVecMap.put(ds, outputMapVector);*/
    +    }
    +    //Now create NDV in the outgoing container which was not avaliable in the incoming
    +    for (VectorWrapper<?> vw : incoming) {
    +      if (vw.getField().getLastName().equalsIgnoreCase("sum_width")) {//NullableFloat8 type
    +        addVectorToOutgoingContainer("ndv", vw, collector);
    +        break;
    +      }
    +    }
    +    container.setRecordCount(0);
    +    recordCount = 0;
    +    container.buildSchema(incoming.getSchema().getSelectionVectorMode());
    +  }
    +
    +  private void addVectorToOutgoingContainer(String field, VectorWrapper vw, ErrorCollector collector)
    +     throws SchemaChangeException {
    +    // Input map vector
    +    MapVector mapVector = (MapVector) vw.getValueVector();
    +    MaterializedField mf;
    +    assert mapVector.getPrimitiveVectors().size() > 0;
    +    // Proceed to create output map vector with same name e.g. statcount etc.
    +    MajorType mt = mapVector.getField().getType();
    +    if (functions.get(field) != null) {
    +      mf = MaterializedField.create(functions.get(field), mt);
    +    } else {
    +      mf = MaterializedField.create(field, mt);
    +    }
    +    assert !dataSrcVecMap.containsKey(mf);
    +    ValueVector vector = TypeHelper.getNewVector(mf, oContext.getAllocator());
    +    container.add(vector);
    +    MapVector outputMapVector = (MapVector) vector;
    +
    +    for (ValueVector vv : mapVector) {
    +      String fieldName = vv.getField().getLastName();
    +      if (!keyList.contains(fieldName)) {
    +        throw new UnsupportedOperationException("Unpivot data vector " +
    +                field + " contains key " + fieldName + " not contained in key source!");
    +      }
    +      if (vv.getField().getType().getMinorType() == TypeProtos.MinorType.MAP) {
    +        throw new UnsupportedOperationException("Unpivot of nested map is not supported!");
    +      }
    +      if (field.equals("column")) {
    +        outputMapVector.addOrGet(fieldName, vv.getField().getType(), vv.getClass());
    +      } else {
    +        List<LogicalExpression> args = Lists.newArrayList();
    +        LogicalExpression call;
    +        //TODO: Something else to access value of col such as emp_id?
    +        args.add(SchemaPath.getSimplePath(vv.getField().getPath()));
    +        //TODO: Put in the mapVector
    +        if (functions.get(field) != null) {
    +          call = FunctionCallFactory.createExpression(functions.get(field), args);
    +        } else {
    +          call = FunctionCallFactory.createExpression(field, args);
    +        }
    +        //TODO: Is this sufficient to add to new Map?
    +        ValueVector vector1 = addMapVector(fieldName, outputMapVector, call);
    +        if (collector.hasErrors()) {
    +          throw new SchemaChangeException("Failure while materializing expression. "
    +                  + collector.toErrorString());
    +        }
    +      }
    +    }
    +  }
    +
    +  @Override
    +  protected boolean setupNewSchema() throws SchemaChangeException {
    +    container.clear();
    +    buildKeyList();
    +    buildOutputContainer();
    +    return true;
    +  }
    +
    +  @Override
    +  protected IterOutcome doWork() {
    +    int outRecordCount = incoming.getRecordCount();
    +    HashMap<String, Long> nonNullRows = new HashMap<>();
    +
    +    for (VectorWrapper<?> vw : incoming) {
    +      ValueVector vv = vw.getValueVector();
    +      if (vv.getField().getType().getMinorType() != TypeProtos.MinorType.MAP) {
    +        //We only expect Maps in the incoming. STOP, if this is not the case.
    +        return IterOutcome.STOP;
    --- 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 #729: Drill 1328: Support table statistics for Parquet

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/729#discussion_r102871196
  
    --- Diff: exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/statistics/AvgWidthMergedStatistic.java ---
    @@ -0,0 +1,135 @@
    +/*
    + * 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.impl.statistics;
    +
    +import org.apache.drill.common.types.TypeProtos;
    +import org.apache.drill.exec.expr.holders.NullableFloat8Holder;
    +import org.apache.drill.exec.expr.holders.ValueHolder;
    +import org.apache.drill.exec.vector.NullableFloat8Vector;
    +import org.apache.drill.exec.vector.ValueVector;
    +import org.apache.drill.exec.vector.complex.MapVector;
    +
    +import java.util.HashMap;
    +import java.util.List;
    +import java.util.Map;
    +
    +public class AvgWidthMergedStatistic extends AbstractMergedStatistic {
    +
    +  private String name;
    +  private String inputName;
    +  private boolean configureComplete = false;
    +  private boolean mergeComplete = false;
    +  private Map<String, ValueHolder> sumHolder;
    +  MergedStatistic types, nonNullStatCounts, statCounts;
    +
    +  public AvgWidthMergedStatistic (String name, String inputName) {
    +    this.name = name;
    +    this.inputName = inputName;
    +    this.sumHolder = new HashMap<>();
    +    types = nonNullStatCounts = statCounts = null;
    +  }
    +
    +  @Override
    +  public String getName() {
    +    return name;
    +  }
    +
    +  @Override
    +  public String getInput() {
    +    return inputName;
    +  }
    +
    +  @Override
    +  public void merge(ValueVector input) {
    +    // Check the input is a Map Vector
    +    assert (input.getField().getType().getMinorType() == TypeProtos.MinorType.MAP);
    +    MapVector inputMap = (MapVector) input;
    +    for (ValueVector vv : inputMap) {
    +      String colName = vv.getField().getLastName();
    +      NullableFloat8Holder colSumHolder;
    +      if (sumHolder.get(colName) != null) {
    +        colSumHolder = (NullableFloat8Holder) sumHolder.get(colName);
    +      } else {
    +        colSumHolder = new NullableFloat8Holder();
    +        sumHolder.put(colName, colSumHolder);
    +      }
    +      Object val = vv.getAccessor().getObject(0);
    +      if (val != null) {
    +        colSumHolder.value += (double) val;
    +        colSumHolder.isSet = 1;
    +      }
    +    }
    +  }
    +
    +  @Override
    +  public Object getStat(String colName) {
    +      if (mergeComplete != true) {
    +        throw new IllegalStateException(
    +            String.format("Statistic `%s` has not completed merging statistics", name));
    +      }
    +      NullableFloat8Holder colSumHolder = (NullableFloat8Holder) sumHolder.get(colName);
    +      return (long) (colSumHolder.value/ getRowCount(colName));
    +    }
    +
    +  @Override
    +  public void setOutput(ValueVector output) {
    +    // Check the input is a Map Vector
    +    assert (output.getField().getType().getMinorType() == TypeProtos.MinorType.MAP);
    +    // Dependencies have been configured correctly
    +    assert (configureComplete == true);
    +    MapVector outputMap = (MapVector) output;
    +
    +    for (ValueVector outMapCol : outputMap) {
    +      String colName = outMapCol.getField().getLastName();
    +      NullableFloat8Holder colSumHolder = (NullableFloat8Holder) sumHolder.get(colName);
    +      NullableFloat8Vector vv = (NullableFloat8Vector) outMapCol;
    +      vv.allocateNewSafe();
    +      vv.getMutator().setSafe(0, (colSumHolder.value / getRowCount(colName)));
    +    }
    --- End diff --
    
    This is very expensive implementation! Creating value vectors is pretty heavy-weight.
    
    ```
    for (String colName : sumHolder.keySet()) {
      NullableFloat8Vector vv = outMapCol.getChild(
        colName, NullableFloat8Vector.class);
        vv.allocateNewSafe();
        vv.getMutator().setSafe(0, getStat(colName));
    }
    ```


---
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 #729: Drill 1328: Support table statistics for Parquet

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

    https://github.com/apache/drill/pull/729
  
    I have addressed the comments from the earlier pull request. Please take a look


---
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 #729: Drill 1328: Support table statistics for Parquet

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/729#discussion_r100680361
  
    --- Diff: exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/statistics/StatisticsMergeBatch.java ---
    @@ -0,0 +1,532 @@
    +/**
    + * 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.impl.statistics;
    +
    +import com.google.common.collect.Lists;
    +import com.google.common.collect.Maps;
    +import org.apache.drill.common.expression.LogicalExpression;
    +import org.apache.drill.common.expression.ErrorCollector;
    +import org.apache.drill.common.expression.ErrorCollectorImpl;
    +import org.apache.drill.common.expression.FunctionCallFactory;
    +import org.apache.drill.common.expression.SchemaPath;
    +import org.apache.drill.common.expression.ValueExpressions;
    +import org.apache.drill.common.types.TypeProtos;
    +import org.apache.drill.common.types.TypeProtos.MajorType;
    +import org.apache.drill.exec.exception.OutOfMemoryException;
    +import org.apache.drill.exec.exception.SchemaChangeException;
    +import org.apache.drill.exec.expr.ExpressionTreeMaterializer;
    +import org.apache.drill.exec.expr.TypeHelper;
    +import org.apache.drill.exec.expr.holders.BigIntHolder;
    +import org.apache.drill.exec.expr.holders.NullableFloat8Holder;
    +import org.apache.drill.exec.expr.holders.ObjectHolder;
    +import org.apache.drill.exec.expr.holders.ValueHolder;
    +import org.apache.drill.exec.ops.FragmentContext;
    +import org.apache.drill.exec.physical.config.StatisticsMerge;
    +import org.apache.drill.exec.record.AbstractSingleRecordBatch;
    +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.VectorContainer;
    +import org.apache.drill.exec.record.VectorWrapper;
    +import org.apache.drill.exec.vector.ValueVector;
    +import org.apache.drill.exec.vector.DateVector;
    +import org.apache.drill.exec.vector.BigIntVector;
    +import org.apache.drill.exec.vector.NullableBigIntVector;
    +import org.apache.drill.exec.vector.NullableFloat8Vector;
    +import org.apache.drill.exec.vector.NullableVarBinaryVector;
    +import org.apache.drill.exec.vector.VarCharVector;
    +import org.apache.drill.exec.vector.complex.MapVector;
    +
    +import java.io.ByteArrayInputStream;
    +import java.io.DataInputStream;
    +import java.io.IOException;
    +import java.util.List;
    +import java.util.GregorianCalendar;
    +import java.util.HashMap;
    +import java.util.Map;
    +import java.util.TimeZone;
    +
    +import com.clearspring.analytics.stream.cardinality.HyperLogLog;
    +
    +public class StatisticsMergeBatch extends AbstractSingleRecordBatch<StatisticsMerge> {
    +  private static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(StatisticsMergeBatch.class);
    +  private Map<String, String> functions;
    +  private boolean first = true;
    +  private boolean finished = false;
    +  private int schema = 0;
    +  private int recordCount = 0;
    +  private List<String> keyList = null;
    +  private Map<MaterializedField, ValueVector> dataSrcVecMap = null;
    +  // Map of non-map fields to VV in the incoming schema
    +  private Map<MaterializedField, ValueVector> copySrcVecMap = null;
    +  private Map<String, Map<String, ValueHolder>> aggregationMap = null;
    +  public StatisticsMergeBatch(StatisticsMerge popConfig, RecordBatch incoming,
    +                              FragmentContext context) throws OutOfMemoryException {
    +    super(popConfig, context, incoming);
    +    this.functions = new HashMap<>();
    +    this.aggregationMap = new HashMap<>();
    +
    +    /*for (String key : popConfig.getFunctions()) {
    --- End diff --
    
    Remove debug code prior to PR?


---
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 #729: Drill 1328: Support table statistics for Parquet

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

    https://github.com/apache/drill/pull/729#discussion_r100709767
  
    --- Diff: exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/statistics/StatisticsAggBatch.java ---
    @@ -0,0 +1,256 @@
    +/**
    + * 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.impl.statistics;
    +
    +import com.google.common.collect.Lists;
    +import com.sun.codemodel.JExpr;
    +import org.apache.drill.common.expression.ErrorCollector;
    +import org.apache.drill.common.expression.ErrorCollectorImpl;
    +import org.apache.drill.common.expression.FunctionCallFactory;
    +import org.apache.drill.common.expression.LogicalExpression;
    +import org.apache.drill.common.expression.SchemaPath;
    +import org.apache.drill.common.expression.ValueExpressions;
    +import org.apache.drill.exec.exception.ClassTransformationException;
    +import org.apache.drill.exec.exception.OutOfMemoryException;
    +import org.apache.drill.exec.exception.SchemaChangeException;
    +import org.apache.drill.exec.expr.ClassGenerator;
    +import org.apache.drill.exec.expr.CodeGenerator;
    +import org.apache.drill.exec.expr.ExpressionTreeMaterializer;
    +import org.apache.drill.exec.expr.TypeHelper;
    +import org.apache.drill.exec.expr.ValueVectorWriteExpression;
    +import org.apache.drill.exec.ops.FragmentContext;
    +import org.apache.drill.exec.physical.config.StatisticsAggregate;
    +import org.apache.drill.exec.physical.impl.aggregate.StreamingAggBatch;
    +import org.apache.drill.exec.physical.impl.aggregate.StreamingAggTemplate;
    +import org.apache.drill.exec.physical.impl.aggregate.StreamingAggregator;
    +import org.apache.drill.exec.planner.physical.StatsAggPrel.OperatorPhase;
    +import org.apache.drill.exec.record.BatchSchema.SelectionVectorMode;
    +import org.apache.drill.exec.record.MaterializedField;
    +import org.apache.drill.exec.record.RecordBatch;
    +import org.apache.drill.exec.record.TypedFieldId;
    +import org.apache.drill.exec.store.ImplicitColumnExplorer;
    +import org.apache.drill.exec.vector.ValueVector;
    +import org.apache.drill.exec.vector.complex.FieldIdUtil;
    +import org.apache.drill.exec.vector.complex.MapVector;
    +
    +import java.io.IOException;
    +import java.util.GregorianCalendar;
    +import java.util.List;
    +import java.util.TimeZone;
    +
    +/**
    + * TODO: This needs cleanup. Currently the key values are constants and we compare the constants
    + * for every record. Seems unnecessary.
    + *
    + * Example input and output:
    + * Schema of incoming batch: region_id (VARCHAR), sales_city (VARCHAR), cnt (BIGINT)
    + * Schema of output:
    + *    "schema" : BIGINT - Schema number. For each schema change this number is incremented.
    + *    "computed" : BIGINT - What time is it computed?
    + *    "columns"       : MAP - Column names
    + *       "region_id"  : VARCHAR
    + *       "sales_city" : VARCHAR
    + *       "cnt"        : VARCHAR
    + *    "statscount" : MAP
    + *       "region_id"  : BIGINT - statscount(region_id) - aggregation over all values of region_id
    + *                      in incoming batch
    + *       "sales_city" : BIGINT - statscount(sales_city)
    + *       "cnt"        : BIGINT - statscount(cnt)
    + *    "nonnullstatcount" : MAP
    + *       "region_id"  : BIGINT - nonnullstatcount(region_id)
    + *       "sales_city" : BIGINT - nonnullstatcount(sales_city)
    + *       "cnt"        : BIGINT - nonnullstatcount(cnt)
    + *   .... another map for next stats function ....
    + */
    +public class StatisticsAggBatch extends StreamingAggBatch {
    +  private List<String> functions;
    +  private int schema = 0;
    +
    +  public StatisticsAggBatch(StatisticsAggregate popConfig, RecordBatch incoming,
    +      FragmentContext context) throws OutOfMemoryException {
    +    super(popConfig, incoming, context);
    +    this.functions = popConfig.getFunctions();
    +  }
    +
    +  private void createKeyColumn(String name, LogicalExpression expr, List<LogicalExpression> keyExprs,
    +      List<TypedFieldId> keyOutputIds) throws SchemaChangeException {
    +    ErrorCollector collector = new ErrorCollectorImpl();
    +
    +    LogicalExpression mle = ExpressionTreeMaterializer.materialize(expr, incoming, collector,
    +        context.getFunctionRegistry());
    +
    +    MaterializedField outputField = MaterializedField.create(name, mle.getMajorType());
    +    ValueVector vector = TypeHelper.getNewVector(outputField, oContext.getAllocator());
    +
    +    keyExprs.add(mle);
    +    keyOutputIds.add(container.add(vector));
    +
    +    if (collector.hasErrors()) {
    +      throw new SchemaChangeException("Failure while materializing expression. "
    +          + collector.toErrorString());
    +    }
    +  }
    +
    +  private void createNestedKeyColumn(MapVector parent, String name, LogicalExpression expr,
    +      List<LogicalExpression> keyExprs, List<TypedFieldId> keyOutputIds)
    +          throws SchemaChangeException {
    +    ErrorCollector collector = new ErrorCollectorImpl();
    +
    +    LogicalExpression mle = ExpressionTreeMaterializer.materialize(expr, incoming, collector,
    +        context.getFunctionRegistry());
    +
    +    Class<? extends ValueVector> vvc =
    +        TypeHelper.getValueVectorClass(mle.getMajorType().getMinorType(),
    +            mle.getMajorType().getMode());
    +
    +    ValueVector vv = parent.addOrGet(name, mle.getMajorType(), vvc);
    +
    +    TypedFieldId pfid = container.getValueVectorId(SchemaPath.getSimplePath(parent.getField().getPath()));
    +    assert pfid.getFieldIds().length == 1;
    +    TypedFieldId.Builder builder = TypedFieldId.newBuilder();
    +    builder.addId(pfid.getFieldIds()[0]);
    +    TypedFieldId id =
    +        FieldIdUtil.getFieldIdIfMatches(parent, builder, true,
    +            SchemaPath.getSimplePath(vv.getField().getPath()).getRootSegment());
    +
    +    keyExprs.add(mle);
    +    keyOutputIds.add(id);
    +
    +    if (collector.hasErrors()) {
    +      throw new SchemaChangeException("Failure while materializing expression. "
    +          + collector.toErrorString());
    +    }
    +  }
    +
    +  private void addMapVector(String name, MapVector parent, LogicalExpression expr,
    +      List<LogicalExpression> valueExprs) throws SchemaChangeException {
    +    ErrorCollector collector = new ErrorCollectorImpl();
    +
    +    LogicalExpression mle = ExpressionTreeMaterializer.materialize(expr, incoming, collector,
    +        context.getFunctionRegistry());
    +
    +    Class<? extends ValueVector> vvc =
    +        TypeHelper.getValueVectorClass(mle.getMajorType().getMinorType(), mle.getMajorType().getMode());
    +    ValueVector vv = parent.addOrGet(name, mle.getMajorType(), vvc);
    +
    +    TypedFieldId pfid = container.getValueVectorId(SchemaPath.getSimplePath(parent.getField().getPath()));
    +    assert pfid.getFieldIds().length == 1;
    +    TypedFieldId.Builder builder = TypedFieldId.newBuilder();
    +    builder.addId(pfid.getFieldIds()[0]);
    +    TypedFieldId id = FieldIdUtil.getFieldIdIfMatches(parent, builder, true,
    +        SchemaPath.getSimplePath(vv.getField().getPath()).getRootSegment());
    +
    +    valueExprs.add(new ValueVectorWriteExpression(id, mle, true));
    +
    +    if (collector.hasErrors()) {
    +      throw new SchemaChangeException("Failure while materializing expression. "
    +          + collector.toErrorString());
    +    }
    +  }
    +
    +  private StreamingAggregator codegenAggregator(List<LogicalExpression> keyExprs,
    +      List<LogicalExpression> valueExprs, List<TypedFieldId> keyOutputIds)
    +          throws SchemaChangeException, ClassTransformationException, IOException {
    +    ClassGenerator<StreamingAggregator> cg =
    +        CodeGenerator.getRoot(StreamingAggTemplate.TEMPLATE_DEFINITION, context.getFunctionRegistry(),
    +            context.getOptions());
    +
    +    LogicalExpression[] keyExprsArray = new LogicalExpression[keyExprs.size()];
    +    LogicalExpression[] valueExprsArray = new LogicalExpression[valueExprs.size()];
    +    TypedFieldId[] keyOutputIdsArray = new TypedFieldId[keyOutputIds.size()];
    +
    +    keyExprs.toArray(keyExprsArray);
    +    valueExprs.toArray(valueExprsArray);
    +    keyOutputIds.toArray(keyOutputIdsArray);
    +
    +    setupIsSame(cg, keyExprsArray);
    +    setupIsSameApart(cg, keyExprsArray);
    +    addRecordValues(cg, valueExprsArray);
    +    outputRecordKeys(cg, keyOutputIdsArray, keyExprsArray);
    +    outputRecordKeysPrev(cg, keyOutputIdsArray, keyExprsArray);
    +
    +    cg.getBlock("resetValues")._return(JExpr.TRUE);
    +    getIndex(cg);
    +
    +    container.buildSchema(SelectionVectorMode.NONE);
    +    StreamingAggregator agg = context.getImplementationClass(cg);
    +    agg.setup(oContext, incoming, this);
    +    return agg;
    +  }
    +
    +  private boolean isImplicitFileColumn(MaterializedField mf) {
    +    return ImplicitColumnExplorer.initImplicitFileColumns(context.getOptions()).get(mf.getName()) != null;
    +  }
    +
    +  protected StreamingAggregator createAggregatorInternal()
    +      throws SchemaChangeException, ClassTransformationException, IOException {
    +    container.clear();
    +
    +    List<LogicalExpression> keyExprs = Lists.newArrayList();
    +    List<LogicalExpression> valueExprs = Lists.newArrayList();
    +    List<TypedFieldId> keyOutputIds = Lists.newArrayList();
    +    GregorianCalendar calendar = new GregorianCalendar(TimeZone.getTimeZone("UTC"));
    +    calendar.setTimeInMillis(System.currentTimeMillis());
    +
    +    if (this.getPopConfig() instanceof StatisticsAggregate
    --- End diff --
    
    I do not have a preference either way. Removed it.


---
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 #729: Drill 1328: Support table statistics for Parquet

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

    https://github.com/apache/drill/pull/729#discussion_r100421972
  
    --- Diff: exec/java-exec/src/main/java/org/apache/drill/exec/ops/ContextInformation.java ---
    @@ -28,12 +28,14 @@
       private final String currentDefaultSchema;
       private final long queryStartTime;
       private final int rootFragmentTimeZone;
    +  private final int hllMemoryLimit;
     
       public ContextInformation(final UserCredentials userCredentials, final QueryContextInformation queryContextInfo) {
         this.queryUser = userCredentials.getUserName();
         this.currentDefaultSchema = queryContextInfo.getDefaultSchemaName();
         this.queryStartTime = queryContextInfo.getQueryStartTime();
         this.rootFragmentTimeZone = queryContextInfo.getTimeZone();
    +    this.hllMemoryLimit = queryContextInfo.getHllMemoryLimit();
    --- End diff --
    
    Re-labeled as hll_accuracy to avoid confusion. This option is meant to control the accuracy of NDV computation. However, more accuracy comes at the cost of using more memory.


---
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 #729: Drill 1328: Support table statistics for Parquet

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

    https://github.com/apache/drill/pull/729#discussion_r103419914
  
    --- Diff: exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/statistics/StatisticsAggBatch.java ---
    @@ -0,0 +1,256 @@
    +/**
    + * 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.impl.statistics;
    +
    +import com.google.common.collect.Lists;
    +import com.sun.codemodel.JExpr;
    +import org.apache.drill.common.expression.ErrorCollector;
    +import org.apache.drill.common.expression.ErrorCollectorImpl;
    +import org.apache.drill.common.expression.FunctionCallFactory;
    +import org.apache.drill.common.expression.LogicalExpression;
    +import org.apache.drill.common.expression.SchemaPath;
    +import org.apache.drill.common.expression.ValueExpressions;
    +import org.apache.drill.exec.exception.ClassTransformationException;
    +import org.apache.drill.exec.exception.OutOfMemoryException;
    +import org.apache.drill.exec.exception.SchemaChangeException;
    +import org.apache.drill.exec.expr.ClassGenerator;
    +import org.apache.drill.exec.expr.CodeGenerator;
    +import org.apache.drill.exec.expr.ExpressionTreeMaterializer;
    +import org.apache.drill.exec.expr.TypeHelper;
    +import org.apache.drill.exec.expr.ValueVectorWriteExpression;
    +import org.apache.drill.exec.ops.FragmentContext;
    +import org.apache.drill.exec.physical.config.StatisticsAggregate;
    +import org.apache.drill.exec.physical.impl.aggregate.StreamingAggBatch;
    +import org.apache.drill.exec.physical.impl.aggregate.StreamingAggTemplate;
    +import org.apache.drill.exec.physical.impl.aggregate.StreamingAggregator;
    +import org.apache.drill.exec.planner.physical.StatsAggPrel.OperatorPhase;
    +import org.apache.drill.exec.record.BatchSchema.SelectionVectorMode;
    +import org.apache.drill.exec.record.MaterializedField;
    +import org.apache.drill.exec.record.RecordBatch;
    +import org.apache.drill.exec.record.TypedFieldId;
    +import org.apache.drill.exec.store.ImplicitColumnExplorer;
    +import org.apache.drill.exec.vector.ValueVector;
    +import org.apache.drill.exec.vector.complex.FieldIdUtil;
    +import org.apache.drill.exec.vector.complex.MapVector;
    +
    +import java.io.IOException;
    +import java.util.GregorianCalendar;
    +import java.util.List;
    +import java.util.TimeZone;
    +
    +/**
    + * TODO: This needs cleanup. Currently the key values are constants and we compare the constants
    + * for every record. Seems unnecessary.
    + *
    + * Example input and output:
    + * Schema of incoming batch: region_id (VARCHAR), sales_city (VARCHAR), cnt (BIGINT)
    + * Schema of output:
    + *    "schema" : BIGINT - Schema number. For each schema change this number is incremented.
    + *    "computed" : BIGINT - What time is it computed?
    + *    "columns"       : MAP - Column names
    + *       "region_id"  : VARCHAR
    + *       "sales_city" : VARCHAR
    + *       "cnt"        : VARCHAR
    + *    "statscount" : MAP
    --- End diff --
    
    Rowcount for each column


---
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 #729: Drill 1328: Support table statistics for Parquet

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

    https://github.com/apache/drill/pull/729#discussion_r103614271
  
    --- Diff: exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/statistics/StatisticsMergeBatch.java ---
    @@ -0,0 +1,532 @@
    +/**
    + * 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.impl.statistics;
    +
    +import com.google.common.collect.Lists;
    +import com.google.common.collect.Maps;
    +import org.apache.drill.common.expression.LogicalExpression;
    +import org.apache.drill.common.expression.ErrorCollector;
    +import org.apache.drill.common.expression.ErrorCollectorImpl;
    +import org.apache.drill.common.expression.FunctionCallFactory;
    +import org.apache.drill.common.expression.SchemaPath;
    +import org.apache.drill.common.expression.ValueExpressions;
    +import org.apache.drill.common.types.TypeProtos;
    +import org.apache.drill.common.types.TypeProtos.MajorType;
    +import org.apache.drill.exec.exception.OutOfMemoryException;
    +import org.apache.drill.exec.exception.SchemaChangeException;
    +import org.apache.drill.exec.expr.ExpressionTreeMaterializer;
    +import org.apache.drill.exec.expr.TypeHelper;
    +import org.apache.drill.exec.expr.holders.BigIntHolder;
    +import org.apache.drill.exec.expr.holders.NullableFloat8Holder;
    +import org.apache.drill.exec.expr.holders.ObjectHolder;
    +import org.apache.drill.exec.expr.holders.ValueHolder;
    +import org.apache.drill.exec.ops.FragmentContext;
    +import org.apache.drill.exec.physical.config.StatisticsMerge;
    +import org.apache.drill.exec.record.AbstractSingleRecordBatch;
    +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.VectorContainer;
    +import org.apache.drill.exec.record.VectorWrapper;
    +import org.apache.drill.exec.vector.ValueVector;
    +import org.apache.drill.exec.vector.DateVector;
    +import org.apache.drill.exec.vector.BigIntVector;
    +import org.apache.drill.exec.vector.NullableBigIntVector;
    +import org.apache.drill.exec.vector.NullableFloat8Vector;
    +import org.apache.drill.exec.vector.NullableVarBinaryVector;
    +import org.apache.drill.exec.vector.VarCharVector;
    +import org.apache.drill.exec.vector.complex.MapVector;
    +
    +import java.io.ByteArrayInputStream;
    +import java.io.DataInputStream;
    +import java.io.IOException;
    +import java.util.List;
    +import java.util.GregorianCalendar;
    +import java.util.HashMap;
    +import java.util.Map;
    +import java.util.TimeZone;
    +
    +import com.clearspring.analytics.stream.cardinality.HyperLogLog;
    +
    +public class StatisticsMergeBatch extends AbstractSingleRecordBatch<StatisticsMerge> {
    +  private static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(StatisticsMergeBatch.class);
    +  private Map<String, String> functions;
    +  private boolean first = true;
    +  private boolean finished = false;
    +  private int schema = 0;
    +  private int recordCount = 0;
    +  private List<String> keyList = null;
    +  private Map<MaterializedField, ValueVector> dataSrcVecMap = null;
    +  // Map of non-map fields to VV in the incoming schema
    +  private Map<MaterializedField, ValueVector> copySrcVecMap = null;
    +  private Map<String, Map<String, ValueHolder>> aggregationMap = null;
    +  public StatisticsMergeBatch(StatisticsMerge popConfig, RecordBatch incoming,
    +                              FragmentContext context) throws OutOfMemoryException {
    +    super(popConfig, context, incoming);
    +    this.functions = new HashMap<>();
    +    this.aggregationMap = new HashMap<>();
    +
    +    /*for (String key : popConfig.getFunctions()) {
    +      aggregationMap.put(key, new HashMap<String, ValueHolder>());
    +      if (key.equalsIgnoreCase("statcount") || key.equalsIgnoreCase("nonnullstatcount")) {
    +        functions.put(key, "sum");
    +      } else if (key.equalsIgnoreCase("hll")) {
    +        functions.put(key, "hll_merge");
    +      } else if (key.equalsIgnoreCase("sum_width")) {
    +        functions.put(key, "avg_width");
    +      }
    +    }*/
    +    for (String key : popConfig.getFunctions()) {
    +      if (key.equalsIgnoreCase("sum_width")) {
    +        functions.put(key, "avg_width");
    +      } else if (key.equalsIgnoreCase("hll")) {
    +        functions.put(key, "hll_merge");
    +      } else {
    +        functions.put(key, key);
    +      }
    +      aggregationMap.put(functions.get(key), new HashMap<String, ValueHolder>());
    +    }
    +  }
    +
    +  private void createKeyColumn(String name, LogicalExpression expr, Map<MaterializedField, ValueVector> parentMap)
    +      throws SchemaChangeException {
    +    ErrorCollector collector = new ErrorCollectorImpl();
    +
    +    LogicalExpression mle = ExpressionTreeMaterializer.materialize(expr, incoming, collector,
    +        context.getFunctionRegistry());
    +
    +    MaterializedField outputField = MaterializedField.create(name, mle.getMajorType());
    +    ValueVector vector = TypeHelper.getNewVector(outputField, oContext.getAllocator());
    +    container.add(vector);
    +
    +    if (collector.hasErrors()) {
    +      throw new SchemaChangeException("Failure while materializing expression. "
    +          + collector.toErrorString());
    +    }
    +    parentMap.put(outputField, vector);
    +  }
    +
    +  private ValueVector addMapVector(String name, MapVector parent, LogicalExpression expr) throws SchemaChangeException {
    +    ErrorCollector collector = new ErrorCollectorImpl();
    +
    +    LogicalExpression mle = ExpressionTreeMaterializer.materialize(expr, incoming, collector,
    +        context.getFunctionRegistry());
    +
    +    Class<? extends ValueVector> vvc =
    +        TypeHelper.getValueVectorClass(mle.getMajorType().getMinorType(), mle.getMajorType().getMode());
    +    ValueVector vector = parent.addOrGet(name, mle.getMajorType(), vvc);
    +
    +    if (collector.hasErrors()) {
    +      throw new SchemaChangeException("Failure while materializing expression. "
    +          + collector.toErrorString());
    +    }
    +    return vector;
    +  }
    +
    +  /**
    +   * Identify the list of fields within a map which are unpivoted as columns in output
    +   */
    +  private void buildKeyList() {
    --- End diff --
    
    Renamed `buildKeyList()` to `buildColumnsList()`. Use the "columns" map and verify the rest against it. Also check that only the expected input functions appear in the incoming batch.


---
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 #729: Drill 1328: Support table statistics for Parquet

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/729#discussion_r102869874
  
    --- Diff: exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/statistics/AvgWidthMergedStatistic.java ---
    @@ -0,0 +1,135 @@
    +/*
    + * 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.impl.statistics;
    +
    +import org.apache.drill.common.types.TypeProtos;
    +import org.apache.drill.exec.expr.holders.NullableFloat8Holder;
    +import org.apache.drill.exec.expr.holders.ValueHolder;
    +import org.apache.drill.exec.vector.NullableFloat8Vector;
    +import org.apache.drill.exec.vector.ValueVector;
    +import org.apache.drill.exec.vector.complex.MapVector;
    +
    +import java.util.HashMap;
    +import java.util.List;
    +import java.util.Map;
    +
    +public class AvgWidthMergedStatistic extends AbstractMergedStatistic {
    +
    +  private String name;
    +  private String inputName;
    +  private boolean configureComplete = false;
    +  private boolean mergeComplete = false;
    +  private Map<String, ValueHolder> sumHolder;
    +  MergedStatistic types, nonNullStatCounts, statCounts;
    +
    +  public AvgWidthMergedStatistic (String name, String inputName) {
    +    this.name = name;
    +    this.inputName = inputName;
    +    this.sumHolder = new HashMap<>();
    +    types = nonNullStatCounts = statCounts = null;
    +  }
    +
    +  @Override
    +  public String getName() {
    +    return name;
    +  }
    +
    +  @Override
    +  public String getInput() {
    +    return inputName;
    +  }
    +
    +  @Override
    +  public void merge(ValueVector input) {
    +    // Check the input is a Map Vector
    +    assert (input.getField().getType().getMinorType() == TypeProtos.MinorType.MAP);
    +    MapVector inputMap = (MapVector) input;
    +    for (ValueVector vv : inputMap) {
    +      String colName = vv.getField().getLastName();
    +      NullableFloat8Holder colSumHolder;
    +      if (sumHolder.get(colName) != null) {
    +        colSumHolder = (NullableFloat8Holder) sumHolder.get(colName);
    +      } else {
    +        colSumHolder = new NullableFloat8Holder();
    +        sumHolder.put(colName, colSumHolder);
    +      }
    +      Object val = vv.getAccessor().getObject(0);
    --- End diff --
    
    This call is quite expensive: it materializes the value as a Java object, which you will unbox later. Since the only way you can get here is via creating a `NullableFloat8Holder`, we can take a step back.
    
    You really don't need a holder; they are meant for code generation. Heck, they don't even do anything useful here.
    
    What you want is the value vector itself, of the proper type.
    
    ```
      for (ValueVector vv : inputMap) {
        NullableFloat8Vector fv = (NullableFloat8Vector) vv;
        NullableFloat8Vector.Accessor accessor = fv.getAccessor();
        if (! accessor.isNull()) {
          colSumHolder.value += accessor.get(0);
          colSumHolder.isSet = 1;
        }
      }
    ```
    
    Note that you also don't need a holder for the sum. A simple double should be fine.
    
    Can the input values actually be null? Under what situation? If they can, then a simple `boolean isSet;` would handle the case of propagating null values.
    
    Using holder introduces overhead, causes more heap thrashing, and really does not buy you anything 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 #729: Drill 1328: Support table statistics for Parquet

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/729#discussion_r102874232
  
    --- Diff: exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/statistics/MergedStatisticFactory.java ---
    @@ -0,0 +1,46 @@
    +/*
    + * 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.impl.statistics;
    +
    +public class MergedStatisticFactory {
    +  /*
    +   * Creates the appropriate statistics object given the name of the statistics and the input statistic
    +   */
    +  public static MergedStatistic getMergedStatistic(String outputStatName, String inputStatName) {
    +    if (outputStatName == null || inputStatName == null) {
    +      return null;
    +    } else if (outputStatName.equals(Statistic.COLNAME)) {
    +      return new ColumnMergedStatistic(outputStatName, inputStatName);
    +    } else if (outputStatName.equals(Statistic.COLTYPE)) {
    +      return new ColTypeMergedStatistic(outputStatName, inputStatName);
    +    } else if (outputStatName.equals(Statistic.STATCOUNT)) {
    +      return new StatCountMergedStatistic(outputStatName, inputStatName);
    +    } else if (outputStatName.equals(Statistic.NNSTATCOUNT)) {
    +      return new NNStatCountMergedStatistic(outputStatName, inputStatName);
    +    } else if (outputStatName.equals(Statistic.AVG_WIDTH)) {
    +      return new AvgWidthMergedStatistic(outputStatName, inputStatName);
    +    } else if (outputStatName.equals(Statistic.HLL_MERGE)) {
    +      return new HLLMergedStatistic(outputStatName, inputStatName);
    +    } else if (outputStatName.equals(Statistic.NDV)) {
    +      return new NDVMergedStatistic(outputStatName, inputStatName);
    +    } else {
    +      return null;
    --- End diff --
    
    Is this an expected case? Or
    ```
    throw new IllegalArgumentException("No implementation for " + outputStatName);
    ```


---
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 #729: Drill 1328: Support table statistics for Parquet

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

    https://github.com/apache/drill/pull/729#discussion_r103614296
  
    --- Diff: exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/statistics/StatisticsMergeBatch.java ---
    @@ -0,0 +1,532 @@
    +/**
    + * 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.impl.statistics;
    +
    +import com.google.common.collect.Lists;
    +import com.google.common.collect.Maps;
    +import org.apache.drill.common.expression.LogicalExpression;
    +import org.apache.drill.common.expression.ErrorCollector;
    +import org.apache.drill.common.expression.ErrorCollectorImpl;
    +import org.apache.drill.common.expression.FunctionCallFactory;
    +import org.apache.drill.common.expression.SchemaPath;
    +import org.apache.drill.common.expression.ValueExpressions;
    +import org.apache.drill.common.types.TypeProtos;
    +import org.apache.drill.common.types.TypeProtos.MajorType;
    +import org.apache.drill.exec.exception.OutOfMemoryException;
    +import org.apache.drill.exec.exception.SchemaChangeException;
    +import org.apache.drill.exec.expr.ExpressionTreeMaterializer;
    +import org.apache.drill.exec.expr.TypeHelper;
    +import org.apache.drill.exec.expr.holders.BigIntHolder;
    +import org.apache.drill.exec.expr.holders.NullableFloat8Holder;
    +import org.apache.drill.exec.expr.holders.ObjectHolder;
    +import org.apache.drill.exec.expr.holders.ValueHolder;
    +import org.apache.drill.exec.ops.FragmentContext;
    +import org.apache.drill.exec.physical.config.StatisticsMerge;
    +import org.apache.drill.exec.record.AbstractSingleRecordBatch;
    +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.VectorContainer;
    +import org.apache.drill.exec.record.VectorWrapper;
    +import org.apache.drill.exec.vector.ValueVector;
    +import org.apache.drill.exec.vector.DateVector;
    +import org.apache.drill.exec.vector.BigIntVector;
    +import org.apache.drill.exec.vector.NullableBigIntVector;
    +import org.apache.drill.exec.vector.NullableFloat8Vector;
    +import org.apache.drill.exec.vector.NullableVarBinaryVector;
    +import org.apache.drill.exec.vector.VarCharVector;
    +import org.apache.drill.exec.vector.complex.MapVector;
    +
    +import java.io.ByteArrayInputStream;
    +import java.io.DataInputStream;
    +import java.io.IOException;
    +import java.util.List;
    +import java.util.GregorianCalendar;
    +import java.util.HashMap;
    +import java.util.Map;
    +import java.util.TimeZone;
    +
    +import com.clearspring.analytics.stream.cardinality.HyperLogLog;
    +
    +public class StatisticsMergeBatch extends AbstractSingleRecordBatch<StatisticsMerge> {
    +  private static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(StatisticsMergeBatch.class);
    +  private Map<String, String> functions;
    +  private boolean first = true;
    +  private boolean finished = false;
    +  private int schema = 0;
    +  private int recordCount = 0;
    +  private List<String> keyList = null;
    +  private Map<MaterializedField, ValueVector> dataSrcVecMap = null;
    +  // Map of non-map fields to VV in the incoming schema
    +  private Map<MaterializedField, ValueVector> copySrcVecMap = null;
    +  private Map<String, Map<String, ValueHolder>> aggregationMap = null;
    +  public StatisticsMergeBatch(StatisticsMerge popConfig, RecordBatch incoming,
    +                              FragmentContext context) throws OutOfMemoryException {
    +    super(popConfig, context, incoming);
    +    this.functions = new HashMap<>();
    +    this.aggregationMap = new HashMap<>();
    +
    +    /*for (String key : popConfig.getFunctions()) {
    +      aggregationMap.put(key, new HashMap<String, ValueHolder>());
    +      if (key.equalsIgnoreCase("statcount") || key.equalsIgnoreCase("nonnullstatcount")) {
    +        functions.put(key, "sum");
    +      } else if (key.equalsIgnoreCase("hll")) {
    +        functions.put(key, "hll_merge");
    +      } else if (key.equalsIgnoreCase("sum_width")) {
    +        functions.put(key, "avg_width");
    +      }
    +    }*/
    +    for (String key : popConfig.getFunctions()) {
    +      if (key.equalsIgnoreCase("sum_width")) {
    +        functions.put(key, "avg_width");
    +      } else if (key.equalsIgnoreCase("hll")) {
    +        functions.put(key, "hll_merge");
    +      } else {
    +        functions.put(key, key);
    +      }
    +      aggregationMap.put(functions.get(key), new HashMap<String, ValueHolder>());
    +    }
    +  }
    +
    +  private void createKeyColumn(String name, LogicalExpression expr, Map<MaterializedField, ValueVector> parentMap)
    +      throws SchemaChangeException {
    +    ErrorCollector collector = new ErrorCollectorImpl();
    +
    +    LogicalExpression mle = ExpressionTreeMaterializer.materialize(expr, incoming, collector,
    +        context.getFunctionRegistry());
    +
    +    MaterializedField outputField = MaterializedField.create(name, mle.getMajorType());
    +    ValueVector vector = TypeHelper.getNewVector(outputField, oContext.getAllocator());
    +    container.add(vector);
    +
    +    if (collector.hasErrors()) {
    +      throw new SchemaChangeException("Failure while materializing expression. "
    +          + collector.toErrorString());
    +    }
    +    parentMap.put(outputField, vector);
    +  }
    +
    +  private ValueVector addMapVector(String name, MapVector parent, LogicalExpression expr) throws SchemaChangeException {
    +    ErrorCollector collector = new ErrorCollectorImpl();
    +
    +    LogicalExpression mle = ExpressionTreeMaterializer.materialize(expr, incoming, collector,
    +        context.getFunctionRegistry());
    +
    +    Class<? extends ValueVector> vvc =
    +        TypeHelper.getValueVectorClass(mle.getMajorType().getMinorType(), mle.getMajorType().getMode());
    +    ValueVector vector = parent.addOrGet(name, mle.getMajorType(), vvc);
    +
    +    if (collector.hasErrors()) {
    +      throw new SchemaChangeException("Failure while materializing expression. "
    +          + collector.toErrorString());
    +    }
    +    return vector;
    +  }
    +
    +  /**
    +   * Identify the list of fields within a map which are unpivoted as columns in output
    +   */
    +  private void buildKeyList() {
    +    List<String> lastMapKeyList = null;
    +    for (VectorWrapper<?> vw : incoming) {
    +      if (vw.getField().getType().getMinorType() != TypeProtos.MinorType.MAP) {
    +        continue;
    +      }
    +
    +      keyList = Lists.newArrayList();
    +
    +      for (ValueVector vv : vw.getValueVector()) {
    +        keyList.add(vv.getField().getLastName());
    +      }
    +
    +      if (lastMapKeyList == null) {
    +        lastMapKeyList = keyList;
    +      } else {
    +        if (keyList.size() != lastMapKeyList.size() || !lastMapKeyList.containsAll(keyList)) {
    +          throw new UnsupportedOperationException("Maps have different fields");
    +        }
    +      }
    +    }
    +  }
    +
    +  private void buildOutputContainer() throws SchemaChangeException {
    +    dataSrcVecMap = Maps.newHashMap();
    +    copySrcVecMap = Maps.newHashMap();
    +    MajorType mt = null;
    +
    +    ErrorCollector collector = new ErrorCollectorImpl();
    +    GregorianCalendar calendar = new GregorianCalendar(TimeZone.getTimeZone("UTC"));
    +
    +    calendar.setTimeInMillis(System.currentTimeMillis());
    +    createKeyColumn("schema", ValueExpressions.getBigInt(schema++), copySrcVecMap);
    +    createKeyColumn("computed", ValueExpressions.getDate(calendar), copySrcVecMap);
    +
    +    for (VectorWrapper<?> vw : incoming) {
    +      addVectorToOutgoingContainer(vw.getField().getLastName(), vw, collector);
    +      /*MaterializedField ds = vw.getField();
    +      String field = vw.getField().getLastName();
    +      // Input map vector
    +      MapVector mapVector = (MapVector) vw.getValueVector();
    +      assert mapVector.getPrimitiveVectors().size() > 0;
    +      // Proceed to create output map vector with same name e.g. statcount etc.
    +      mt = mapVector.getField().getType();
    +      MaterializedField mf = MaterializedField.create(functions.get(field), mt);
    +      assert !dataSrcVecMap.containsKey(mf);
    +      ValueVector vector = TypeHelper.getNewVector(mf, oContext.getAllocator());
    +      container.add(vector);
    +      MapVector outputMapVector = (MapVector) vector;
    +
    +      for (ValueVector vv : mapVector) {
    +        String fieldName = vv.getField().getLastName();
    +        if (!keyList.contains(fieldName)) {
    +          throw new UnsupportedOperationException("Unpivot data vector " +
    +              ds + " contains key " + fieldName + " not contained in key source!");
    +        }
    +        if (vv.getField().getType().getMinorType() == TypeProtos.MinorType.MAP) {
    +          throw new UnsupportedOperationException("Unpivot of nested map is not supported!");
    +        }
    +        if (field.equals("column")) {
    +          outputMapVector.addOrGet(fieldName, vv.getField().getType(), vv.getClass());
    +        } else {
    +          List<LogicalExpression> args = Lists.newArrayList();
    +          //TODO: Something else to access value of col such as emp_id?
    +          args.add(SchemaPath.getSimplePath(vv.getField().getPath()));
    +          //TODO: Put in the mapVector
    +          LogicalExpression call = FunctionCallFactory.createExpression(functions.get(field), args);
    +          //TODO: Is this sufficient to add to new Map?
    +          ValueVector vector1 = addMapVector(fieldName, outputMapVector, call);
    +          if (collector.hasErrors()) {
    +            throw new SchemaChangeException("Failure while materializing expression. "
    +                + collector.toErrorString());
    +          }
    +        }
    +      }
    +      dataSrcVecMap.put(ds, outputMapVector);*/
    +    }
    +    //Now create NDV in the outgoing container which was not avaliable in the incoming
    +    for (VectorWrapper<?> vw : incoming) {
    +      if (vw.getField().getLastName().equalsIgnoreCase("sum_width")) {//NullableFloat8 type
    +        addVectorToOutgoingContainer("ndv", vw, collector);
    --- End diff --
    
    Refactored out


---
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 #729: Drill 1328: Support table statistics for Parquet

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

    https://github.com/apache/drill/pull/729#discussion_r100418715
  
    --- Diff: exec/java-exec/pom.xml ---
    @@ -410,6 +410,11 @@
           <artifactId>joda-time</artifactId>
           <version>2.9</version>
         </dependency>
    +    <dependency>
    +      <groupId>com.clearspring.analytics</groupId>
    +      <artifactId>stream</artifactId>
    +      <version>2.7.0</version>
    +    </dependency>
    --- End diff --
    
    It is already excluded in the jdbc-all/pom.xml. Do you mean something else? 


---
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 #729: Drill 1328: Support table statistics for Parquet

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

    https://github.com/apache/drill/pull/729#discussion_r103611021
  
    --- Diff: exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/aggregate/StreamingAggBatch.java ---
    @@ -82,11 +82,18 @@
       private boolean specialBatchSent = false;
       private static final int SPECIAL_BATCH_COUNT = 1;
     
    -  public StreamingAggBatch(StreamingAggregate popConfig, RecordBatch incoming, FragmentContext context) throws OutOfMemoryException {
    +  public StreamingAggBatch(StreamingAggregate popConfig, RecordBatch incoming, FragmentContext context)
    +      throws OutOfMemoryException {
         super(popConfig, context);
         this.incoming = incoming;
       }
     
    +  public StreamingAggBatch(StreamingAggregate popConfig, RecordBatch incoming, FragmentContext context,
    +                           final boolean buildSchema) throws OutOfMemoryException {
    --- End diff --
    
    I am not familiar with the entire inner implementation of StreamingAggBatch. @amansinha100 could you please explain why we need another buildSchema 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 #729: Drill 1328 r4

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/729#discussion_r99393301
  
    --- Diff: exec/java-exec/src/main/java/org/apache/drill/exec/ops/QueryContext.java ---
    @@ -61,6 +61,7 @@
     // in fragment contexts
     public class QueryContext implements AutoCloseable, OptimizerRulesContext, SchemaConfigInfoProvider {
       private static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(QueryContext.class);
    +  public enum StatementType {UNKNOWN, ANALYZE, CTAS, EXPLAIN, REFRESH, SELECT, SETOPTION};
    --- End diff --
    
    We don't have UNKNOWN statements. (Drill created the statement, it should know what it created.) Perhaps call this `OTHER`?
    
    Or, better, figure out what that other statement type is and include it in the list. Throw an exception if we find we don't know the statement time because, in that case, something is wrong in 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 #729: Drill 1328 r4

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/729#discussion_r99391270
  
    --- Diff: exec/java-exec/src/main/java/org/apache/drill/exec/ExecConstants.java ---
    @@ -390,4 +391,15 @@
     
       String DYNAMIC_UDF_SUPPORT_ENABLED = "exec.udf.enable_dynamic_support";
       BooleanValidator DYNAMIC_UDF_SUPPORT_ENABLED_VALIDATOR = new BooleanValidator(DYNAMIC_UDF_SUPPORT_ENABLED, true, true);
    +
    +  /**
    +   * Option whose value is a long value representing the number of bits required for computing ndv (using HLL)
    +   */
    +  LongValidator NDV_MEMORY_LIMIT = new PositiveLongValidator("exec.statistics.ndv_memory_limit", 30, 20);
    +
    +  /**
    +   * Option whose value represents the current version of the statistics. Decreasing the value will generate
    +   * the older version of statistics
    +   */
    +  LongValidator STATISTICS_VERSION = new NonNegativeLongValidator("exec.statistics.capability_version", 1, 1);
    --- End diff --
    
    The version number should be set in the code to represent the version of the code that gathered the stats, shouldn't it? What does it mean for the user to change the version number at run time? Does this allow the user to select the version of the stats collector to use? Do we support multiple versions?


---
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 #729: Drill 1328: Support table statistics for Parquet

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

    https://github.com/apache/drill/pull/729#discussion_r100452345
  
    --- Diff: exec/java-exec/src/main/java/org/apache/drill/exec/ops/QueryContext.java ---
    @@ -61,6 +61,7 @@
     // in fragment contexts
     public class QueryContext implements AutoCloseable, OptimizerRulesContext, SchemaConfigInfoProvider {
       private static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(QueryContext.class);
    +  public enum StatementType {UNKNOWN, ANALYZE, CTAS, EXPLAIN, REFRESH, SELECT, SETOPTION};
    --- End diff --
    
    There are several other SQL statement types which require additional code to figure out (Please see child classes of DefaultSqlHandler). I have renamed the UNKNOWN type to OTHER. For now, this is sufficient since there is no custom logic for these statement types e.g. VIEWs/DESCRIBEs/DROPs. If need be, we can extend the statement types and add the logic to determine if the new statement type.


---
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 #729: Drill 1328: Support table statistics for Parquet

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/729#discussion_r102872187
  
    --- Diff: exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/statistics/AvgWidthMergedStatistic.java ---
    @@ -0,0 +1,135 @@
    +/*
    + * 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.impl.statistics;
    +
    +import org.apache.drill.common.types.TypeProtos;
    +import org.apache.drill.exec.expr.holders.NullableFloat8Holder;
    +import org.apache.drill.exec.expr.holders.ValueHolder;
    +import org.apache.drill.exec.vector.NullableFloat8Vector;
    +import org.apache.drill.exec.vector.ValueVector;
    +import org.apache.drill.exec.vector.complex.MapVector;
    +
    +import java.util.HashMap;
    +import java.util.List;
    +import java.util.Map;
    +
    +public class AvgWidthMergedStatistic extends AbstractMergedStatistic {
    +
    +  private String name;
    +  private String inputName;
    +  private boolean configureComplete = false;
    +  private boolean mergeComplete = false;
    +  private Map<String, ValueHolder> sumHolder;
    +  MergedStatistic types, nonNullStatCounts, statCounts;
    +
    +  public AvgWidthMergedStatistic (String name, String inputName) {
    +    this.name = name;
    +    this.inputName = inputName;
    +    this.sumHolder = new HashMap<>();
    +    types = nonNullStatCounts = statCounts = null;
    +  }
    +
    +  @Override
    +  public String getName() {
    +    return name;
    +  }
    +
    +  @Override
    +  public String getInput() {
    +    return inputName;
    +  }
    +
    +  @Override
    +  public void merge(ValueVector input) {
    +    // Check the input is a Map Vector
    +    assert (input.getField().getType().getMinorType() == TypeProtos.MinorType.MAP);
    +    MapVector inputMap = (MapVector) input;
    +    for (ValueVector vv : inputMap) {
    +      String colName = vv.getField().getLastName();
    +      NullableFloat8Holder colSumHolder;
    +      if (sumHolder.get(colName) != null) {
    +        colSumHolder = (NullableFloat8Holder) sumHolder.get(colName);
    +      } else {
    +        colSumHolder = new NullableFloat8Holder();
    +        sumHolder.put(colName, colSumHolder);
    +      }
    +      Object val = vv.getAccessor().getObject(0);
    +      if (val != null) {
    +        colSumHolder.value += (double) val;
    +        colSumHolder.isSet = 1;
    +      }
    +    }
    +  }
    +
    +  @Override
    +  public Object getStat(String colName) {
    +      if (mergeComplete != true) {
    +        throw new IllegalStateException(
    +            String.format("Statistic `%s` has not completed merging statistics", name));
    +      }
    +      NullableFloat8Holder colSumHolder = (NullableFloat8Holder) sumHolder.get(colName);
    +      return (long) (colSumHolder.value/ getRowCount(colName));
    +    }
    +
    +  @Override
    +  public void setOutput(ValueVector output) {
    +    // Check the input is a Map Vector
    +    assert (output.getField().getType().getMinorType() == TypeProtos.MinorType.MAP);
    +    // Dependencies have been configured correctly
    +    assert (configureComplete == true);
    +    MapVector outputMap = (MapVector) output;
    +
    +    for (ValueVector outMapCol : outputMap) {
    +      String colName = outMapCol.getField().getLastName();
    +      NullableFloat8Holder colSumHolder = (NullableFloat8Holder) sumHolder.get(colName);
    +      NullableFloat8Vector vv = (NullableFloat8Vector) outMapCol;
    +      vv.allocateNewSafe();
    +      vv.getMutator().setSafe(0, (colSumHolder.value / getRowCount(colName)));
    +    }
    +    mergeComplete = true;
    +  }
    +
    +  @Override
    +  public void configure(Object configurations) {
    +    List<MergedStatistic> statistics = (List<MergedStatistic>) configurations;
    +    for (MergedStatistic statistic : statistics) {
    +      if (statistic.getName().equals("type")) {
    +        types = statistic;
    +      } else if (statistic.getName().equals("statcount")) {
    +        statCounts = statistic;
    +      } else if (statistic.getName().equals("nonnullstatcount")) {
    +        nonNullStatCounts = statistic;
    +      }
    +    }
    +    assert (types != null && statCounts != null && nonNullStatCounts != null);
    +    configureComplete = true;
    +  }
    +
    +  private long getRowCount(String colName) {
    +    int type = (int) types.getStat(colName);
    +    // If variable type - then use the nonNullCount. Otherwise, use the Count,
    +    // since even NULL values take up the same space.
    +    if (type == TypeProtos.MinorType.VAR16CHAR.getNumber()
    +        || type == TypeProtos.MinorType.VARCHAR.getNumber()
    +        || type == TypeProtos.MinorType.VARBINARY.getNumber()) {
    +      return (long) nonNullStatCounts.getStat(colName);
    +    } else {
    +      return (long) statCounts.getStat(colName);
    --- End diff --
    
    If `getStat` is typed, as noted above, then:
    ```
    return statCounts.getStat(colName);
    ```
    
    I do wonder, however if the structure can be simplified. We do many hash lookups here, but they all seem unnecessary. We have a list of known list of columns. Can columns have an index that corresponds to the map vector items, and the various collections used here? Hash map lookups are not free; doing many of them unnecessarily impacts performance.


---
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 #729: Drill 1328: Support table statistics for Parquet

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/729#discussion_r100676401
  
    --- Diff: exec/java-exec/src/main/java/org/apache/drill/exec/ExecConstants.java ---
    @@ -390,4 +391,15 @@
     
       String DYNAMIC_UDF_SUPPORT_ENABLED = "exec.udf.enable_dynamic_support";
       BooleanValidator DYNAMIC_UDF_SUPPORT_ENABLED_VALIDATOR = new BooleanValidator(DYNAMIC_UDF_SUPPORT_ENABLED, true, true);
    +
    +  /**
    +   * Option whose value is a long value representing the number of bits required for computing ndv (using HLL)
    +   */
    +  LongValidator NDV_MEMORY_LIMIT = new PositiveLongValidator("exec.statistics.ndv_memory_limit", 30, 20);
    +
    +  /**
    +   * Option whose value represents the current version of the statistics. Decreasing the value will generate
    +   * the older version of statistics
    +   */
    +  LongValidator STATISTICS_VERSION = new NonNegativeLongValidator("exec.statistics.capability_version", 1, 1);
    --- End diff --
    
    Not sure this is clear, or desirable. When the stats are computed, they use the version for the code that computes them, right? Are we saying that the user can select to use an older version of the code for computation? Or that the code has if statements to support all old versions? If so, this would be the only place in Drill to do so.
    
    On read size, doesn't the code have to use the version of code compatible with the version of the stats in the file? How can I use, say, version 2 of stats with a version 3 file?
    
    Maybe some background explanation is needed (in the spec? Somewhere in the JIRA or code?) to explain the use case.


---
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 #729: Drill 1328 r4

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/729#discussion_r99393723
  
    --- Diff: exec/java-exec/src/main/java/org/apache/drill/exec/ops/QueryContext.java ---
    @@ -283,4 +286,18 @@ public void close() throws Exception {
           closed = true;
         }
       }
    +
    +  /**
    +  * @param stmtType : The type {@link StatementType} e.g. CTAS, ANALYZE of the statement
    +  */
    +  public void setStatementType(StatementType stmtType) {
    --- End diff --
    
    See comment above. We probably don't want to set the type this way as it suggests that the type can change over 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 #729: Drill 1328: Support table statistics for Parquet

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

    https://github.com/apache/drill/pull/729#discussion_r103405690
  
    --- Diff: exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/statistics/AvgWidthMergedStatistic.java ---
    @@ -0,0 +1,135 @@
    +/*
    + * 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.impl.statistics;
    +
    +import org.apache.drill.common.types.TypeProtos;
    +import org.apache.drill.exec.expr.holders.NullableFloat8Holder;
    +import org.apache.drill.exec.expr.holders.ValueHolder;
    +import org.apache.drill.exec.vector.NullableFloat8Vector;
    +import org.apache.drill.exec.vector.ValueVector;
    +import org.apache.drill.exec.vector.complex.MapVector;
    +
    +import java.util.HashMap;
    +import java.util.List;
    +import java.util.Map;
    +
    +public class AvgWidthMergedStatistic extends AbstractMergedStatistic {
    +
    +  private String name;
    +  private String inputName;
    +  private boolean configureComplete = false;
    +  private boolean mergeComplete = false;
    +  private Map<String, ValueHolder> sumHolder;
    +  MergedStatistic types, nonNullStatCounts, statCounts;
    +
    +  public AvgWidthMergedStatistic (String name, String inputName) {
    +    this.name = name;
    +    this.inputName = inputName;
    +    this.sumHolder = new HashMap<>();
    +    types = nonNullStatCounts = statCounts = null;
    +  }
    +
    +  @Override
    +  public String getName() {
    +    return name;
    +  }
    +
    +  @Override
    +  public String getInput() {
    +    return inputName;
    +  }
    +
    +  @Override
    +  public void merge(ValueVector input) {
    +    // Check the input is a Map Vector
    +    assert (input.getField().getType().getMinorType() == TypeProtos.MinorType.MAP);
    +    MapVector inputMap = (MapVector) input;
    +    for (ValueVector vv : inputMap) {
    +      String colName = vv.getField().getLastName();
    +      NullableFloat8Holder colSumHolder;
    +      if (sumHolder.get(colName) != null) {
    +        colSumHolder = (NullableFloat8Holder) sumHolder.get(colName);
    +      } else {
    +        colSumHolder = new NullableFloat8Holder();
    +        sumHolder.put(colName, colSumHolder);
    +      }
    +      Object val = vv.getAccessor().getObject(0);
    +      if (val != null) {
    +        colSumHolder.value += (double) val;
    +        colSumHolder.isSet = 1;
    +      }
    +    }
    +  }
    +
    +  @Override
    +  public Object getStat(String colName) {
    +      if (mergeComplete != true) {
    +        throw new IllegalStateException(
    +            String.format("Statistic `%s` has not completed merging statistics", name));
    +      }
    +      NullableFloat8Holder colSumHolder = (NullableFloat8Holder) sumHolder.get(colName);
    +      return (long) (colSumHolder.value/ getRowCount(colName));
    +    }
    +
    +  @Override
    +  public void setOutput(ValueVector output) {
    +    // Check the input is a Map Vector
    +    assert (output.getField().getType().getMinorType() == TypeProtos.MinorType.MAP);
    +    // Dependencies have been configured correctly
    +    assert (configureComplete == true);
    +    MapVector outputMap = (MapVector) output;
    +
    +    for (ValueVector outMapCol : outputMap) {
    +      String colName = outMapCol.getField().getLastName();
    +      NullableFloat8Holder colSumHolder = (NullableFloat8Holder) sumHolder.get(colName);
    +      NullableFloat8Vector vv = (NullableFloat8Vector) outMapCol;
    +      vv.allocateNewSafe();
    +      vv.getMutator().setSafe(0, (colSumHolder.value / getRowCount(colName)));
    +    }
    +    mergeComplete = true;
    +  }
    +
    +  @Override
    +  public void configure(Object configurations) {
    +    List<MergedStatistic> statistics = (List<MergedStatistic>) configurations;
    +    for (MergedStatistic statistic : statistics) {
    +      if (statistic.getName().equals("type")) {
    +        types = statistic;
    +      } else if (statistic.getName().equals("statcount")) {
    +        statCounts = statistic;
    +      } else if (statistic.getName().equals("nonnullstatcount")) {
    +        nonNullStatCounts = statistic;
    +      }
    +    }
    +    assert (types != null && statCounts != null && nonNullStatCounts != null);
    +    configureComplete = true;
    +  }
    +
    +  private long getRowCount(String colName) {
    +    int type = (int) types.getStat(colName);
    --- End diff --
    
    This does not seem to work as you suggested. Included `TypeProtos.MinorType`


---
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 #729: Drill 1328: Support table statistics for Parquet

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

    https://github.com/apache/drill/pull/729#discussion_r100708170
  
    --- Diff: exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/statistics/StatisticsAggBatch.java ---
    @@ -0,0 +1,256 @@
    +/**
    + * 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.impl.statistics;
    +
    +import com.google.common.collect.Lists;
    +import com.sun.codemodel.JExpr;
    +import org.apache.drill.common.expression.ErrorCollector;
    +import org.apache.drill.common.expression.ErrorCollectorImpl;
    +import org.apache.drill.common.expression.FunctionCallFactory;
    +import org.apache.drill.common.expression.LogicalExpression;
    +import org.apache.drill.common.expression.SchemaPath;
    +import org.apache.drill.common.expression.ValueExpressions;
    +import org.apache.drill.exec.exception.ClassTransformationException;
    +import org.apache.drill.exec.exception.OutOfMemoryException;
    +import org.apache.drill.exec.exception.SchemaChangeException;
    +import org.apache.drill.exec.expr.ClassGenerator;
    +import org.apache.drill.exec.expr.CodeGenerator;
    +import org.apache.drill.exec.expr.ExpressionTreeMaterializer;
    +import org.apache.drill.exec.expr.TypeHelper;
    +import org.apache.drill.exec.expr.ValueVectorWriteExpression;
    +import org.apache.drill.exec.ops.FragmentContext;
    +import org.apache.drill.exec.physical.config.StatisticsAggregate;
    +import org.apache.drill.exec.physical.impl.aggregate.StreamingAggBatch;
    +import org.apache.drill.exec.physical.impl.aggregate.StreamingAggTemplate;
    +import org.apache.drill.exec.physical.impl.aggregate.StreamingAggregator;
    +import org.apache.drill.exec.planner.physical.StatsAggPrel.OperatorPhase;
    +import org.apache.drill.exec.record.BatchSchema.SelectionVectorMode;
    +import org.apache.drill.exec.record.MaterializedField;
    +import org.apache.drill.exec.record.RecordBatch;
    +import org.apache.drill.exec.record.TypedFieldId;
    +import org.apache.drill.exec.store.ImplicitColumnExplorer;
    +import org.apache.drill.exec.vector.ValueVector;
    +import org.apache.drill.exec.vector.complex.FieldIdUtil;
    +import org.apache.drill.exec.vector.complex.MapVector;
    +
    +import java.io.IOException;
    +import java.util.GregorianCalendar;
    +import java.util.List;
    +import java.util.TimeZone;
    +
    +/**
    + * TODO: This needs cleanup. Currently the key values are constants and we compare the constants
    + * for every record. Seems unnecessary.
    + *
    + * Example input and output:
    + * Schema of incoming batch: region_id (VARCHAR), sales_city (VARCHAR), cnt (BIGINT)
    + * Schema of output:
    + *    "schema" : BIGINT - Schema number. For each schema change this number is incremented.
    + *    "computed" : BIGINT - What time is it computed?
    + *    "columns"       : MAP - Column names
    + *       "region_id"  : VARCHAR
    + *       "sales_city" : VARCHAR
    + *       "cnt"        : VARCHAR
    + *    "statscount" : MAP
    + *       "region_id"  : BIGINT - statscount(region_id) - aggregation over all values of region_id
    + *                      in incoming batch
    + *       "sales_city" : BIGINT - statscount(sales_city)
    + *       "cnt"        : BIGINT - statscount(cnt)
    + *    "nonnullstatcount" : MAP
    + *       "region_id"  : BIGINT - nonnullstatcount(region_id)
    + *       "sales_city" : BIGINT - nonnullstatcount(sales_city)
    + *       "cnt"        : BIGINT - nonnullstatcount(cnt)
    + *   .... another map for next stats function ....
    + */
    +public class StatisticsAggBatch extends StreamingAggBatch {
    +  private List<String> functions;
    +  private int schema = 0;
    +
    +  public StatisticsAggBatch(StatisticsAggregate popConfig, RecordBatch incoming,
    +      FragmentContext context) throws OutOfMemoryException {
    +    super(popConfig, incoming, context);
    +    this.functions = popConfig.getFunctions();
    +  }
    +
    +  private void createKeyColumn(String name, LogicalExpression expr, List<LogicalExpression> keyExprs,
    +      List<TypedFieldId> keyOutputIds) throws SchemaChangeException {
    +    ErrorCollector collector = new ErrorCollectorImpl();
    +
    +    LogicalExpression mle = ExpressionTreeMaterializer.materialize(expr, incoming, collector,
    +        context.getFunctionRegistry());
    +
    +    MaterializedField outputField = MaterializedField.create(name, mle.getMajorType());
    +    ValueVector vector = TypeHelper.getNewVector(outputField, oContext.getAllocator());
    +
    +    keyExprs.add(mle);
    +    keyOutputIds.add(container.add(vector));
    +
    +    if (collector.hasErrors()) {
    --- End diff --
    
    Moved it earlier as suggested. However, the ErrorCollector collects ExpressionValidationError which is not subclassed from error or exception classes. We would need to subclass it and generate the equivalent error/exception in order to re-throw it.


---
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 #729: Drill 1328: Support table statistics for Parquet

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/729#discussion_r100677095
  
    --- Diff: exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/statistics/StatisticsAggBatch.java ---
    @@ -0,0 +1,256 @@
    +/**
    + * 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.impl.statistics;
    +
    +import com.google.common.collect.Lists;
    +import com.sun.codemodel.JExpr;
    +import org.apache.drill.common.expression.ErrorCollector;
    +import org.apache.drill.common.expression.ErrorCollectorImpl;
    +import org.apache.drill.common.expression.FunctionCallFactory;
    +import org.apache.drill.common.expression.LogicalExpression;
    +import org.apache.drill.common.expression.SchemaPath;
    +import org.apache.drill.common.expression.ValueExpressions;
    +import org.apache.drill.exec.exception.ClassTransformationException;
    +import org.apache.drill.exec.exception.OutOfMemoryException;
    +import org.apache.drill.exec.exception.SchemaChangeException;
    +import org.apache.drill.exec.expr.ClassGenerator;
    +import org.apache.drill.exec.expr.CodeGenerator;
    +import org.apache.drill.exec.expr.ExpressionTreeMaterializer;
    +import org.apache.drill.exec.expr.TypeHelper;
    +import org.apache.drill.exec.expr.ValueVectorWriteExpression;
    +import org.apache.drill.exec.ops.FragmentContext;
    +import org.apache.drill.exec.physical.config.StatisticsAggregate;
    +import org.apache.drill.exec.physical.impl.aggregate.StreamingAggBatch;
    +import org.apache.drill.exec.physical.impl.aggregate.StreamingAggTemplate;
    +import org.apache.drill.exec.physical.impl.aggregate.StreamingAggregator;
    +import org.apache.drill.exec.planner.physical.StatsAggPrel.OperatorPhase;
    +import org.apache.drill.exec.record.BatchSchema.SelectionVectorMode;
    +import org.apache.drill.exec.record.MaterializedField;
    +import org.apache.drill.exec.record.RecordBatch;
    +import org.apache.drill.exec.record.TypedFieldId;
    +import org.apache.drill.exec.store.ImplicitColumnExplorer;
    +import org.apache.drill.exec.vector.ValueVector;
    +import org.apache.drill.exec.vector.complex.FieldIdUtil;
    +import org.apache.drill.exec.vector.complex.MapVector;
    +
    +import java.io.IOException;
    +import java.util.GregorianCalendar;
    +import java.util.List;
    +import java.util.TimeZone;
    +
    +/**
    + * TODO: This needs cleanup. Currently the key values are constants and we compare the constants
    + * for every record. Seems unnecessary.
    + *
    + * Example input and output:
    + * Schema of incoming batch: region_id (VARCHAR), sales_city (VARCHAR), cnt (BIGINT)
    + * Schema of output:
    + *    "schema" : BIGINT - Schema number. For each schema change this number is incremented.
    + *    "computed" : BIGINT - What time is it computed?
    + *    "columns"       : MAP - Column names
    + *       "region_id"  : VARCHAR
    + *       "sales_city" : VARCHAR
    + *       "cnt"        : VARCHAR
    + *    "statscount" : MAP
    + *       "region_id"  : BIGINT - statscount(region_id) - aggregation over all values of region_id
    + *                      in incoming batch
    + *       "sales_city" : BIGINT - statscount(sales_city)
    + *       "cnt"        : BIGINT - statscount(cnt)
    + *    "nonnullstatcount" : MAP
    + *       "region_id"  : BIGINT - nonnullstatcount(region_id)
    + *       "sales_city" : BIGINT - nonnullstatcount(sales_city)
    + *       "cnt"        : BIGINT - nonnullstatcount(cnt)
    + *   .... another map for next stats function ....
    + */
    +public class StatisticsAggBatch extends StreamingAggBatch {
    +  private List<String> functions;
    +  private int schema = 0;
    +
    +  public StatisticsAggBatch(StatisticsAggregate popConfig, RecordBatch incoming,
    +      FragmentContext context) throws OutOfMemoryException {
    +    super(popConfig, incoming, context);
    +    this.functions = popConfig.getFunctions();
    +  }
    +
    --- End diff --
    
    Maybe some description of what these methods 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 #729: Drill 1328: Support table statistics for Parquet

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

    https://github.com/apache/drill/pull/729#discussion_r100708181
  
    --- Diff: exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/statistics/StatisticsAggBatch.java ---
    @@ -0,0 +1,256 @@
    +/**
    + * 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.impl.statistics;
    +
    +import com.google.common.collect.Lists;
    +import com.sun.codemodel.JExpr;
    +import org.apache.drill.common.expression.ErrorCollector;
    +import org.apache.drill.common.expression.ErrorCollectorImpl;
    +import org.apache.drill.common.expression.FunctionCallFactory;
    +import org.apache.drill.common.expression.LogicalExpression;
    +import org.apache.drill.common.expression.SchemaPath;
    +import org.apache.drill.common.expression.ValueExpressions;
    +import org.apache.drill.exec.exception.ClassTransformationException;
    +import org.apache.drill.exec.exception.OutOfMemoryException;
    +import org.apache.drill.exec.exception.SchemaChangeException;
    +import org.apache.drill.exec.expr.ClassGenerator;
    +import org.apache.drill.exec.expr.CodeGenerator;
    +import org.apache.drill.exec.expr.ExpressionTreeMaterializer;
    +import org.apache.drill.exec.expr.TypeHelper;
    +import org.apache.drill.exec.expr.ValueVectorWriteExpression;
    +import org.apache.drill.exec.ops.FragmentContext;
    +import org.apache.drill.exec.physical.config.StatisticsAggregate;
    +import org.apache.drill.exec.physical.impl.aggregate.StreamingAggBatch;
    +import org.apache.drill.exec.physical.impl.aggregate.StreamingAggTemplate;
    +import org.apache.drill.exec.physical.impl.aggregate.StreamingAggregator;
    +import org.apache.drill.exec.planner.physical.StatsAggPrel.OperatorPhase;
    +import org.apache.drill.exec.record.BatchSchema.SelectionVectorMode;
    +import org.apache.drill.exec.record.MaterializedField;
    +import org.apache.drill.exec.record.RecordBatch;
    +import org.apache.drill.exec.record.TypedFieldId;
    +import org.apache.drill.exec.store.ImplicitColumnExplorer;
    +import org.apache.drill.exec.vector.ValueVector;
    +import org.apache.drill.exec.vector.complex.FieldIdUtil;
    +import org.apache.drill.exec.vector.complex.MapVector;
    +
    +import java.io.IOException;
    +import java.util.GregorianCalendar;
    +import java.util.List;
    +import java.util.TimeZone;
    +
    +/**
    + * TODO: This needs cleanup. Currently the key values are constants and we compare the constants
    + * for every record. Seems unnecessary.
    + *
    + * Example input and output:
    + * Schema of incoming batch: region_id (VARCHAR), sales_city (VARCHAR), cnt (BIGINT)
    + * Schema of output:
    + *    "schema" : BIGINT - Schema number. For each schema change this number is incremented.
    + *    "computed" : BIGINT - What time is it computed?
    + *    "columns"       : MAP - Column names
    + *       "region_id"  : VARCHAR
    + *       "sales_city" : VARCHAR
    + *       "cnt"        : VARCHAR
    + *    "statscount" : MAP
    + *       "region_id"  : BIGINT - statscount(region_id) - aggregation over all values of region_id
    + *                      in incoming batch
    + *       "sales_city" : BIGINT - statscount(sales_city)
    + *       "cnt"        : BIGINT - statscount(cnt)
    + *    "nonnullstatcount" : MAP
    + *       "region_id"  : BIGINT - nonnullstatcount(region_id)
    + *       "sales_city" : BIGINT - nonnullstatcount(sales_city)
    + *       "cnt"        : BIGINT - nonnullstatcount(cnt)
    + *   .... another map for next stats function ....
    + */
    +public class StatisticsAggBatch extends StreamingAggBatch {
    +  private List<String> functions;
    +  private int schema = 0;
    +
    +  public StatisticsAggBatch(StatisticsAggregate popConfig, RecordBatch incoming,
    +      FragmentContext context) throws OutOfMemoryException {
    +    super(popConfig, incoming, context);
    +    this.functions = popConfig.getFunctions();
    +  }
    +
    +  private void createKeyColumn(String name, LogicalExpression expr, List<LogicalExpression> keyExprs,
    +      List<TypedFieldId> keyOutputIds) throws SchemaChangeException {
    +    ErrorCollector collector = new ErrorCollectorImpl();
    +
    +    LogicalExpression mle = ExpressionTreeMaterializer.materialize(expr, incoming, collector,
    +        context.getFunctionRegistry());
    +
    +    MaterializedField outputField = MaterializedField.create(name, mle.getMajorType());
    +    ValueVector vector = TypeHelper.getNewVector(outputField, oContext.getAllocator());
    +
    +    keyExprs.add(mle);
    +    keyOutputIds.add(container.add(vector));
    +
    +    if (collector.hasErrors()) {
    +      throw new SchemaChangeException("Failure while materializing expression. "
    +          + collector.toErrorString());
    +    }
    +  }
    +
    +  private void createNestedKeyColumn(MapVector parent, String name, LogicalExpression expr,
    +      List<LogicalExpression> keyExprs, List<TypedFieldId> keyOutputIds)
    +          throws SchemaChangeException {
    +    ErrorCollector collector = new ErrorCollectorImpl();
    +
    +    LogicalExpression mle = ExpressionTreeMaterializer.materialize(expr, incoming, collector,
    +        context.getFunctionRegistry());
    +
    +    Class<? extends ValueVector> vvc =
    +        TypeHelper.getValueVectorClass(mle.getMajorType().getMinorType(),
    +            mle.getMajorType().getMode());
    +
    +    ValueVector vv = parent.addOrGet(name, mle.getMajorType(), vvc);
    +
    +    TypedFieldId pfid = container.getValueVectorId(SchemaPath.getSimplePath(parent.getField().getPath()));
    +    assert pfid.getFieldIds().length == 1;
    +    TypedFieldId.Builder builder = TypedFieldId.newBuilder();
    +    builder.addId(pfid.getFieldIds()[0]);
    +    TypedFieldId id =
    +        FieldIdUtil.getFieldIdIfMatches(parent, builder, true,
    +            SchemaPath.getSimplePath(vv.getField().getPath()).getRootSegment());
    +
    +    keyExprs.add(mle);
    +    keyOutputIds.add(id);
    +
    +    if (collector.hasErrors()) {
    +      throw new SchemaChangeException("Failure while materializing expression. "
    +          + collector.toErrorString());
    +    }
    +  }
    +
    +  private void addMapVector(String name, MapVector parent, LogicalExpression expr,
    +      List<LogicalExpression> valueExprs) throws SchemaChangeException {
    +    ErrorCollector collector = new ErrorCollectorImpl();
    +
    +    LogicalExpression mle = ExpressionTreeMaterializer.materialize(expr, incoming, collector,
    +        context.getFunctionRegistry());
    +
    +    Class<? extends ValueVector> vvc =
    +        TypeHelper.getValueVectorClass(mle.getMajorType().getMinorType(), mle.getMajorType().getMode());
    +    ValueVector vv = parent.addOrGet(name, mle.getMajorType(), vvc);
    +
    +    TypedFieldId pfid = container.getValueVectorId(SchemaPath.getSimplePath(parent.getField().getPath()));
    +    assert pfid.getFieldIds().length == 1;
    +    TypedFieldId.Builder builder = TypedFieldId.newBuilder();
    +    builder.addId(pfid.getFieldIds()[0]);
    +    TypedFieldId id = FieldIdUtil.getFieldIdIfMatches(parent, builder, true,
    +        SchemaPath.getSimplePath(vv.getField().getPath()).getRootSegment());
    +
    +    valueExprs.add(new ValueVectorWriteExpression(id, mle, true));
    +
    +    if (collector.hasErrors()) {
    --- 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 #729: Drill 1328: Support table statistics for Parquet

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/729#discussion_r103060381
  
    --- Diff: exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/statistics/NNStatCountMergedStatistic.java ---
    @@ -0,0 +1,98 @@
    +/*
    + * 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.impl.statistics;
    +
    +import org.apache.drill.common.types.TypeProtos;
    +import org.apache.drill.exec.expr.holders.BigIntHolder;
    +import org.apache.drill.exec.expr.holders.ValueHolder;
    +import org.apache.drill.exec.vector.NullableBigIntVector;
    +import org.apache.drill.exec.vector.ValueVector;
    +import org.apache.drill.exec.vector.complex.MapVector;
    +
    +import java.util.HashMap;
    +import java.util.Map;
    +
    +public class NNStatCountMergedStatistic extends AbstractMergedStatistic {
    +
    +  private String name;
    +  private String inputName;
    +  private boolean mergeComplete = false;
    +  private Map<String, ValueHolder> sumHolder;
    +
    +  public NNStatCountMergedStatistic (String name, String inputName) {
    +    this.name = name;
    +    this.inputName = inputName;
    +    this.sumHolder = new HashMap<>();
    +  }
    +
    +  @Override
    +  public String getName() {
    +      return name;
    +  }
    +
    +  @Override
    +  public String getInput() {
    +    return inputName;
    +  }
    +
    +  @Override
    +  public void merge(ValueVector input) {
    +    // Check the input is a Map Vector
    +    assert (input.getField().getType().getMinorType() == TypeProtos.MinorType.MAP);
    +    MapVector inputMap = (MapVector) input;
    +    for (ValueVector vv : inputMap) {
    +      String colName = vv.getField().getLastName();
    +      BigIntHolder colSumHolder;
    +      if (sumHolder.get(colName) != null) {
    +        colSumHolder = (BigIntHolder) sumHolder.get(colName);
    +      } else {
    +        colSumHolder = new BigIntHolder();
    +        sumHolder.put(colName, colSumHolder);
    +      }
    +      Object val = vv.getAccessor().getObject(0);
    +      if (val != null) {
    +        colSumHolder.value += (long) val;
    +      }
    +    }
    +  }
    +
    +  @Override
    +  public Object getStat(String colName) {
    +    if (mergeComplete != true) {
    +      throw new IllegalStateException(String.format("Statistic `%s` has not completed merging statistics",
    +          name));
    +    }
    +    BigIntHolder colSumHolder = (BigIntHolder) sumHolder.get(colName);
    +    return colSumHolder.value;
    +  }
    +
    +  @Override
    +  public void setOutput(ValueVector output) {
    +    // Check the input is a Map Vector
    +    assert (output.getField().getType().getMinorType() == TypeProtos.MinorType.MAP);
    +    MapVector outputMap = (MapVector) output;
    +    for (ValueVector outMapCol : outputMap) {
    +      String colName = outMapCol.getField().getLastName();
    +      BigIntHolder holder = (BigIntHolder) sumHolder.get(colName);
    +      NullableBigIntVector vv = (NullableBigIntVector) outMapCol;
    +      vv.allocateNewSafe();
    +      vv.getMutator().setSafe(0, holder);
    +    }
    +    mergeComplete = true;
    --- End diff --
    
    The bulk of the code here is copy & paste across implementations. Any way to factor out the common code? The part that seems unique is the type-specific set of the mutator on line 94. 


---
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 #729: Drill 1328: Support table statistics for Parquet

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

    https://github.com/apache/drill/pull/729#discussion_r102324596
  
    --- Diff: exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/statistics/StatisticsMergeBatch.java ---
    @@ -0,0 +1,532 @@
    +/**
    + * 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.impl.statistics;
    +
    +import com.google.common.collect.Lists;
    +import com.google.common.collect.Maps;
    +import org.apache.drill.common.expression.LogicalExpression;
    +import org.apache.drill.common.expression.ErrorCollector;
    +import org.apache.drill.common.expression.ErrorCollectorImpl;
    +import org.apache.drill.common.expression.FunctionCallFactory;
    +import org.apache.drill.common.expression.SchemaPath;
    +import org.apache.drill.common.expression.ValueExpressions;
    +import org.apache.drill.common.types.TypeProtos;
    +import org.apache.drill.common.types.TypeProtos.MajorType;
    +import org.apache.drill.exec.exception.OutOfMemoryException;
    +import org.apache.drill.exec.exception.SchemaChangeException;
    +import org.apache.drill.exec.expr.ExpressionTreeMaterializer;
    +import org.apache.drill.exec.expr.TypeHelper;
    +import org.apache.drill.exec.expr.holders.BigIntHolder;
    +import org.apache.drill.exec.expr.holders.NullableFloat8Holder;
    +import org.apache.drill.exec.expr.holders.ObjectHolder;
    +import org.apache.drill.exec.expr.holders.ValueHolder;
    +import org.apache.drill.exec.ops.FragmentContext;
    +import org.apache.drill.exec.physical.config.StatisticsMerge;
    +import org.apache.drill.exec.record.AbstractSingleRecordBatch;
    +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.VectorContainer;
    +import org.apache.drill.exec.record.VectorWrapper;
    +import org.apache.drill.exec.vector.ValueVector;
    +import org.apache.drill.exec.vector.DateVector;
    +import org.apache.drill.exec.vector.BigIntVector;
    +import org.apache.drill.exec.vector.NullableBigIntVector;
    +import org.apache.drill.exec.vector.NullableFloat8Vector;
    +import org.apache.drill.exec.vector.NullableVarBinaryVector;
    +import org.apache.drill.exec.vector.VarCharVector;
    +import org.apache.drill.exec.vector.complex.MapVector;
    +
    +import java.io.ByteArrayInputStream;
    +import java.io.DataInputStream;
    +import java.io.IOException;
    +import java.util.List;
    +import java.util.GregorianCalendar;
    +import java.util.HashMap;
    +import java.util.Map;
    +import java.util.TimeZone;
    +
    +import com.clearspring.analytics.stream.cardinality.HyperLogLog;
    +
    +public class StatisticsMergeBatch extends AbstractSingleRecordBatch<StatisticsMerge> {
    +  private static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(StatisticsMergeBatch.class);
    +  private Map<String, String> functions;
    +  private boolean first = true;
    +  private boolean finished = false;
    +  private int schema = 0;
    +  private int recordCount = 0;
    +  private List<String> keyList = null;
    +  private Map<MaterializedField, ValueVector> dataSrcVecMap = null;
    +  // Map of non-map fields to VV in the incoming schema
    +  private Map<MaterializedField, ValueVector> copySrcVecMap = null;
    +  private Map<String, Map<String, ValueHolder>> aggregationMap = null;
    +  public StatisticsMergeBatch(StatisticsMerge popConfig, RecordBatch incoming,
    +                              FragmentContext context) throws OutOfMemoryException {
    +    super(popConfig, context, incoming);
    +    this.functions = new HashMap<>();
    +    this.aggregationMap = new HashMap<>();
    +
    +    /*for (String key : popConfig.getFunctions()) {
    +      aggregationMap.put(key, new HashMap<String, ValueHolder>());
    +      if (key.equalsIgnoreCase("statcount") || key.equalsIgnoreCase("nonnullstatcount")) {
    +        functions.put(key, "sum");
    +      } else if (key.equalsIgnoreCase("hll")) {
    +        functions.put(key, "hll_merge");
    +      } else if (key.equalsIgnoreCase("sum_width")) {
    +        functions.put(key, "avg_width");
    +      }
    +    }*/
    +    for (String key : popConfig.getFunctions()) {
    +      if (key.equalsIgnoreCase("sum_width")) {
    +        functions.put(key, "avg_width");
    +      } else if (key.equalsIgnoreCase("hll")) {
    +        functions.put(key, "hll_merge");
    +      } else {
    +        functions.put(key, key);
    +      }
    +      aggregationMap.put(functions.get(key), new HashMap<String, ValueHolder>());
    +    }
    +  }
    +
    +  private void createKeyColumn(String name, LogicalExpression expr, Map<MaterializedField, ValueVector> parentMap)
    +      throws SchemaChangeException {
    +    ErrorCollector collector = new ErrorCollectorImpl();
    +
    +    LogicalExpression mle = ExpressionTreeMaterializer.materialize(expr, incoming, collector,
    +        context.getFunctionRegistry());
    +
    +    MaterializedField outputField = MaterializedField.create(name, mle.getMajorType());
    +    ValueVector vector = TypeHelper.getNewVector(outputField, oContext.getAllocator());
    +    container.add(vector);
    +
    +    if (collector.hasErrors()) {
    +      throw new SchemaChangeException("Failure while materializing expression. "
    +          + collector.toErrorString());
    +    }
    +    parentMap.put(outputField, vector);
    +  }
    +
    +  private ValueVector addMapVector(String name, MapVector parent, LogicalExpression expr) throws SchemaChangeException {
    +    ErrorCollector collector = new ErrorCollectorImpl();
    +
    +    LogicalExpression mle = ExpressionTreeMaterializer.materialize(expr, incoming, collector,
    +        context.getFunctionRegistry());
    +
    +    Class<? extends ValueVector> vvc =
    +        TypeHelper.getValueVectorClass(mle.getMajorType().getMinorType(), mle.getMajorType().getMode());
    +    ValueVector vector = parent.addOrGet(name, mle.getMajorType(), vvc);
    +
    +    if (collector.hasErrors()) {
    +      throw new SchemaChangeException("Failure while materializing expression. "
    +          + collector.toErrorString());
    +    }
    +    return vector;
    +  }
    +
    +  /**
    +   * Identify the list of fields within a map which are unpivoted as columns in output
    +   */
    +  private void buildKeyList() {
    +    List<String> lastMapKeyList = null;
    +    for (VectorWrapper<?> vw : incoming) {
    +      if (vw.getField().getType().getMinorType() != TypeProtos.MinorType.MAP) {
    +        continue;
    +      }
    +
    +      keyList = Lists.newArrayList();
    +
    +      for (ValueVector vv : vw.getValueVector()) {
    +        keyList.add(vv.getField().getLastName());
    +      }
    +
    +      if (lastMapKeyList == null) {
    +        lastMapKeyList = keyList;
    +      } else {
    +        if (keyList.size() != lastMapKeyList.size() || !lastMapKeyList.containsAll(keyList)) {
    +          throw new UnsupportedOperationException("Maps have different fields");
    +        }
    +      }
    +    }
    +  }
    +
    +  private void buildOutputContainer() throws SchemaChangeException {
    +    dataSrcVecMap = Maps.newHashMap();
    +    copySrcVecMap = Maps.newHashMap();
    +    MajorType mt = null;
    +
    +    ErrorCollector collector = new ErrorCollectorImpl();
    +    GregorianCalendar calendar = new GregorianCalendar(TimeZone.getTimeZone("UTC"));
    +
    +    calendar.setTimeInMillis(System.currentTimeMillis());
    +    createKeyColumn("schema", ValueExpressions.getBigInt(schema++), copySrcVecMap);
    +    createKeyColumn("computed", ValueExpressions.getDate(calendar), copySrcVecMap);
    +
    +    for (VectorWrapper<?> vw : incoming) {
    +      addVectorToOutgoingContainer(vw.getField().getLastName(), vw, collector);
    --- End diff --
    
    No, we just use the input vector to obtain the MajorType and column information. We create a new output vector


---
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 #729: Drill 1328: Support table statistics for Parquet

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/729#discussion_r100677041
  
    --- Diff: exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/statistics/StatisticsAggBatch.java ---
    @@ -0,0 +1,256 @@
    +/**
    + * 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.impl.statistics;
    +
    +import com.google.common.collect.Lists;
    +import com.sun.codemodel.JExpr;
    +import org.apache.drill.common.expression.ErrorCollector;
    +import org.apache.drill.common.expression.ErrorCollectorImpl;
    +import org.apache.drill.common.expression.FunctionCallFactory;
    +import org.apache.drill.common.expression.LogicalExpression;
    +import org.apache.drill.common.expression.SchemaPath;
    +import org.apache.drill.common.expression.ValueExpressions;
    +import org.apache.drill.exec.exception.ClassTransformationException;
    +import org.apache.drill.exec.exception.OutOfMemoryException;
    +import org.apache.drill.exec.exception.SchemaChangeException;
    +import org.apache.drill.exec.expr.ClassGenerator;
    +import org.apache.drill.exec.expr.CodeGenerator;
    +import org.apache.drill.exec.expr.ExpressionTreeMaterializer;
    +import org.apache.drill.exec.expr.TypeHelper;
    +import org.apache.drill.exec.expr.ValueVectorWriteExpression;
    +import org.apache.drill.exec.ops.FragmentContext;
    +import org.apache.drill.exec.physical.config.StatisticsAggregate;
    +import org.apache.drill.exec.physical.impl.aggregate.StreamingAggBatch;
    +import org.apache.drill.exec.physical.impl.aggregate.StreamingAggTemplate;
    +import org.apache.drill.exec.physical.impl.aggregate.StreamingAggregator;
    +import org.apache.drill.exec.planner.physical.StatsAggPrel.OperatorPhase;
    +import org.apache.drill.exec.record.BatchSchema.SelectionVectorMode;
    +import org.apache.drill.exec.record.MaterializedField;
    +import org.apache.drill.exec.record.RecordBatch;
    +import org.apache.drill.exec.record.TypedFieldId;
    +import org.apache.drill.exec.store.ImplicitColumnExplorer;
    +import org.apache.drill.exec.vector.ValueVector;
    +import org.apache.drill.exec.vector.complex.FieldIdUtil;
    +import org.apache.drill.exec.vector.complex.MapVector;
    +
    +import java.io.IOException;
    +import java.util.GregorianCalendar;
    +import java.util.List;
    +import java.util.TimeZone;
    +
    +/**
    + * TODO: This needs cleanup. Currently the key values are constants and we compare the constants
    + * for every record. Seems unnecessary.
    + *
    + * Example input and output:
    + * Schema of incoming batch: region_id (VARCHAR), sales_city (VARCHAR), cnt (BIGINT)
    + * Schema of output:
    + *    "schema" : BIGINT - Schema number. For each schema change this number is incremented.
    + *    "computed" : BIGINT - What time is it computed?
    + *    "columns"       : MAP - Column names
    + *       "region_id"  : VARCHAR
    + *       "sales_city" : VARCHAR
    + *       "cnt"        : VARCHAR
    + *    "statscount" : MAP
    --- End diff --
    
    What is a "statscount"?


---
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 #729: Drill 1328: Support table statistics for Parquet

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

    https://github.com/apache/drill/pull/729#discussion_r100706700
  
    --- Diff: exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/statistics/StatisticsAggBatch.java ---
    @@ -0,0 +1,256 @@
    +/**
    + * 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.impl.statistics;
    +
    +import com.google.common.collect.Lists;
    +import com.sun.codemodel.JExpr;
    +import org.apache.drill.common.expression.ErrorCollector;
    +import org.apache.drill.common.expression.ErrorCollectorImpl;
    +import org.apache.drill.common.expression.FunctionCallFactory;
    +import org.apache.drill.common.expression.LogicalExpression;
    +import org.apache.drill.common.expression.SchemaPath;
    +import org.apache.drill.common.expression.ValueExpressions;
    +import org.apache.drill.exec.exception.ClassTransformationException;
    +import org.apache.drill.exec.exception.OutOfMemoryException;
    +import org.apache.drill.exec.exception.SchemaChangeException;
    +import org.apache.drill.exec.expr.ClassGenerator;
    +import org.apache.drill.exec.expr.CodeGenerator;
    +import org.apache.drill.exec.expr.ExpressionTreeMaterializer;
    +import org.apache.drill.exec.expr.TypeHelper;
    +import org.apache.drill.exec.expr.ValueVectorWriteExpression;
    +import org.apache.drill.exec.ops.FragmentContext;
    +import org.apache.drill.exec.physical.config.StatisticsAggregate;
    +import org.apache.drill.exec.physical.impl.aggregate.StreamingAggBatch;
    +import org.apache.drill.exec.physical.impl.aggregate.StreamingAggTemplate;
    +import org.apache.drill.exec.physical.impl.aggregate.StreamingAggregator;
    +import org.apache.drill.exec.planner.physical.StatsAggPrel.OperatorPhase;
    +import org.apache.drill.exec.record.BatchSchema.SelectionVectorMode;
    +import org.apache.drill.exec.record.MaterializedField;
    +import org.apache.drill.exec.record.RecordBatch;
    +import org.apache.drill.exec.record.TypedFieldId;
    +import org.apache.drill.exec.store.ImplicitColumnExplorer;
    +import org.apache.drill.exec.vector.ValueVector;
    +import org.apache.drill.exec.vector.complex.FieldIdUtil;
    +import org.apache.drill.exec.vector.complex.MapVector;
    +
    +import java.io.IOException;
    +import java.util.GregorianCalendar;
    +import java.util.List;
    +import java.util.TimeZone;
    +
    +/**
    + * TODO: This needs cleanup. Currently the key values are constants and we compare the constants
    + * for every record. Seems unnecessary.
    + *
    + * Example input and output:
    + * Schema of incoming batch: region_id (VARCHAR), sales_city (VARCHAR), cnt (BIGINT)
    + * Schema of output:
    + *    "schema" : BIGINT - Schema number. For each schema change this number is incremented.
    + *    "computed" : BIGINT - What time is it computed?
    + *    "columns"       : MAP - Column names
    --- End diff --
    
    The maps except the `column name` map, would contain the specific stats (e.g. rowcount) as the value. The `column name` map contains redundant information. As of 1.10, we would not support nested data types e.g. maps.


---
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 #729: Drill 1328: Support table statistics for Parquet

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

    https://github.com/apache/drill/pull/729#discussion_r103411133
  
    --- Diff: exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/statistics/AvgWidthMergedStatistic.java ---
    @@ -0,0 +1,135 @@
    +/*
    + * 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.impl.statistics;
    +
    +import org.apache.drill.common.types.TypeProtos;
    +import org.apache.drill.exec.expr.holders.NullableFloat8Holder;
    +import org.apache.drill.exec.expr.holders.ValueHolder;
    +import org.apache.drill.exec.vector.NullableFloat8Vector;
    +import org.apache.drill.exec.vector.ValueVector;
    +import org.apache.drill.exec.vector.complex.MapVector;
    +
    +import java.util.HashMap;
    +import java.util.List;
    +import java.util.Map;
    +
    +public class AvgWidthMergedStatistic extends AbstractMergedStatistic {
    +
    +  private String name;
    +  private String inputName;
    +  private boolean configureComplete = false;
    +  private boolean mergeComplete = false;
    +  private Map<String, ValueHolder> sumHolder;
    +  MergedStatistic types, nonNullStatCounts, statCounts;
    +
    +  public AvgWidthMergedStatistic (String name, String inputName) {
    +    this.name = name;
    +    this.inputName = inputName;
    +    this.sumHolder = new HashMap<>();
    +    types = nonNullStatCounts = statCounts = null;
    +  }
    +
    +  @Override
    +  public String getName() {
    +    return name;
    +  }
    +
    +  @Override
    +  public String getInput() {
    +    return inputName;
    +  }
    +
    +  @Override
    +  public void merge(ValueVector input) {
    +    // Check the input is a Map Vector
    +    assert (input.getField().getType().getMinorType() == TypeProtos.MinorType.MAP);
    +    MapVector inputMap = (MapVector) input;
    +    for (ValueVector vv : inputMap) {
    +      String colName = vv.getField().getLastName();
    +      NullableFloat8Holder colSumHolder;
    +      if (sumHolder.get(colName) != null) {
    +        colSumHolder = (NullableFloat8Holder) sumHolder.get(colName);
    +      } else {
    +        colSumHolder = new NullableFloat8Holder();
    +        sumHolder.put(colName, colSumHolder);
    +      }
    +      Object val = vv.getAccessor().getObject(0);
    +      if (val != null) {
    +        colSumHolder.value += (double) val;
    +        colSumHolder.isSet = 1;
    +      }
    +    }
    +  }
    +
    +  @Override
    +  public Object getStat(String colName) {
    --- End diff --
    
    GetStat() returns different types based on the Statistic Type - addressed in a subsequent comment.


---
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 #729: Drill 1328: Support table statistics for Parquet

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/729#discussion_r100676589
  
    --- Diff: exec/java-exec/src/main/java/org/apache/drill/exec/physical/config/StatisticsAggregate.java ---
    @@ -0,0 +1,73 @@
    +/**
    --- End diff --
    
    Nit. Much old code has the above form. But, the pattern "/**" means that this is a Javadoc comment. The copyright notice does not need to appear as text in any generated Javadoc, however. So, please use "/*" to start the comment. 


---
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 #729: Drill 1328: Support table statistics for Parquet

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/729#discussion_r100680416
  
    --- Diff: exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/statistics/StatisticsMergeBatch.java ---
    @@ -0,0 +1,532 @@
    +/**
    + * 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.impl.statistics;
    +
    +import com.google.common.collect.Lists;
    +import com.google.common.collect.Maps;
    +import org.apache.drill.common.expression.LogicalExpression;
    +import org.apache.drill.common.expression.ErrorCollector;
    +import org.apache.drill.common.expression.ErrorCollectorImpl;
    +import org.apache.drill.common.expression.FunctionCallFactory;
    +import org.apache.drill.common.expression.SchemaPath;
    +import org.apache.drill.common.expression.ValueExpressions;
    +import org.apache.drill.common.types.TypeProtos;
    +import org.apache.drill.common.types.TypeProtos.MajorType;
    +import org.apache.drill.exec.exception.OutOfMemoryException;
    +import org.apache.drill.exec.exception.SchemaChangeException;
    +import org.apache.drill.exec.expr.ExpressionTreeMaterializer;
    +import org.apache.drill.exec.expr.TypeHelper;
    +import org.apache.drill.exec.expr.holders.BigIntHolder;
    +import org.apache.drill.exec.expr.holders.NullableFloat8Holder;
    +import org.apache.drill.exec.expr.holders.ObjectHolder;
    +import org.apache.drill.exec.expr.holders.ValueHolder;
    +import org.apache.drill.exec.ops.FragmentContext;
    +import org.apache.drill.exec.physical.config.StatisticsMerge;
    +import org.apache.drill.exec.record.AbstractSingleRecordBatch;
    +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.VectorContainer;
    +import org.apache.drill.exec.record.VectorWrapper;
    +import org.apache.drill.exec.vector.ValueVector;
    +import org.apache.drill.exec.vector.DateVector;
    +import org.apache.drill.exec.vector.BigIntVector;
    +import org.apache.drill.exec.vector.NullableBigIntVector;
    +import org.apache.drill.exec.vector.NullableFloat8Vector;
    +import org.apache.drill.exec.vector.NullableVarBinaryVector;
    +import org.apache.drill.exec.vector.VarCharVector;
    +import org.apache.drill.exec.vector.complex.MapVector;
    +
    +import java.io.ByteArrayInputStream;
    +import java.io.DataInputStream;
    +import java.io.IOException;
    +import java.util.List;
    +import java.util.GregorianCalendar;
    +import java.util.HashMap;
    +import java.util.Map;
    +import java.util.TimeZone;
    +
    +import com.clearspring.analytics.stream.cardinality.HyperLogLog;
    +
    +public class StatisticsMergeBatch extends AbstractSingleRecordBatch<StatisticsMerge> {
    +  private static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(StatisticsMergeBatch.class);
    +  private Map<String, String> functions;
    +  private boolean first = true;
    +  private boolean finished = false;
    +  private int schema = 0;
    +  private int recordCount = 0;
    +  private List<String> keyList = null;
    +  private Map<MaterializedField, ValueVector> dataSrcVecMap = null;
    +  // Map of non-map fields to VV in the incoming schema
    +  private Map<MaterializedField, ValueVector> copySrcVecMap = null;
    +  private Map<String, Map<String, ValueHolder>> aggregationMap = null;
    +  public StatisticsMergeBatch(StatisticsMerge popConfig, RecordBatch incoming,
    +                              FragmentContext context) throws OutOfMemoryException {
    +    super(popConfig, context, incoming);
    +    this.functions = new HashMap<>();
    +    this.aggregationMap = new HashMap<>();
    +
    +    /*for (String key : popConfig.getFunctions()) {
    +      aggregationMap.put(key, new HashMap<String, ValueHolder>());
    +      if (key.equalsIgnoreCase("statcount") || key.equalsIgnoreCase("nonnullstatcount")) {
    +        functions.put(key, "sum");
    +      } else if (key.equalsIgnoreCase("hll")) {
    +        functions.put(key, "hll_merge");
    +      } else if (key.equalsIgnoreCase("sum_width")) {
    +        functions.put(key, "avg_width");
    +      }
    +    }*/
    +    for (String key : popConfig.getFunctions()) {
    +      if (key.equalsIgnoreCase("sum_width")) {
    +        functions.put(key, "avg_width");
    +      } else if (key.equalsIgnoreCase("hll")) {
    +        functions.put(key, "hll_merge");
    +      } else {
    +        functions.put(key, key);
    +      }
    +      aggregationMap.put(functions.get(key), new HashMap<String, ValueHolder>());
    +    }
    +  }
    +
    +  private void createKeyColumn(String name, LogicalExpression expr, Map<MaterializedField, ValueVector> parentMap)
    +      throws SchemaChangeException {
    +    ErrorCollector collector = new ErrorCollectorImpl();
    +
    +    LogicalExpression mle = ExpressionTreeMaterializer.materialize(expr, incoming, collector,
    +        context.getFunctionRegistry());
    +
    +    MaterializedField outputField = MaterializedField.create(name, mle.getMajorType());
    +    ValueVector vector = TypeHelper.getNewVector(outputField, oContext.getAllocator());
    +    container.add(vector);
    +
    +    if (collector.hasErrors()) {
    --- End diff --
    
    Same issue as noted previously.


---
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 #729: Drill 1328: Support table statistics for Parquet

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/729#discussion_r102870259
  
    --- Diff: exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/statistics/AvgWidthMergedStatistic.java ---
    @@ -0,0 +1,135 @@
    +/*
    + * 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.impl.statistics;
    +
    +import org.apache.drill.common.types.TypeProtos;
    +import org.apache.drill.exec.expr.holders.NullableFloat8Holder;
    +import org.apache.drill.exec.expr.holders.ValueHolder;
    +import org.apache.drill.exec.vector.NullableFloat8Vector;
    +import org.apache.drill.exec.vector.ValueVector;
    +import org.apache.drill.exec.vector.complex.MapVector;
    +
    +import java.util.HashMap;
    +import java.util.List;
    +import java.util.Map;
    +
    +public class AvgWidthMergedStatistic extends AbstractMergedStatistic {
    +
    +  private String name;
    +  private String inputName;
    +  private boolean configureComplete = false;
    +  private boolean mergeComplete = false;
    +  private Map<String, ValueHolder> sumHolder;
    +  MergedStatistic types, nonNullStatCounts, statCounts;
    +
    +  public AvgWidthMergedStatistic (String name, String inputName) {
    +    this.name = name;
    +    this.inputName = inputName;
    +    this.sumHolder = new HashMap<>();
    +    types = nonNullStatCounts = statCounts = null;
    +  }
    +
    +  @Override
    +  public String getName() {
    +    return name;
    +  }
    +
    +  @Override
    +  public String getInput() {
    +    return inputName;
    +  }
    +
    +  @Override
    +  public void merge(ValueVector input) {
    +    // Check the input is a Map Vector
    +    assert (input.getField().getType().getMinorType() == TypeProtos.MinorType.MAP);
    +    MapVector inputMap = (MapVector) input;
    +    for (ValueVector vv : inputMap) {
    +      String colName = vv.getField().getLastName();
    +      NullableFloat8Holder colSumHolder;
    +      if (sumHolder.get(colName) != null) {
    +        colSumHolder = (NullableFloat8Holder) sumHolder.get(colName);
    +      } else {
    +        colSumHolder = new NullableFloat8Holder();
    +        sumHolder.put(colName, colSumHolder);
    +      }
    +      Object val = vv.getAccessor().getObject(0);
    +      if (val != null) {
    +        colSumHolder.value += (double) val;
    +        colSumHolder.isSet = 1;
    +      }
    +    }
    +  }
    +
    +  @Override
    +  public Object getStat(String colName) {
    +      if (mergeComplete != true) {
    +        throw new IllegalStateException(
    +            String.format("Statistic `%s` has not completed merging statistics", name));
    +      }
    +      NullableFloat8Holder colSumHolder = (NullableFloat8Holder) sumHolder.get(colName);
    +      return (long) (colSumHolder.value/ getRowCount(colName));
    --- End diff --
    
    In Drill, different columns generally cannot have different row counts. In any vector, all columns must have the same number of values. Or, is the `getRowCount` method really `getNonNullValueCount`: the number of rows for which the column in question had non-null values?
    
    By discarding the holder, the above reduces to:
    
    ```
    return Math.round(colSum / rowCount);
    ```
    
    Also, I'm not really sure what this is doing. The sum seems global, but the value returned is per column, with only the row count differing between columns? Perhaps explain that a bit?...



---
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 #729: Drill 1328: Support table statistics for Parquet

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

    https://github.com/apache/drill/pull/729
  
    I have addressed the second set of comments. @amansinha100 @paul-rogers could you please review and/or approve? 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 #729: Drill 1328 r4

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

    https://github.com/apache/drill/pull/729#discussion_r97915146
  
    --- Diff: exec/java-exec/src/main/java/org/apache/drill/exec/planner/common/DrillJoinRelBase.java ---
    @@ -99,9 +105,40 @@ public RelOptCost computeSelfCost(RelOptPlanner planner) {
       public double getRows() {
         if (this.condition.isAlwaysTrue()) {
           return joinRowFactor * this.getLeft().getRows() * this.getRight().getRows();
    -    } else {
    -      return joinRowFactor * Math.max(this.getLeft().getRows(), this.getRight().getRows());
         }
    +
    +    int[] joinFields = new int[2];
    +
    +    LogicalJoin jr = new LogicalJoin(
    +        this.getCluster(),
    +        this.getTraitSet(),
    +        this.getLeft(),
    +        this.getRight(),
    +        this.getCondition(),
    +        this.getJoinType(),
    +        this.getVariablesStopped(),
    +        false,
    +        ImmutableList.<RelDataTypeField>of());
    +
    +    if (!DrillRelOptUtil.guessRows(this)         //Statistics present for left and right side of the join
    +        && jr.getJoinType() == JoinRelType.INNER
    +        && DrillRelOptUtil.analyzeSimpleEquiJoin((Join)jr, joinFields)) {
    +      ImmutableBitSet leq = ImmutableBitSet.of(joinFields[0]);
    +      ImmutableBitSet req = ImmutableBitSet.of(joinFields[1]);
    +
    +      Double ldrc = RelMetadataQuery.getDistinctRowCount(this.getLeft(), leq, null);
    +      Double rdrc = RelMetadataQuery.getDistinctRowCount(this.getRight(), req, null);
    +
    +      Double lrc = RelMetadataQuery.getRowCount(this.getLeft());
    +      Double rrc = RelMetadataQuery.getRowCount(this.getRight());
    +
    +      if (ldrc != null && rdrc != null && lrc != null && rrc != null) {
    +        return (lrc * rrc) / Math.max(ldrc, rdrc);
    --- End diff --
    
    Multiplying two very large row counts could create overflow. 


---
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 #729: Drill 1328: Support table statistics for Parquet

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

    https://github.com/apache/drill/pull/729#discussion_r100708249
  
    --- Diff: exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/statistics/StatisticsAggBatch.java ---
    @@ -0,0 +1,256 @@
    +/**
    + * 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.impl.statistics;
    +
    +import com.google.common.collect.Lists;
    +import com.sun.codemodel.JExpr;
    +import org.apache.drill.common.expression.ErrorCollector;
    +import org.apache.drill.common.expression.ErrorCollectorImpl;
    +import org.apache.drill.common.expression.FunctionCallFactory;
    +import org.apache.drill.common.expression.LogicalExpression;
    +import org.apache.drill.common.expression.SchemaPath;
    +import org.apache.drill.common.expression.ValueExpressions;
    +import org.apache.drill.exec.exception.ClassTransformationException;
    +import org.apache.drill.exec.exception.OutOfMemoryException;
    +import org.apache.drill.exec.exception.SchemaChangeException;
    +import org.apache.drill.exec.expr.ClassGenerator;
    +import org.apache.drill.exec.expr.CodeGenerator;
    +import org.apache.drill.exec.expr.ExpressionTreeMaterializer;
    +import org.apache.drill.exec.expr.TypeHelper;
    +import org.apache.drill.exec.expr.ValueVectorWriteExpression;
    +import org.apache.drill.exec.ops.FragmentContext;
    +import org.apache.drill.exec.physical.config.StatisticsAggregate;
    +import org.apache.drill.exec.physical.impl.aggregate.StreamingAggBatch;
    +import org.apache.drill.exec.physical.impl.aggregate.StreamingAggTemplate;
    +import org.apache.drill.exec.physical.impl.aggregate.StreamingAggregator;
    +import org.apache.drill.exec.planner.physical.StatsAggPrel.OperatorPhase;
    +import org.apache.drill.exec.record.BatchSchema.SelectionVectorMode;
    +import org.apache.drill.exec.record.MaterializedField;
    +import org.apache.drill.exec.record.RecordBatch;
    +import org.apache.drill.exec.record.TypedFieldId;
    +import org.apache.drill.exec.store.ImplicitColumnExplorer;
    +import org.apache.drill.exec.vector.ValueVector;
    +import org.apache.drill.exec.vector.complex.FieldIdUtil;
    +import org.apache.drill.exec.vector.complex.MapVector;
    +
    +import java.io.IOException;
    +import java.util.GregorianCalendar;
    +import java.util.List;
    +import java.util.TimeZone;
    +
    +/**
    + * TODO: This needs cleanup. Currently the key values are constants and we compare the constants
    + * for every record. Seems unnecessary.
    + *
    + * Example input and output:
    + * Schema of incoming batch: region_id (VARCHAR), sales_city (VARCHAR), cnt (BIGINT)
    + * Schema of output:
    + *    "schema" : BIGINT - Schema number. For each schema change this number is incremented.
    + *    "computed" : BIGINT - What time is it computed?
    + *    "columns"       : MAP - Column names
    + *       "region_id"  : VARCHAR
    + *       "sales_city" : VARCHAR
    + *       "cnt"        : VARCHAR
    + *    "statscount" : MAP
    + *       "region_id"  : BIGINT - statscount(region_id) - aggregation over all values of region_id
    + *                      in incoming batch
    + *       "sales_city" : BIGINT - statscount(sales_city)
    + *       "cnt"        : BIGINT - statscount(cnt)
    + *    "nonnullstatcount" : MAP
    + *       "region_id"  : BIGINT - nonnullstatcount(region_id)
    + *       "sales_city" : BIGINT - nonnullstatcount(sales_city)
    + *       "cnt"        : BIGINT - nonnullstatcount(cnt)
    + *   .... another map for next stats function ....
    + */
    +public class StatisticsAggBatch extends StreamingAggBatch {
    +  private List<String> functions;
    +  private int schema = 0;
    +
    +  public StatisticsAggBatch(StatisticsAggregate popConfig, RecordBatch incoming,
    +      FragmentContext context) throws OutOfMemoryException {
    +    super(popConfig, incoming, context);
    +    this.functions = popConfig.getFunctions();
    +  }
    +
    +  private void createKeyColumn(String name, LogicalExpression expr, List<LogicalExpression> keyExprs,
    +      List<TypedFieldId> keyOutputIds) throws SchemaChangeException {
    +    ErrorCollector collector = new ErrorCollectorImpl();
    +
    +    LogicalExpression mle = ExpressionTreeMaterializer.materialize(expr, incoming, collector,
    +        context.getFunctionRegistry());
    +
    +    MaterializedField outputField = MaterializedField.create(name, mle.getMajorType());
    +    ValueVector vector = TypeHelper.getNewVector(outputField, oContext.getAllocator());
    +
    +    keyExprs.add(mle);
    +    keyOutputIds.add(container.add(vector));
    +
    +    if (collector.hasErrors()) {
    +      throw new SchemaChangeException("Failure while materializing expression. "
    +          + collector.toErrorString());
    +    }
    +  }
    +
    +  private void createNestedKeyColumn(MapVector parent, String name, LogicalExpression expr,
    +      List<LogicalExpression> keyExprs, List<TypedFieldId> keyOutputIds)
    +          throws SchemaChangeException {
    +    ErrorCollector collector = new ErrorCollectorImpl();
    +
    +    LogicalExpression mle = ExpressionTreeMaterializer.materialize(expr, incoming, collector,
    +        context.getFunctionRegistry());
    +
    +    Class<? extends ValueVector> vvc =
    +        TypeHelper.getValueVectorClass(mle.getMajorType().getMinorType(),
    +            mle.getMajorType().getMode());
    +
    +    ValueVector vv = parent.addOrGet(name, mle.getMajorType(), vvc);
    +
    +    TypedFieldId pfid = container.getValueVectorId(SchemaPath.getSimplePath(parent.getField().getPath()));
    +    assert pfid.getFieldIds().length == 1;
    +    TypedFieldId.Builder builder = TypedFieldId.newBuilder();
    +    builder.addId(pfid.getFieldIds()[0]);
    +    TypedFieldId id =
    +        FieldIdUtil.getFieldIdIfMatches(parent, builder, true,
    +            SchemaPath.getSimplePath(vv.getField().getPath()).getRootSegment());
    +
    +    keyExprs.add(mle);
    +    keyOutputIds.add(id);
    +
    +    if (collector.hasErrors()) {
    +      throw new SchemaChangeException("Failure while materializing expression. "
    +          + collector.toErrorString());
    +    }
    +  }
    +
    +  private void addMapVector(String name, MapVector parent, LogicalExpression expr,
    +      List<LogicalExpression> valueExprs) throws SchemaChangeException {
    +    ErrorCollector collector = new ErrorCollectorImpl();
    +
    +    LogicalExpression mle = ExpressionTreeMaterializer.materialize(expr, incoming, collector,
    +        context.getFunctionRegistry());
    +
    +    Class<? extends ValueVector> vvc =
    +        TypeHelper.getValueVectorClass(mle.getMajorType().getMinorType(), mle.getMajorType().getMode());
    +    ValueVector vv = parent.addOrGet(name, mle.getMajorType(), vvc);
    +
    +    TypedFieldId pfid = container.getValueVectorId(SchemaPath.getSimplePath(parent.getField().getPath()));
    +    assert pfid.getFieldIds().length == 1;
    +    TypedFieldId.Builder builder = TypedFieldId.newBuilder();
    +    builder.addId(pfid.getFieldIds()[0]);
    +    TypedFieldId id = FieldIdUtil.getFieldIdIfMatches(parent, builder, true,
    +        SchemaPath.getSimplePath(vv.getField().getPath()).getRootSegment());
    +
    +    valueExprs.add(new ValueVectorWriteExpression(id, mle, true));
    +
    +    if (collector.hasErrors()) {
    +      throw new SchemaChangeException("Failure while materializing expression. "
    +          + collector.toErrorString());
    +    }
    +  }
    +
    +  private StreamingAggregator codegenAggregator(List<LogicalExpression> keyExprs,
    +      List<LogicalExpression> valueExprs, List<TypedFieldId> keyOutputIds)
    +          throws SchemaChangeException, ClassTransformationException, IOException {
    +    ClassGenerator<StreamingAggregator> cg =
    +        CodeGenerator.getRoot(StreamingAggTemplate.TEMPLATE_DEFINITION, context.getFunctionRegistry(),
    +            context.getOptions());
    --- End diff --
    
    Thanks for adding the feature! This would certainly be helpful while debugging.


---
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 #729: Drill 1328: Support table statistics for Parquet

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/729#discussion_r100680601
  
    --- Diff: exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/statistics/StatisticsMergeBatch.java ---
    @@ -0,0 +1,532 @@
    +/**
    + * 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.impl.statistics;
    +
    +import com.google.common.collect.Lists;
    +import com.google.common.collect.Maps;
    +import org.apache.drill.common.expression.LogicalExpression;
    +import org.apache.drill.common.expression.ErrorCollector;
    +import org.apache.drill.common.expression.ErrorCollectorImpl;
    +import org.apache.drill.common.expression.FunctionCallFactory;
    +import org.apache.drill.common.expression.SchemaPath;
    +import org.apache.drill.common.expression.ValueExpressions;
    +import org.apache.drill.common.types.TypeProtos;
    +import org.apache.drill.common.types.TypeProtos.MajorType;
    +import org.apache.drill.exec.exception.OutOfMemoryException;
    +import org.apache.drill.exec.exception.SchemaChangeException;
    +import org.apache.drill.exec.expr.ExpressionTreeMaterializer;
    +import org.apache.drill.exec.expr.TypeHelper;
    +import org.apache.drill.exec.expr.holders.BigIntHolder;
    +import org.apache.drill.exec.expr.holders.NullableFloat8Holder;
    +import org.apache.drill.exec.expr.holders.ObjectHolder;
    +import org.apache.drill.exec.expr.holders.ValueHolder;
    +import org.apache.drill.exec.ops.FragmentContext;
    +import org.apache.drill.exec.physical.config.StatisticsMerge;
    +import org.apache.drill.exec.record.AbstractSingleRecordBatch;
    +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.VectorContainer;
    +import org.apache.drill.exec.record.VectorWrapper;
    +import org.apache.drill.exec.vector.ValueVector;
    +import org.apache.drill.exec.vector.DateVector;
    +import org.apache.drill.exec.vector.BigIntVector;
    +import org.apache.drill.exec.vector.NullableBigIntVector;
    +import org.apache.drill.exec.vector.NullableFloat8Vector;
    +import org.apache.drill.exec.vector.NullableVarBinaryVector;
    +import org.apache.drill.exec.vector.VarCharVector;
    +import org.apache.drill.exec.vector.complex.MapVector;
    +
    +import java.io.ByteArrayInputStream;
    +import java.io.DataInputStream;
    +import java.io.IOException;
    +import java.util.List;
    +import java.util.GregorianCalendar;
    +import java.util.HashMap;
    +import java.util.Map;
    +import java.util.TimeZone;
    +
    +import com.clearspring.analytics.stream.cardinality.HyperLogLog;
    +
    +public class StatisticsMergeBatch extends AbstractSingleRecordBatch<StatisticsMerge> {
    +  private static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(StatisticsMergeBatch.class);
    +  private Map<String, String> functions;
    +  private boolean first = true;
    +  private boolean finished = false;
    +  private int schema = 0;
    +  private int recordCount = 0;
    +  private List<String> keyList = null;
    +  private Map<MaterializedField, ValueVector> dataSrcVecMap = null;
    +  // Map of non-map fields to VV in the incoming schema
    +  private Map<MaterializedField, ValueVector> copySrcVecMap = null;
    +  private Map<String, Map<String, ValueHolder>> aggregationMap = null;
    +  public StatisticsMergeBatch(StatisticsMerge popConfig, RecordBatch incoming,
    +                              FragmentContext context) throws OutOfMemoryException {
    +    super(popConfig, context, incoming);
    +    this.functions = new HashMap<>();
    +    this.aggregationMap = new HashMap<>();
    +
    +    /*for (String key : popConfig.getFunctions()) {
    +      aggregationMap.put(key, new HashMap<String, ValueHolder>());
    +      if (key.equalsIgnoreCase("statcount") || key.equalsIgnoreCase("nonnullstatcount")) {
    +        functions.put(key, "sum");
    +      } else if (key.equalsIgnoreCase("hll")) {
    +        functions.put(key, "hll_merge");
    +      } else if (key.equalsIgnoreCase("sum_width")) {
    +        functions.put(key, "avg_width");
    +      }
    +    }*/
    +    for (String key : popConfig.getFunctions()) {
    +      if (key.equalsIgnoreCase("sum_width")) {
    +        functions.put(key, "avg_width");
    +      } else if (key.equalsIgnoreCase("hll")) {
    +        functions.put(key, "hll_merge");
    +      } else {
    +        functions.put(key, key);
    +      }
    +      aggregationMap.put(functions.get(key), new HashMap<String, ValueHolder>());
    +    }
    +  }
    +
    +  private void createKeyColumn(String name, LogicalExpression expr, Map<MaterializedField, ValueVector> parentMap)
    +      throws SchemaChangeException {
    +    ErrorCollector collector = new ErrorCollectorImpl();
    +
    +    LogicalExpression mle = ExpressionTreeMaterializer.materialize(expr, incoming, collector,
    +        context.getFunctionRegistry());
    +
    +    MaterializedField outputField = MaterializedField.create(name, mle.getMajorType());
    +    ValueVector vector = TypeHelper.getNewVector(outputField, oContext.getAllocator());
    +    container.add(vector);
    +
    +    if (collector.hasErrors()) {
    +      throw new SchemaChangeException("Failure while materializing expression. "
    +          + collector.toErrorString());
    +    }
    +    parentMap.put(outputField, vector);
    +  }
    +
    +  private ValueVector addMapVector(String name, MapVector parent, LogicalExpression expr) throws SchemaChangeException {
    +    ErrorCollector collector = new ErrorCollectorImpl();
    +
    +    LogicalExpression mle = ExpressionTreeMaterializer.materialize(expr, incoming, collector,
    +        context.getFunctionRegistry());
    +
    +    Class<? extends ValueVector> vvc =
    +        TypeHelper.getValueVectorClass(mle.getMajorType().getMinorType(), mle.getMajorType().getMode());
    +    ValueVector vector = parent.addOrGet(name, mle.getMajorType(), vvc);
    +
    +    if (collector.hasErrors()) {
    +      throw new SchemaChangeException("Failure while materializing expression. "
    +          + collector.toErrorString());
    +    }
    +    return vector;
    +  }
    +
    +  /**
    +   * Identify the list of fields within a map which are unpivoted as columns in output
    +   */
    +  private void buildKeyList() {
    +    List<String> lastMapKeyList = null;
    +    for (VectorWrapper<?> vw : incoming) {
    +      if (vw.getField().getType().getMinorType() != TypeProtos.MinorType.MAP) {
    +        continue;
    +      }
    +
    +      keyList = Lists.newArrayList();
    +
    +      for (ValueVector vv : vw.getValueVector()) {
    +        keyList.add(vv.getField().getLastName());
    +      }
    +
    +      if (lastMapKeyList == null) {
    +        lastMapKeyList = keyList;
    +      } else {
    +        if (keyList.size() != lastMapKeyList.size() || !lastMapKeyList.containsAll(keyList)) {
    +          throw new UnsupportedOperationException("Maps have different fields");
    +        }
    +      }
    +    }
    +  }
    +
    +  private void buildOutputContainer() throws SchemaChangeException {
    +    dataSrcVecMap = Maps.newHashMap();
    +    copySrcVecMap = Maps.newHashMap();
    +    MajorType mt = null;
    +
    +    ErrorCollector collector = new ErrorCollectorImpl();
    +    GregorianCalendar calendar = new GregorianCalendar(TimeZone.getTimeZone("UTC"));
    +
    +    calendar.setTimeInMillis(System.currentTimeMillis());
    +    createKeyColumn("schema", ValueExpressions.getBigInt(schema++), copySrcVecMap);
    +    createKeyColumn("computed", ValueExpressions.getDate(calendar), copySrcVecMap);
    +
    +    for (VectorWrapper<?> vw : incoming) {
    +      addVectorToOutgoingContainer(vw.getField().getLastName(), vw, collector);
    +      /*MaterializedField ds = vw.getField();
    +      String field = vw.getField().getLastName();
    +      // Input map vector
    +      MapVector mapVector = (MapVector) vw.getValueVector();
    +      assert mapVector.getPrimitiveVectors().size() > 0;
    +      // Proceed to create output map vector with same name e.g. statcount etc.
    +      mt = mapVector.getField().getType();
    +      MaterializedField mf = MaterializedField.create(functions.get(field), mt);
    +      assert !dataSrcVecMap.containsKey(mf);
    +      ValueVector vector = TypeHelper.getNewVector(mf, oContext.getAllocator());
    +      container.add(vector);
    +      MapVector outputMapVector = (MapVector) vector;
    +
    +      for (ValueVector vv : mapVector) {
    +        String fieldName = vv.getField().getLastName();
    +        if (!keyList.contains(fieldName)) {
    +          throw new UnsupportedOperationException("Unpivot data vector " +
    +              ds + " contains key " + fieldName + " not contained in key source!");
    +        }
    +        if (vv.getField().getType().getMinorType() == TypeProtos.MinorType.MAP) {
    +          throw new UnsupportedOperationException("Unpivot of nested map is not supported!");
    +        }
    +        if (field.equals("column")) {
    +          outputMapVector.addOrGet(fieldName, vv.getField().getType(), vv.getClass());
    +        } else {
    +          List<LogicalExpression> args = Lists.newArrayList();
    +          //TODO: Something else to access value of col such as emp_id?
    +          args.add(SchemaPath.getSimplePath(vv.getField().getPath()));
    +          //TODO: Put in the mapVector
    +          LogicalExpression call = FunctionCallFactory.createExpression(functions.get(field), args);
    +          //TODO: Is this sufficient to add to new Map?
    +          ValueVector vector1 = addMapVector(fieldName, outputMapVector, call);
    +          if (collector.hasErrors()) {
    +            throw new SchemaChangeException("Failure while materializing expression. "
    +                + collector.toErrorString());
    +          }
    +        }
    +      }
    +      dataSrcVecMap.put(ds, outputMapVector);*/
    +    }
    +    //Now create NDV in the outgoing container which was not avaliable in the incoming
    +    for (VectorWrapper<?> vw : incoming) {
    +      if (vw.getField().getLastName().equalsIgnoreCase("sum_width")) {//NullableFloat8 type
    +        addVectorToOutgoingContainer("ndv", vw, collector);
    +        break;
    +      }
    +    }
    +    container.setRecordCount(0);
    +    recordCount = 0;
    +    container.buildSchema(incoming.getSchema().getSelectionVectorMode());
    +  }
    +
    +  private void addVectorToOutgoingContainer(String field, VectorWrapper vw, ErrorCollector collector)
    +     throws SchemaChangeException {
    +    // Input map vector
    +    MapVector mapVector = (MapVector) vw.getValueVector();
    +    MaterializedField mf;
    +    assert mapVector.getPrimitiveVectors().size() > 0;
    +    // Proceed to create output map vector with same name e.g. statcount etc.
    +    MajorType mt = mapVector.getField().getType();
    +    if (functions.get(field) != null) {
    --- End diff --
    
    `functions.contains(field)`


---
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 #729: Drill 1328: Support table statistics for Parquet

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

    https://github.com/apache/drill/pull/729#discussion_r102289330
  
    --- Diff: exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/statistics/StatisticsMergeBatch.java ---
    @@ -0,0 +1,532 @@
    +/**
    + * 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.impl.statistics;
    +
    +import com.google.common.collect.Lists;
    +import com.google.common.collect.Maps;
    +import org.apache.drill.common.expression.LogicalExpression;
    +import org.apache.drill.common.expression.ErrorCollector;
    +import org.apache.drill.common.expression.ErrorCollectorImpl;
    +import org.apache.drill.common.expression.FunctionCallFactory;
    +import org.apache.drill.common.expression.SchemaPath;
    +import org.apache.drill.common.expression.ValueExpressions;
    +import org.apache.drill.common.types.TypeProtos;
    +import org.apache.drill.common.types.TypeProtos.MajorType;
    +import org.apache.drill.exec.exception.OutOfMemoryException;
    +import org.apache.drill.exec.exception.SchemaChangeException;
    +import org.apache.drill.exec.expr.ExpressionTreeMaterializer;
    +import org.apache.drill.exec.expr.TypeHelper;
    +import org.apache.drill.exec.expr.holders.BigIntHolder;
    +import org.apache.drill.exec.expr.holders.NullableFloat8Holder;
    +import org.apache.drill.exec.expr.holders.ObjectHolder;
    +import org.apache.drill.exec.expr.holders.ValueHolder;
    +import org.apache.drill.exec.ops.FragmentContext;
    +import org.apache.drill.exec.physical.config.StatisticsMerge;
    +import org.apache.drill.exec.record.AbstractSingleRecordBatch;
    +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.VectorContainer;
    +import org.apache.drill.exec.record.VectorWrapper;
    +import org.apache.drill.exec.vector.ValueVector;
    +import org.apache.drill.exec.vector.DateVector;
    +import org.apache.drill.exec.vector.BigIntVector;
    +import org.apache.drill.exec.vector.NullableBigIntVector;
    +import org.apache.drill.exec.vector.NullableFloat8Vector;
    +import org.apache.drill.exec.vector.NullableVarBinaryVector;
    +import org.apache.drill.exec.vector.VarCharVector;
    +import org.apache.drill.exec.vector.complex.MapVector;
    +
    +import java.io.ByteArrayInputStream;
    +import java.io.DataInputStream;
    +import java.io.IOException;
    +import java.util.List;
    +import java.util.GregorianCalendar;
    +import java.util.HashMap;
    +import java.util.Map;
    +import java.util.TimeZone;
    +
    +import com.clearspring.analytics.stream.cardinality.HyperLogLog;
    +
    +public class StatisticsMergeBatch extends AbstractSingleRecordBatch<StatisticsMerge> {
    +  private static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(StatisticsMergeBatch.class);
    +  private Map<String, String> functions;
    +  private boolean first = true;
    +  private boolean finished = false;
    +  private int schema = 0;
    +  private int recordCount = 0;
    +  private List<String> keyList = null;
    +  private Map<MaterializedField, ValueVector> dataSrcVecMap = null;
    +  // Map of non-map fields to VV in the incoming schema
    +  private Map<MaterializedField, ValueVector> copySrcVecMap = null;
    +  private Map<String, Map<String, ValueHolder>> aggregationMap = null;
    +  public StatisticsMergeBatch(StatisticsMerge popConfig, RecordBatch incoming,
    +                              FragmentContext context) throws OutOfMemoryException {
    +    super(popConfig, context, incoming);
    +    this.functions = new HashMap<>();
    +    this.aggregationMap = new HashMap<>();
    +
    +    /*for (String key : popConfig.getFunctions()) {
    +      aggregationMap.put(key, new HashMap<String, ValueHolder>());
    +      if (key.equalsIgnoreCase("statcount") || key.equalsIgnoreCase("nonnullstatcount")) {
    +        functions.put(key, "sum");
    +      } else if (key.equalsIgnoreCase("hll")) {
    +        functions.put(key, "hll_merge");
    +      } else if (key.equalsIgnoreCase("sum_width")) {
    +        functions.put(key, "avg_width");
    +      }
    +    }*/
    +    for (String key : popConfig.getFunctions()) {
    +      if (key.equalsIgnoreCase("sum_width")) {
    +        functions.put(key, "avg_width");
    +      } else if (key.equalsIgnoreCase("hll")) {
    +        functions.put(key, "hll_merge");
    +      } else {
    +        functions.put(key, key);
    +      }
    +      aggregationMap.put(functions.get(key), new HashMap<String, ValueHolder>());
    +    }
    +  }
    +
    +  private void createKeyColumn(String name, LogicalExpression expr, Map<MaterializedField, ValueVector> parentMap)
    +      throws SchemaChangeException {
    +    ErrorCollector collector = new ErrorCollectorImpl();
    +
    +    LogicalExpression mle = ExpressionTreeMaterializer.materialize(expr, incoming, collector,
    +        context.getFunctionRegistry());
    +
    +    MaterializedField outputField = MaterializedField.create(name, mle.getMajorType());
    +    ValueVector vector = TypeHelper.getNewVector(outputField, oContext.getAllocator());
    +    container.add(vector);
    +
    +    if (collector.hasErrors()) {
    +      throw new SchemaChangeException("Failure while materializing expression. "
    +          + collector.toErrorString());
    +    }
    +    parentMap.put(outputField, vector);
    +  }
    +
    +  private ValueVector addMapVector(String name, MapVector parent, LogicalExpression expr) throws SchemaChangeException {
    +    ErrorCollector collector = new ErrorCollectorImpl();
    +
    +    LogicalExpression mle = ExpressionTreeMaterializer.materialize(expr, incoming, collector,
    +        context.getFunctionRegistry());
    +
    +    Class<? extends ValueVector> vvc =
    +        TypeHelper.getValueVectorClass(mle.getMajorType().getMinorType(), mle.getMajorType().getMode());
    +    ValueVector vector = parent.addOrGet(name, mle.getMajorType(), vvc);
    +
    +    if (collector.hasErrors()) {
    +      throw new SchemaChangeException("Failure while materializing expression. "
    +          + collector.toErrorString());
    +    }
    +    return vector;
    +  }
    +
    +  /**
    +   * Identify the list of fields within a map which are unpivoted as columns in output
    +   */
    +  private void buildKeyList() {
    +    List<String> lastMapKeyList = null;
    +    for (VectorWrapper<?> vw : incoming) {
    +      if (vw.getField().getType().getMinorType() != TypeProtos.MinorType.MAP) {
    +        continue;
    +      }
    +
    +      keyList = Lists.newArrayList();
    +
    +      for (ValueVector vv : vw.getValueVector()) {
    +        keyList.add(vv.getField().getLastName());
    +      }
    +
    +      if (lastMapKeyList == null) {
    +        lastMapKeyList = keyList;
    +      } else {
    +        if (keyList.size() != lastMapKeyList.size() || !lastMapKeyList.containsAll(keyList)) {
    +          throw new UnsupportedOperationException("Maps have different fields");
    +        }
    +      }
    +    }
    +  }
    +
    +  private void buildOutputContainer() throws SchemaChangeException {
    +    dataSrcVecMap = Maps.newHashMap();
    +    copySrcVecMap = Maps.newHashMap();
    +    MajorType mt = null;
    +
    +    ErrorCollector collector = new ErrorCollectorImpl();
    +    GregorianCalendar calendar = new GregorianCalendar(TimeZone.getTimeZone("UTC"));
    +
    +    calendar.setTimeInMillis(System.currentTimeMillis());
    +    createKeyColumn("schema", ValueExpressions.getBigInt(schema++), copySrcVecMap);
    +    createKeyColumn("computed", ValueExpressions.getDate(calendar), copySrcVecMap);
    +
    +    for (VectorWrapper<?> vw : incoming) {
    +      addVectorToOutgoingContainer(vw.getField().getLastName(), vw, collector);
    +      /*MaterializedField ds = vw.getField();
    +      String field = vw.getField().getLastName();
    +      // Input map vector
    +      MapVector mapVector = (MapVector) vw.getValueVector();
    +      assert mapVector.getPrimitiveVectors().size() > 0;
    +      // Proceed to create output map vector with same name e.g. statcount etc.
    +      mt = mapVector.getField().getType();
    +      MaterializedField mf = MaterializedField.create(functions.get(field), mt);
    +      assert !dataSrcVecMap.containsKey(mf);
    +      ValueVector vector = TypeHelper.getNewVector(mf, oContext.getAllocator());
    +      container.add(vector);
    +      MapVector outputMapVector = (MapVector) vector;
    +
    +      for (ValueVector vv : mapVector) {
    +        String fieldName = vv.getField().getLastName();
    +        if (!keyList.contains(fieldName)) {
    +          throw new UnsupportedOperationException("Unpivot data vector " +
    +              ds + " contains key " + fieldName + " not contained in key source!");
    +        }
    +        if (vv.getField().getType().getMinorType() == TypeProtos.MinorType.MAP) {
    +          throw new UnsupportedOperationException("Unpivot of nested map is not supported!");
    +        }
    +        if (field.equals("column")) {
    +          outputMapVector.addOrGet(fieldName, vv.getField().getType(), vv.getClass());
    +        } else {
    +          List<LogicalExpression> args = Lists.newArrayList();
    +          //TODO: Something else to access value of col such as emp_id?
    +          args.add(SchemaPath.getSimplePath(vv.getField().getPath()));
    +          //TODO: Put in the mapVector
    +          LogicalExpression call = FunctionCallFactory.createExpression(functions.get(field), args);
    +          //TODO: Is this sufficient to add to new Map?
    +          ValueVector vector1 = addMapVector(fieldName, outputMapVector, call);
    +          if (collector.hasErrors()) {
    +            throw new SchemaChangeException("Failure while materializing expression. "
    +                + collector.toErrorString());
    +          }
    +        }
    +      }
    +      dataSrcVecMap.put(ds, outputMapVector);*/
    +    }
    +    //Now create NDV in the outgoing container which was not avaliable in the incoming
    +    for (VectorWrapper<?> vw : incoming) {
    +      if (vw.getField().getLastName().equalsIgnoreCase("sum_width")) {//NullableFloat8 type
    +        addVectorToOutgoingContainer("ndv", vw, collector);
    +        break;
    +      }
    +    }
    +    container.setRecordCount(0);
    +    recordCount = 0;
    +    container.buildSchema(incoming.getSchema().getSelectionVectorMode());
    +  }
    +
    +  private void addVectorToOutgoingContainer(String field, VectorWrapper vw, ErrorCollector collector)
    +     throws SchemaChangeException {
    +    // Input map vector
    +    MapVector mapVector = (MapVector) vw.getValueVector();
    +    MaterializedField mf;
    +    assert mapVector.getPrimitiveVectors().size() > 0;
    +    // Proceed to create output map vector with same name e.g. statcount etc.
    +    MajorType mt = mapVector.getField().getType();
    +    if (functions.get(field) != null) {
    +      mf = MaterializedField.create(functions.get(field), mt);
    +    } else {
    +      mf = MaterializedField.create(field, mt);
    +    }
    +    assert !dataSrcVecMap.containsKey(mf);
    +    ValueVector vector = TypeHelper.getNewVector(mf, oContext.getAllocator());
    +    container.add(vector);
    +    MapVector outputMapVector = (MapVector) vector;
    +
    +    for (ValueVector vv : mapVector) {
    +      String fieldName = vv.getField().getLastName();
    +      if (!keyList.contains(fieldName)) {
    +        throw new UnsupportedOperationException("Unpivot data vector " +
    --- End diff --
    
    Yes, this check is redundant. Removed it.


---
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 #729: Drill 1328: Support table statistics for Parquet

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

    https://github.com/apache/drill/pull/729#discussion_r101180535
  
    --- Diff: exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/statistics/StatisticsMergeBatch.java ---
    @@ -0,0 +1,532 @@
    +/**
    + * 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.impl.statistics;
    +
    +import com.google.common.collect.Lists;
    +import com.google.common.collect.Maps;
    +import org.apache.drill.common.expression.LogicalExpression;
    +import org.apache.drill.common.expression.ErrorCollector;
    +import org.apache.drill.common.expression.ErrorCollectorImpl;
    +import org.apache.drill.common.expression.FunctionCallFactory;
    +import org.apache.drill.common.expression.SchemaPath;
    +import org.apache.drill.common.expression.ValueExpressions;
    +import org.apache.drill.common.types.TypeProtos;
    +import org.apache.drill.common.types.TypeProtos.MajorType;
    +import org.apache.drill.exec.exception.OutOfMemoryException;
    +import org.apache.drill.exec.exception.SchemaChangeException;
    +import org.apache.drill.exec.expr.ExpressionTreeMaterializer;
    +import org.apache.drill.exec.expr.TypeHelper;
    +import org.apache.drill.exec.expr.holders.BigIntHolder;
    +import org.apache.drill.exec.expr.holders.NullableFloat8Holder;
    +import org.apache.drill.exec.expr.holders.ObjectHolder;
    +import org.apache.drill.exec.expr.holders.ValueHolder;
    +import org.apache.drill.exec.ops.FragmentContext;
    +import org.apache.drill.exec.physical.config.StatisticsMerge;
    +import org.apache.drill.exec.record.AbstractSingleRecordBatch;
    +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.VectorContainer;
    +import org.apache.drill.exec.record.VectorWrapper;
    +import org.apache.drill.exec.vector.ValueVector;
    +import org.apache.drill.exec.vector.DateVector;
    +import org.apache.drill.exec.vector.BigIntVector;
    +import org.apache.drill.exec.vector.NullableBigIntVector;
    +import org.apache.drill.exec.vector.NullableFloat8Vector;
    +import org.apache.drill.exec.vector.NullableVarBinaryVector;
    +import org.apache.drill.exec.vector.VarCharVector;
    +import org.apache.drill.exec.vector.complex.MapVector;
    +
    +import java.io.ByteArrayInputStream;
    +import java.io.DataInputStream;
    +import java.io.IOException;
    +import java.util.List;
    +import java.util.GregorianCalendar;
    +import java.util.HashMap;
    +import java.util.Map;
    +import java.util.TimeZone;
    +
    +import com.clearspring.analytics.stream.cardinality.HyperLogLog;
    +
    +public class StatisticsMergeBatch extends AbstractSingleRecordBatch<StatisticsMerge> {
    +  private static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(StatisticsMergeBatch.class);
    +  private Map<String, String> functions;
    +  private boolean first = true;
    +  private boolean finished = false;
    +  private int schema = 0;
    +  private int recordCount = 0;
    +  private List<String> keyList = null;
    +  private Map<MaterializedField, ValueVector> dataSrcVecMap = null;
    +  // Map of non-map fields to VV in the incoming schema
    +  private Map<MaterializedField, ValueVector> copySrcVecMap = null;
    +  private Map<String, Map<String, ValueHolder>> aggregationMap = null;
    +  public StatisticsMergeBatch(StatisticsMerge popConfig, RecordBatch incoming,
    +                              FragmentContext context) throws OutOfMemoryException {
    +    super(popConfig, context, incoming);
    +    this.functions = new HashMap<>();
    +    this.aggregationMap = new HashMap<>();
    +
    +    /*for (String key : popConfig.getFunctions()) {
    +      aggregationMap.put(key, new HashMap<String, ValueHolder>());
    +      if (key.equalsIgnoreCase("statcount") || key.equalsIgnoreCase("nonnullstatcount")) {
    +        functions.put(key, "sum");
    +      } else if (key.equalsIgnoreCase("hll")) {
    +        functions.put(key, "hll_merge");
    +      } else if (key.equalsIgnoreCase("sum_width")) {
    +        functions.put(key, "avg_width");
    +      }
    +    }*/
    +    for (String key : popConfig.getFunctions()) {
    +      if (key.equalsIgnoreCase("sum_width")) {
    +        functions.put(key, "avg_width");
    +      } else if (key.equalsIgnoreCase("hll")) {
    +        functions.put(key, "hll_merge");
    +      } else {
    +        functions.put(key, key);
    +      }
    +      aggregationMap.put(functions.get(key), new HashMap<String, ValueHolder>());
    +    }
    +  }
    +
    +  private void createKeyColumn(String name, LogicalExpression expr, Map<MaterializedField, ValueVector> parentMap)
    +      throws SchemaChangeException {
    +    ErrorCollector collector = new ErrorCollectorImpl();
    +
    +    LogicalExpression mle = ExpressionTreeMaterializer.materialize(expr, incoming, collector,
    +        context.getFunctionRegistry());
    +
    +    MaterializedField outputField = MaterializedField.create(name, mle.getMajorType());
    +    ValueVector vector = TypeHelper.getNewVector(outputField, oContext.getAllocator());
    +    container.add(vector);
    +
    +    if (collector.hasErrors()) {
    --- 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 #729: Drill 1328: Support table statistics for Parquet

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

    https://github.com/apache/drill/pull/729#discussion_r102310795
  
    --- Diff: exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/statistics/StatisticsMergeBatch.java ---
    @@ -0,0 +1,532 @@
    +/**
    + * 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.impl.statistics;
    +
    +import com.google.common.collect.Lists;
    +import com.google.common.collect.Maps;
    +import org.apache.drill.common.expression.LogicalExpression;
    +import org.apache.drill.common.expression.ErrorCollector;
    +import org.apache.drill.common.expression.ErrorCollectorImpl;
    +import org.apache.drill.common.expression.FunctionCallFactory;
    +import org.apache.drill.common.expression.SchemaPath;
    +import org.apache.drill.common.expression.ValueExpressions;
    +import org.apache.drill.common.types.TypeProtos;
    +import org.apache.drill.common.types.TypeProtos.MajorType;
    +import org.apache.drill.exec.exception.OutOfMemoryException;
    +import org.apache.drill.exec.exception.SchemaChangeException;
    +import org.apache.drill.exec.expr.ExpressionTreeMaterializer;
    +import org.apache.drill.exec.expr.TypeHelper;
    +import org.apache.drill.exec.expr.holders.BigIntHolder;
    +import org.apache.drill.exec.expr.holders.NullableFloat8Holder;
    +import org.apache.drill.exec.expr.holders.ObjectHolder;
    +import org.apache.drill.exec.expr.holders.ValueHolder;
    +import org.apache.drill.exec.ops.FragmentContext;
    +import org.apache.drill.exec.physical.config.StatisticsMerge;
    +import org.apache.drill.exec.record.AbstractSingleRecordBatch;
    +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.VectorContainer;
    +import org.apache.drill.exec.record.VectorWrapper;
    +import org.apache.drill.exec.vector.ValueVector;
    +import org.apache.drill.exec.vector.DateVector;
    +import org.apache.drill.exec.vector.BigIntVector;
    +import org.apache.drill.exec.vector.NullableBigIntVector;
    +import org.apache.drill.exec.vector.NullableFloat8Vector;
    +import org.apache.drill.exec.vector.NullableVarBinaryVector;
    +import org.apache.drill.exec.vector.VarCharVector;
    +import org.apache.drill.exec.vector.complex.MapVector;
    +
    +import java.io.ByteArrayInputStream;
    +import java.io.DataInputStream;
    +import java.io.IOException;
    +import java.util.List;
    +import java.util.GregorianCalendar;
    +import java.util.HashMap;
    +import java.util.Map;
    +import java.util.TimeZone;
    +
    +import com.clearspring.analytics.stream.cardinality.HyperLogLog;
    +
    +public class StatisticsMergeBatch extends AbstractSingleRecordBatch<StatisticsMerge> {
    +  private static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(StatisticsMergeBatch.class);
    +  private Map<String, String> functions;
    +  private boolean first = true;
    +  private boolean finished = false;
    +  private int schema = 0;
    +  private int recordCount = 0;
    +  private List<String> keyList = null;
    +  private Map<MaterializedField, ValueVector> dataSrcVecMap = null;
    +  // Map of non-map fields to VV in the incoming schema
    +  private Map<MaterializedField, ValueVector> copySrcVecMap = null;
    +  private Map<String, Map<String, ValueHolder>> aggregationMap = null;
    +  public StatisticsMergeBatch(StatisticsMerge popConfig, RecordBatch incoming,
    +                              FragmentContext context) throws OutOfMemoryException {
    +    super(popConfig, context, incoming);
    +    this.functions = new HashMap<>();
    +    this.aggregationMap = new HashMap<>();
    +
    +    /*for (String key : popConfig.getFunctions()) {
    +      aggregationMap.put(key, new HashMap<String, ValueHolder>());
    +      if (key.equalsIgnoreCase("statcount") || key.equalsIgnoreCase("nonnullstatcount")) {
    +        functions.put(key, "sum");
    +      } else if (key.equalsIgnoreCase("hll")) {
    +        functions.put(key, "hll_merge");
    +      } else if (key.equalsIgnoreCase("sum_width")) {
    +        functions.put(key, "avg_width");
    +      }
    +    }*/
    +    for (String key : popConfig.getFunctions()) {
    +      if (key.equalsIgnoreCase("sum_width")) {
    +        functions.put(key, "avg_width");
    +      } else if (key.equalsIgnoreCase("hll")) {
    +        functions.put(key, "hll_merge");
    +      } else {
    +        functions.put(key, key);
    +      }
    +      aggregationMap.put(functions.get(key), new HashMap<String, ValueHolder>());
    +    }
    +  }
    +
    +  private void createKeyColumn(String name, LogicalExpression expr, Map<MaterializedField, ValueVector> parentMap)
    +      throws SchemaChangeException {
    +    ErrorCollector collector = new ErrorCollectorImpl();
    +
    +    LogicalExpression mle = ExpressionTreeMaterializer.materialize(expr, incoming, collector,
    +        context.getFunctionRegistry());
    +
    +    MaterializedField outputField = MaterializedField.create(name, mle.getMajorType());
    +    ValueVector vector = TypeHelper.getNewVector(outputField, oContext.getAllocator());
    +    container.add(vector);
    +
    +    if (collector.hasErrors()) {
    +      throw new SchemaChangeException("Failure while materializing expression. "
    +          + collector.toErrorString());
    +    }
    +    parentMap.put(outputField, vector);
    +  }
    +
    +  private ValueVector addMapVector(String name, MapVector parent, LogicalExpression expr) throws SchemaChangeException {
    +    ErrorCollector collector = new ErrorCollectorImpl();
    +
    +    LogicalExpression mle = ExpressionTreeMaterializer.materialize(expr, incoming, collector,
    +        context.getFunctionRegistry());
    +
    +    Class<? extends ValueVector> vvc =
    +        TypeHelper.getValueVectorClass(mle.getMajorType().getMinorType(), mle.getMajorType().getMode());
    +    ValueVector vector = parent.addOrGet(name, mle.getMajorType(), vvc);
    +
    +    if (collector.hasErrors()) {
    +      throw new SchemaChangeException("Failure while materializing expression. "
    +          + collector.toErrorString());
    +    }
    +    return vector;
    +  }
    +
    +  /**
    +   * Identify the list of fields within a map which are unpivoted as columns in output
    +   */
    +  private void buildKeyList() {
    +    List<String> lastMapKeyList = null;
    +    for (VectorWrapper<?> vw : incoming) {
    +      if (vw.getField().getType().getMinorType() != TypeProtos.MinorType.MAP) {
    +        continue;
    +      }
    +
    +      keyList = Lists.newArrayList();
    +
    +      for (ValueVector vv : vw.getValueVector()) {
    +        keyList.add(vv.getField().getLastName());
    +      }
    +
    +      if (lastMapKeyList == null) {
    +        lastMapKeyList = keyList;
    +      } else {
    +        if (keyList.size() != lastMapKeyList.size() || !lastMapKeyList.containsAll(keyList)) {
    +          throw new UnsupportedOperationException("Maps have different fields");
    +        }
    +      }
    +    }
    +  }
    +
    +  private void buildOutputContainer() throws SchemaChangeException {
    +    dataSrcVecMap = Maps.newHashMap();
    +    copySrcVecMap = Maps.newHashMap();
    +    MajorType mt = null;
    +
    +    ErrorCollector collector = new ErrorCollectorImpl();
    +    GregorianCalendar calendar = new GregorianCalendar(TimeZone.getTimeZone("UTC"));
    +
    +    calendar.setTimeInMillis(System.currentTimeMillis());
    +    createKeyColumn("schema", ValueExpressions.getBigInt(schema++), copySrcVecMap);
    +    createKeyColumn("computed", ValueExpressions.getDate(calendar), copySrcVecMap);
    +
    +    for (VectorWrapper<?> vw : incoming) {
    +      addVectorToOutgoingContainer(vw.getField().getLastName(), vw, collector);
    +      /*MaterializedField ds = vw.getField();
    +      String field = vw.getField().getLastName();
    +      // Input map vector
    +      MapVector mapVector = (MapVector) vw.getValueVector();
    +      assert mapVector.getPrimitiveVectors().size() > 0;
    +      // Proceed to create output map vector with same name e.g. statcount etc.
    +      mt = mapVector.getField().getType();
    +      MaterializedField mf = MaterializedField.create(functions.get(field), mt);
    +      assert !dataSrcVecMap.containsKey(mf);
    +      ValueVector vector = TypeHelper.getNewVector(mf, oContext.getAllocator());
    +      container.add(vector);
    +      MapVector outputMapVector = (MapVector) vector;
    +
    +      for (ValueVector vv : mapVector) {
    +        String fieldName = vv.getField().getLastName();
    +        if (!keyList.contains(fieldName)) {
    +          throw new UnsupportedOperationException("Unpivot data vector " +
    +              ds + " contains key " + fieldName + " not contained in key source!");
    +        }
    +        if (vv.getField().getType().getMinorType() == TypeProtos.MinorType.MAP) {
    +          throw new UnsupportedOperationException("Unpivot of nested map is not supported!");
    +        }
    +        if (field.equals("column")) {
    +          outputMapVector.addOrGet(fieldName, vv.getField().getType(), vv.getClass());
    +        } else {
    +          List<LogicalExpression> args = Lists.newArrayList();
    +          //TODO: Something else to access value of col such as emp_id?
    +          args.add(SchemaPath.getSimplePath(vv.getField().getPath()));
    +          //TODO: Put in the mapVector
    +          LogicalExpression call = FunctionCallFactory.createExpression(functions.get(field), args);
    +          //TODO: Is this sufficient to add to new Map?
    +          ValueVector vector1 = addMapVector(fieldName, outputMapVector, call);
    +          if (collector.hasErrors()) {
    +            throw new SchemaChangeException("Failure while materializing expression. "
    +                + collector.toErrorString());
    +          }
    +        }
    +      }
    +      dataSrcVecMap.put(ds, outputMapVector);*/
    +    }
    +    //Now create NDV in the outgoing container which was not avaliable in the incoming
    +    for (VectorWrapper<?> vw : incoming) {
    +      if (vw.getField().getLastName().equalsIgnoreCase("sum_width")) {//NullableFloat8 type
    +        addVectorToOutgoingContainer("ndv", vw, collector);
    +        break;
    +      }
    +    }
    +    container.setRecordCount(0);
    +    recordCount = 0;
    +    container.buildSchema(incoming.getSchema().getSelectionVectorMode());
    +  }
    +
    +  private void addVectorToOutgoingContainer(String field, VectorWrapper vw, ErrorCollector collector)
    +     throws SchemaChangeException {
    +    // Input map vector
    +    MapVector mapVector = (MapVector) vw.getValueVector();
    +    MaterializedField mf;
    +    assert mapVector.getPrimitiveVectors().size() > 0;
    +    // Proceed to create output map vector with same name e.g. statcount etc.
    +    MajorType mt = mapVector.getField().getType();
    +    if (functions.get(field) != null) {
    +      mf = MaterializedField.create(functions.get(field), mt);
    +    } else {
    +      mf = MaterializedField.create(field, mt);
    +    }
    +    assert !dataSrcVecMap.containsKey(mf);
    +    ValueVector vector = TypeHelper.getNewVector(mf, oContext.getAllocator());
    +    container.add(vector);
    +    MapVector outputMapVector = (MapVector) vector;
    +
    +    for (ValueVector vv : mapVector) {
    +      String fieldName = vv.getField().getLastName();
    +      if (!keyList.contains(fieldName)) {
    +        throw new UnsupportedOperationException("Unpivot data vector " +
    +                field + " contains key " + fieldName + " not contained in key source!");
    +      }
    +      if (vv.getField().getType().getMinorType() == TypeProtos.MinorType.MAP) {
    +        throw new UnsupportedOperationException("Unpivot of nested map is not supported!");
    +      }
    +      if (field.equals("column")) {
    +        outputMapVector.addOrGet(fieldName, vv.getField().getType(), vv.getClass());
    +      } else {
    +        List<LogicalExpression> args = Lists.newArrayList();
    +        LogicalExpression call;
    +        //TODO: Something else to access value of col such as emp_id?
    +        args.add(SchemaPath.getSimplePath(vv.getField().getPath()));
    +        //TODO: Put in the mapVector
    +        if (functions.get(field) != null) {
    +          call = FunctionCallFactory.createExpression(functions.get(field), args);
    +        } else {
    +          call = FunctionCallFactory.createExpression(field, args);
    +        }
    +        //TODO: Is this sufficient to add to new Map?
    +        ValueVector vector1 = addMapVector(fieldName, outputMapVector, call);
    +        if (collector.hasErrors()) {
    +          throw new SchemaChangeException("Failure while materializing expression. "
    +                  + collector.toErrorString());
    +        }
    +      }
    +    }
    +  }
    +
    +  @Override
    +  protected boolean setupNewSchema() throws SchemaChangeException {
    +    container.clear();
    +    buildKeyList();
    +    buildOutputContainer();
    +    return true;
    +  }
    +
    +  @Override
    +  protected IterOutcome doWork() {
    +    int outRecordCount = incoming.getRecordCount();
    +    HashMap<String, Long> nonNullRows = new HashMap<>();
    +
    +    for (VectorWrapper<?> vw : incoming) {
    +      ValueVector vv = vw.getValueVector();
    +      if (vv.getField().getType().getMinorType() != TypeProtos.MinorType.MAP) {
    +        //We only expect Maps in the incoming. STOP, if this is not the case.
    +        return IterOutcome.STOP;
    +      }
    +      MapVector mapVec = (MapVector) vv;
    +      Map<String, ValueHolder> statMap = aggregationMap.get(vv.getField().getLastName());
    +
    +      for (ValueVector mapElt : mapVec) {
    +        if (vv.getField().getLastName().equalsIgnoreCase("column")) {
    +          BigIntHolder nameHolder;
    +          if (statMap.get(mapElt.getField().getLastName()) != null) {
    +            nameHolder = (BigIntHolder) statMap.get(mapElt.getField().getLastName());
    +          } else {
    +            nameHolder = new BigIntHolder();
    +            statMap.put(mapElt.getField().getLastName(), nameHolder);
    +          }
    +          nameHolder.value = 1;
    +        } else if (vv.getField().getLastName().equalsIgnoreCase("statcount")
    +            || vv.getField().getLastName().equalsIgnoreCase("nonnullstatcount")) {
    +          BigIntHolder sumHolder;
    +          String colName = mapElt.getField().getLastName();
    +          if (statMap.get(colName) != null) {
    +            sumHolder = (BigIntHolder) statMap.get(colName);
    +          } else {
    +            sumHolder = new BigIntHolder();
    +            statMap.put(colName, sumHolder);
    +          }
    +          //TODO: assert size = 1
    +          //TODO: logger
    +          Object val = mapElt.getAccessor().getObject(0);
    +          if (val != null) {
    +            sumHolder.value += (long)val;
    +          }
    +        } else if (vv.getField().getLastName().equalsIgnoreCase("sum_width")) {
    +          NullableFloat8Holder sumHolder;
    +          String colName = mapElt.getField().getLastName();
    +          if (statMap == null) {
    +            statMap = aggregationMap.get(functions.get(vv.getField().getLastName()));
    +          }
    +          if (statMap.get(colName) != null) {
    +            sumHolder = (NullableFloat8Holder) statMap.get(colName);
    +          } else {
    +            sumHolder = new NullableFloat8Holder();
    +            statMap.put(colName, sumHolder);
    +          }
    +          //TODO: assert size = 1
    +          //TODO: logger
    +          Object val = mapElt.getAccessor().getObject(0);
    +          if (val != null) {
    +            sumHolder.value += (double) val;
    +            sumHolder.isSet = 1;
    +          }
    +        } else if (vv.getField().getLastName().equalsIgnoreCase("hll")) {
    +          ObjectHolder hllHolder;
    +          String colName = mapElt.getField().getLastName();
    +          if (statMap == null) {
    +            statMap = aggregationMap.get(functions.get(vv.getField().getLastName()));
    +          }
    +          if (statMap.get(colName) != null) {
    +            hllHolder = (ObjectHolder) statMap.get(colName);
    +          } else {
    +            hllHolder = new ObjectHolder();
    +            hllHolder.obj = new HyperLogLog(context.getContextInformation().getHllAccuracy());
    +            statMap.put(colName, hllHolder);
    +          }
    +          NullableVarBinaryVector hllVector = (NullableVarBinaryVector) mapElt;
    +          try {
    +            if (hllVector.getAccessor().isSet(0) == 1) {
    +              ByteArrayInputStream bais = new ByteArrayInputStream(hllVector.getAccessor().getObject(0), 0,
    +                  mapElt.getBufferSize());
    +              HyperLogLog other = HyperLogLog.Builder.build(new DataInputStream(bais));
    +              ((HyperLogLog) hllHolder.obj).addAll(other);
    +            }
    +          } catch (Exception ex) {
    +            //TODO: Catch IOException/CardinalityMergeException
    +            //TODO: logger
    +            return IterOutcome.STOP;
    +          }
    +        }
    +      }
    +      // Add NDV value vector map using HLL map (since the NDV map is directly generated from HLL and not produced by the underlying
    +      // Statistics Agg)
    +      Map<String, ValueHolder> hllMap = aggregationMap.get("hll");
    +      if (hllMap != null) {
    +        aggregationMap.put("ndv", hllMap);
    +      }
    +    }
    +    return IterOutcome.OK;
    +  }
    +
    +  public VectorContainer getOutgoingContainer() {
    +    return this.container;
    +  }
    +
    +  @Override
    +  public IterOutcome innerNext() {
    +    IterOutcome outcome;
    +    boolean didSomeWork = false;
    +    if (finished) {
    +      return IterOutcome.NONE;
    +    }
    +    try {
    +      outer: while (true) {
    +        outcome = next(incoming);
    +        switch (outcome) {
    +          case NONE:
    +            break outer;
    +          case OUT_OF_MEMORY:
    +          case NOT_YET:
    +          case STOP:
    +            return outcome;
    +          case OK_NEW_SCHEMA:
    +            if (first) {
    +              first =false;
    +              if (!setupNewSchema()) {
    +                outcome = IterOutcome.OK;
    +              }
    +              return outcome;
    +            }
    +            //fall through
    +          case OK:
    +            assert first == false : "First batch should be OK_NEW_SCHEMA";
    +            IterOutcome out = doWork();
    +            didSomeWork = true;
    +            if (out != IterOutcome.OK) {
    +              return out;
    +            }
    +            break;
    +          default:
    +            throw new UnsupportedOperationException("Unsupported upstream state " + outcome);
    +        }
    +      }
    +    } catch (SchemaChangeException ex) {
    +      kill(false);
    +      logger.error("Failure during query", ex);
    +      context.fail(ex);
    +      return IterOutcome.STOP;
    +    }
    +
    +    // We can only get here if upstream is NONE i.e. no more batches. If we did some work prior to
    +    // exhausting all upstream, then return OK. Otherwise, return NONE.
    +    if (didSomeWork) {
    +      IterOutcome out = buildOutgoingRecordBatch();
    +      finished = true;
    +      return out;
    +    } else {
    +      return outcome;
    +    }
    +  }
    +
    +  // Prepare the outgoing container
    +  private IterOutcome buildOutgoingRecordBatch() {
    --- End diff --
    
    Refactored the code. Please take a look.


---
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 #729: Drill 1328: Support table statistics for Parquet

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/729#discussion_r100680510
  
    --- Diff: exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/statistics/StatisticsMergeBatch.java ---
    @@ -0,0 +1,532 @@
    +/**
    + * 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.impl.statistics;
    +
    +import com.google.common.collect.Lists;
    +import com.google.common.collect.Maps;
    +import org.apache.drill.common.expression.LogicalExpression;
    +import org.apache.drill.common.expression.ErrorCollector;
    +import org.apache.drill.common.expression.ErrorCollectorImpl;
    +import org.apache.drill.common.expression.FunctionCallFactory;
    +import org.apache.drill.common.expression.SchemaPath;
    +import org.apache.drill.common.expression.ValueExpressions;
    +import org.apache.drill.common.types.TypeProtos;
    +import org.apache.drill.common.types.TypeProtos.MajorType;
    +import org.apache.drill.exec.exception.OutOfMemoryException;
    +import org.apache.drill.exec.exception.SchemaChangeException;
    +import org.apache.drill.exec.expr.ExpressionTreeMaterializer;
    +import org.apache.drill.exec.expr.TypeHelper;
    +import org.apache.drill.exec.expr.holders.BigIntHolder;
    +import org.apache.drill.exec.expr.holders.NullableFloat8Holder;
    +import org.apache.drill.exec.expr.holders.ObjectHolder;
    +import org.apache.drill.exec.expr.holders.ValueHolder;
    +import org.apache.drill.exec.ops.FragmentContext;
    +import org.apache.drill.exec.physical.config.StatisticsMerge;
    +import org.apache.drill.exec.record.AbstractSingleRecordBatch;
    +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.VectorContainer;
    +import org.apache.drill.exec.record.VectorWrapper;
    +import org.apache.drill.exec.vector.ValueVector;
    +import org.apache.drill.exec.vector.DateVector;
    +import org.apache.drill.exec.vector.BigIntVector;
    +import org.apache.drill.exec.vector.NullableBigIntVector;
    +import org.apache.drill.exec.vector.NullableFloat8Vector;
    +import org.apache.drill.exec.vector.NullableVarBinaryVector;
    +import org.apache.drill.exec.vector.VarCharVector;
    +import org.apache.drill.exec.vector.complex.MapVector;
    +
    +import java.io.ByteArrayInputStream;
    +import java.io.DataInputStream;
    +import java.io.IOException;
    +import java.util.List;
    +import java.util.GregorianCalendar;
    +import java.util.HashMap;
    +import java.util.Map;
    +import java.util.TimeZone;
    +
    +import com.clearspring.analytics.stream.cardinality.HyperLogLog;
    +
    +public class StatisticsMergeBatch extends AbstractSingleRecordBatch<StatisticsMerge> {
    +  private static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(StatisticsMergeBatch.class);
    +  private Map<String, String> functions;
    +  private boolean first = true;
    +  private boolean finished = false;
    +  private int schema = 0;
    +  private int recordCount = 0;
    +  private List<String> keyList = null;
    +  private Map<MaterializedField, ValueVector> dataSrcVecMap = null;
    +  // Map of non-map fields to VV in the incoming schema
    +  private Map<MaterializedField, ValueVector> copySrcVecMap = null;
    +  private Map<String, Map<String, ValueHolder>> aggregationMap = null;
    +  public StatisticsMergeBatch(StatisticsMerge popConfig, RecordBatch incoming,
    +                              FragmentContext context) throws OutOfMemoryException {
    +    super(popConfig, context, incoming);
    +    this.functions = new HashMap<>();
    +    this.aggregationMap = new HashMap<>();
    +
    +    /*for (String key : popConfig.getFunctions()) {
    +      aggregationMap.put(key, new HashMap<String, ValueHolder>());
    +      if (key.equalsIgnoreCase("statcount") || key.equalsIgnoreCase("nonnullstatcount")) {
    +        functions.put(key, "sum");
    +      } else if (key.equalsIgnoreCase("hll")) {
    +        functions.put(key, "hll_merge");
    +      } else if (key.equalsIgnoreCase("sum_width")) {
    +        functions.put(key, "avg_width");
    +      }
    +    }*/
    +    for (String key : popConfig.getFunctions()) {
    +      if (key.equalsIgnoreCase("sum_width")) {
    +        functions.put(key, "avg_width");
    +      } else if (key.equalsIgnoreCase("hll")) {
    +        functions.put(key, "hll_merge");
    +      } else {
    +        functions.put(key, key);
    +      }
    +      aggregationMap.put(functions.get(key), new HashMap<String, ValueHolder>());
    +    }
    +  }
    +
    +  private void createKeyColumn(String name, LogicalExpression expr, Map<MaterializedField, ValueVector> parentMap)
    +      throws SchemaChangeException {
    +    ErrorCollector collector = new ErrorCollectorImpl();
    +
    +    LogicalExpression mle = ExpressionTreeMaterializer.materialize(expr, incoming, collector,
    +        context.getFunctionRegistry());
    +
    +    MaterializedField outputField = MaterializedField.create(name, mle.getMajorType());
    +    ValueVector vector = TypeHelper.getNewVector(outputField, oContext.getAllocator());
    +    container.add(vector);
    +
    +    if (collector.hasErrors()) {
    +      throw new SchemaChangeException("Failure while materializing expression. "
    +          + collector.toErrorString());
    +    }
    +    parentMap.put(outputField, vector);
    +  }
    +
    +  private ValueVector addMapVector(String name, MapVector parent, LogicalExpression expr) throws SchemaChangeException {
    +    ErrorCollector collector = new ErrorCollectorImpl();
    +
    +    LogicalExpression mle = ExpressionTreeMaterializer.materialize(expr, incoming, collector,
    +        context.getFunctionRegistry());
    +
    +    Class<? extends ValueVector> vvc =
    +        TypeHelper.getValueVectorClass(mle.getMajorType().getMinorType(), mle.getMajorType().getMode());
    +    ValueVector vector = parent.addOrGet(name, mle.getMajorType(), vvc);
    +
    +    if (collector.hasErrors()) {
    +      throw new SchemaChangeException("Failure while materializing expression. "
    +          + collector.toErrorString());
    +    }
    +    return vector;
    +  }
    +
    +  /**
    +   * Identify the list of fields within a map which are unpivoted as columns in output
    +   */
    +  private void buildKeyList() {
    --- End diff --
    
    This method is rather passive and reactive. Looks like we are building up a list of original data columns. (The "key" in the name is too generic; this is the input column list.)
    
    As it turns out, we have a map that does just that: the "columns" map. So, seems we should take that map as the known good set of names. Then, if we want to be paranoid, compare the columns map against the others.
    
    While we are at it, should we compare the set of maps against the expected set of functions?


---
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 #729: Drill 1328: Support table statistics for Parquet

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/729#discussion_r100681239
  
    --- Diff: exec/java-exec/src/main/java/org/apache/drill/exec/planner/common/DrillStatsTable.java ---
    @@ -0,0 +1,347 @@
    +/**
    + * 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.planner.common;
    +
    +import java.io.IOException;
    +import java.util.List;
    +import java.util.Map;
    +import java.util.concurrent.TimeUnit;
    +import com.fasterxml.jackson.annotation.JsonIgnore;
    +import com.fasterxml.jackson.annotation.JsonGetter;
    +import com.fasterxml.jackson.annotation.JsonSetter;
    +import com.fasterxml.jackson.annotation.JsonProperty;
    +import com.fasterxml.jackson.annotation.JsonSubTypes;
    +import com.fasterxml.jackson.annotation.JsonTypeInfo;
    +import com.fasterxml.jackson.annotation.JsonTypeName;
    +import com.fasterxml.jackson.databind.DeserializationFeature;
    +import com.fasterxml.jackson.databind.ObjectMapper;
    +import com.google.common.base.Stopwatch;
    +import com.google.common.collect.Maps;
    +import org.apache.calcite.rel.RelNode;
    +import org.apache.calcite.rel.RelVisitor;
    +import org.apache.calcite.rel.core.TableScan;
    +import org.apache.drill.common.exceptions.DrillRuntimeException;
    +import org.apache.drill.exec.ops.QueryContext;
    +import org.apache.drill.exec.planner.logical.DrillTable;
    +import org.apache.drill.exec.util.ImpersonationUtil;
    +import org.apache.hadoop.fs.FSDataInputStream;
    +import org.apache.hadoop.fs.FileSystem;
    +import org.apache.hadoop.fs.Path;
    +import org.joda.time.DateTime;
    +
    +/**
    + * Wraps the stats table info including schema and tableName. Also materializes stats from storage
    + * and keeps them in memory.
    + */
    +public class DrillStatsTable {
    +  private static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(DrillStatsTable.class);
    +  private final FileSystem fs;
    +  private final Path tablePath;
    +
    +  /**
    +   * List of columns in stats table.
    +   */
    +  public static final String COL_COLUMN = "column";
    +  public static final String COL_COMPUTED = "computed";
    +  public static final String COL_STATCOUNT = "statcount";
    +  public static final String COL_NDV = "ndv";
    +
    +  private final String schemaName;
    +  private final String tableName;
    +
    +  private final Map<String, Long> ndv = Maps.newHashMap();
    +  private double rowCount = -1;
    +
    +  private boolean materialized = false;
    +
    +  private TableStatistics statistics = null;
    +
    +  public DrillStatsTable(String schemaName, String tableName, Path tablePath, FileSystem fs) {
    +    this.schemaName = schemaName;
    +    this.tableName = tableName;
    +    this.tablePath = tablePath;
    +    this.fs = ImpersonationUtil.createFileSystem(ImpersonationUtil.getProcessUserName(), fs.getConf());
    +  }
    +
    +  public String getSchemaName() {
    +    return schemaName;
    +  }
    +
    +  public String getTableName() {
    +    return tableName;
    +  }
    +  /**
    +   * Get number of distinct values of given column. If stats are not present for the given column,
    +   * a null is returned.
    +   *
    +   * Note: returned data may not be accurate. Accuracy depends on whether the table data has changed after the
    +   * stats are computed.
    +   *
    +   * @param col
    +   * @return
    +   */
    +  public Double getNdv(String col) {
    +    // Stats might not have materialized because of errors.
    +    if (!materialized) {
    +      return null;
    +    }
    +    final String upperCol = col.toUpperCase();
    +    final Long ndvCol = ndv.get(upperCol);
    +    // Ndv estimation techniques like HLL may over-estimate, hence cap it at rowCount
    +    if (ndvCol != null) {
    +      return Math.min(ndvCol, rowCount);
    --- End diff --
    
    The resulting value, while seemingly useful, may have severe problems. Today we estimate p(a = value) at 15%. This is big data, many columns will have a large NDV. Thus, p(a = value) = 1/NDV which may be a very small number.
    
    When data is uniformly distributed, this may be fine. But, if data is skewed (mostly value "A", with a sprinkling of "B" through "Z"), the estimate will be far off. We may end up thinking that any equality filter greatly reduces row counts when it does not actually do so. Feed the result into the build size of a hash join and "bad things happen."
    
    Should we be more conservative? Set some minimum value? Take a risk-based approach to deciding which side of hash join to be the build side?


---
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 #729: Drill 1328: Support table statistics for Parquet

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

    https://github.com/apache/drill/pull/729#discussion_r100709564
  
    --- Diff: exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/statistics/StatisticsAggBatch.java ---
    @@ -0,0 +1,256 @@
    +/**
    + * 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.impl.statistics;
    +
    +import com.google.common.collect.Lists;
    +import com.sun.codemodel.JExpr;
    +import org.apache.drill.common.expression.ErrorCollector;
    +import org.apache.drill.common.expression.ErrorCollectorImpl;
    +import org.apache.drill.common.expression.FunctionCallFactory;
    +import org.apache.drill.common.expression.LogicalExpression;
    +import org.apache.drill.common.expression.SchemaPath;
    +import org.apache.drill.common.expression.ValueExpressions;
    +import org.apache.drill.exec.exception.ClassTransformationException;
    +import org.apache.drill.exec.exception.OutOfMemoryException;
    +import org.apache.drill.exec.exception.SchemaChangeException;
    +import org.apache.drill.exec.expr.ClassGenerator;
    +import org.apache.drill.exec.expr.CodeGenerator;
    +import org.apache.drill.exec.expr.ExpressionTreeMaterializer;
    +import org.apache.drill.exec.expr.TypeHelper;
    +import org.apache.drill.exec.expr.ValueVectorWriteExpression;
    +import org.apache.drill.exec.ops.FragmentContext;
    +import org.apache.drill.exec.physical.config.StatisticsAggregate;
    +import org.apache.drill.exec.physical.impl.aggregate.StreamingAggBatch;
    +import org.apache.drill.exec.physical.impl.aggregate.StreamingAggTemplate;
    +import org.apache.drill.exec.physical.impl.aggregate.StreamingAggregator;
    +import org.apache.drill.exec.planner.physical.StatsAggPrel.OperatorPhase;
    +import org.apache.drill.exec.record.BatchSchema.SelectionVectorMode;
    +import org.apache.drill.exec.record.MaterializedField;
    +import org.apache.drill.exec.record.RecordBatch;
    +import org.apache.drill.exec.record.TypedFieldId;
    +import org.apache.drill.exec.store.ImplicitColumnExplorer;
    +import org.apache.drill.exec.vector.ValueVector;
    +import org.apache.drill.exec.vector.complex.FieldIdUtil;
    +import org.apache.drill.exec.vector.complex.MapVector;
    +
    +import java.io.IOException;
    +import java.util.GregorianCalendar;
    +import java.util.List;
    +import java.util.TimeZone;
    +
    +/**
    + * TODO: This needs cleanup. Currently the key values are constants and we compare the constants
    + * for every record. Seems unnecessary.
    + *
    + * Example input and output:
    + * Schema of incoming batch: region_id (VARCHAR), sales_city (VARCHAR), cnt (BIGINT)
    + * Schema of output:
    + *    "schema" : BIGINT - Schema number. For each schema change this number is incremented.
    + *    "computed" : BIGINT - What time is it computed?
    + *    "columns"       : MAP - Column names
    + *       "region_id"  : VARCHAR
    + *       "sales_city" : VARCHAR
    + *       "cnt"        : VARCHAR
    + *    "statscount" : MAP
    + *       "region_id"  : BIGINT - statscount(region_id) - aggregation over all values of region_id
    + *                      in incoming batch
    + *       "sales_city" : BIGINT - statscount(sales_city)
    + *       "cnt"        : BIGINT - statscount(cnt)
    + *    "nonnullstatcount" : MAP
    + *       "region_id"  : BIGINT - nonnullstatcount(region_id)
    + *       "sales_city" : BIGINT - nonnullstatcount(sales_city)
    + *       "cnt"        : BIGINT - nonnullstatcount(cnt)
    + *   .... another map for next stats function ....
    + */
    +public class StatisticsAggBatch extends StreamingAggBatch {
    +  private List<String> functions;
    +  private int schema = 0;
    +
    +  public StatisticsAggBatch(StatisticsAggregate popConfig, RecordBatch incoming,
    +      FragmentContext context) throws OutOfMemoryException {
    +    super(popConfig, incoming, context);
    +    this.functions = popConfig.getFunctions();
    +  }
    +
    +  private void createKeyColumn(String name, LogicalExpression expr, List<LogicalExpression> keyExprs,
    +      List<TypedFieldId> keyOutputIds) throws SchemaChangeException {
    +    ErrorCollector collector = new ErrorCollectorImpl();
    +
    +    LogicalExpression mle = ExpressionTreeMaterializer.materialize(expr, incoming, collector,
    +        context.getFunctionRegistry());
    +
    +    MaterializedField outputField = MaterializedField.create(name, mle.getMajorType());
    +    ValueVector vector = TypeHelper.getNewVector(outputField, oContext.getAllocator());
    +
    +    keyExprs.add(mle);
    +    keyOutputIds.add(container.add(vector));
    +
    +    if (collector.hasErrors()) {
    +      throw new SchemaChangeException("Failure while materializing expression. "
    +          + collector.toErrorString());
    +    }
    +  }
    +
    +  private void createNestedKeyColumn(MapVector parent, String name, LogicalExpression expr,
    +      List<LogicalExpression> keyExprs, List<TypedFieldId> keyOutputIds)
    +          throws SchemaChangeException {
    +    ErrorCollector collector = new ErrorCollectorImpl();
    +
    +    LogicalExpression mle = ExpressionTreeMaterializer.materialize(expr, incoming, collector,
    +        context.getFunctionRegistry());
    +
    +    Class<? extends ValueVector> vvc =
    +        TypeHelper.getValueVectorClass(mle.getMajorType().getMinorType(),
    +            mle.getMajorType().getMode());
    +
    +    ValueVector vv = parent.addOrGet(name, mle.getMajorType(), vvc);
    +
    +    TypedFieldId pfid = container.getValueVectorId(SchemaPath.getSimplePath(parent.getField().getPath()));
    +    assert pfid.getFieldIds().length == 1;
    +    TypedFieldId.Builder builder = TypedFieldId.newBuilder();
    +    builder.addId(pfid.getFieldIds()[0]);
    +    TypedFieldId id =
    +        FieldIdUtil.getFieldIdIfMatches(parent, builder, true,
    +            SchemaPath.getSimplePath(vv.getField().getPath()).getRootSegment());
    +
    +    keyExprs.add(mle);
    +    keyOutputIds.add(id);
    +
    +    if (collector.hasErrors()) {
    +      throw new SchemaChangeException("Failure while materializing expression. "
    +          + collector.toErrorString());
    +    }
    +  }
    +
    +  private void addMapVector(String name, MapVector parent, LogicalExpression expr,
    +      List<LogicalExpression> valueExprs) throws SchemaChangeException {
    +    ErrorCollector collector = new ErrorCollectorImpl();
    +
    +    LogicalExpression mle = ExpressionTreeMaterializer.materialize(expr, incoming, collector,
    +        context.getFunctionRegistry());
    +
    +    Class<? extends ValueVector> vvc =
    +        TypeHelper.getValueVectorClass(mle.getMajorType().getMinorType(), mle.getMajorType().getMode());
    +    ValueVector vv = parent.addOrGet(name, mle.getMajorType(), vvc);
    +
    +    TypedFieldId pfid = container.getValueVectorId(SchemaPath.getSimplePath(parent.getField().getPath()));
    +    assert pfid.getFieldIds().length == 1;
    +    TypedFieldId.Builder builder = TypedFieldId.newBuilder();
    +    builder.addId(pfid.getFieldIds()[0]);
    +    TypedFieldId id = FieldIdUtil.getFieldIdIfMatches(parent, builder, true,
    +        SchemaPath.getSimplePath(vv.getField().getPath()).getRootSegment());
    +
    +    valueExprs.add(new ValueVectorWriteExpression(id, mle, true));
    +
    +    if (collector.hasErrors()) {
    +      throw new SchemaChangeException("Failure while materializing expression. "
    +          + collector.toErrorString());
    +    }
    +  }
    +
    +  private StreamingAggregator codegenAggregator(List<LogicalExpression> keyExprs,
    +      List<LogicalExpression> valueExprs, List<TypedFieldId> keyOutputIds)
    +          throws SchemaChangeException, ClassTransformationException, IOException {
    +    ClassGenerator<StreamingAggregator> cg =
    +        CodeGenerator.getRoot(StreamingAggTemplate.TEMPLATE_DEFINITION, context.getFunctionRegistry(),
    +            context.getOptions());
    +
    +    LogicalExpression[] keyExprsArray = new LogicalExpression[keyExprs.size()];
    +    LogicalExpression[] valueExprsArray = new LogicalExpression[valueExprs.size()];
    +    TypedFieldId[] keyOutputIdsArray = new TypedFieldId[keyOutputIds.size()];
    +
    +    keyExprs.toArray(keyExprsArray);
    +    valueExprs.toArray(valueExprsArray);
    +    keyOutputIds.toArray(keyOutputIdsArray);
    +
    +    setupIsSame(cg, keyExprsArray);
    +    setupIsSameApart(cg, keyExprsArray);
    +    addRecordValues(cg, valueExprsArray);
    +    outputRecordKeys(cg, keyOutputIdsArray, keyExprsArray);
    +    outputRecordKeysPrev(cg, keyOutputIdsArray, keyExprsArray);
    +
    +    cg.getBlock("resetValues")._return(JExpr.TRUE);
    +    getIndex(cg);
    +
    +    container.buildSchema(SelectionVectorMode.NONE);
    +    StreamingAggregator agg = context.getImplementationClass(cg);
    +    agg.setup(oContext, incoming, this);
    +    return agg;
    +  }
    +
    +  private boolean isImplicitFileColumn(MaterializedField mf) {
    +    return ImplicitColumnExplorer.initImplicitFileColumns(context.getOptions()).get(mf.getName()) != null;
    +  }
    +
    +  protected StreamingAggregator createAggregatorInternal()
    +      throws SchemaChangeException, ClassTransformationException, IOException {
    +    container.clear();
    +
    +    List<LogicalExpression> keyExprs = Lists.newArrayList();
    +    List<LogicalExpression> valueExprs = Lists.newArrayList();
    +    List<TypedFieldId> keyOutputIds = Lists.newArrayList();
    +    GregorianCalendar calendar = new GregorianCalendar(TimeZone.getTimeZone("UTC"));
    +    calendar.setTimeInMillis(System.currentTimeMillis());
    +
    +    if (this.getPopConfig() instanceof StatisticsAggregate
    +        && (((StatisticsAggregate) this.getPopConfig()).getPhase() == OperatorPhase.PHASE_1of1
    +            || ((StatisticsAggregate) this.getPopConfig()).getPhase() == OperatorPhase.PHASE_2of2)) {
    --- End diff --
    
    This code was refactored out (no longer present).


---
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 #729: Drill 1328 r4

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

    https://github.com/apache/drill/pull/729#discussion_r98317444
  
    --- Diff: exec/java-exec/src/main/java/org/apache/drill/exec/planner/sql/handlers/AnalyzeTableHandler.java ---
    @@ -0,0 +1,256 @@
    +/**
    + * 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.planner.sql.handlers;
    +
    +import java.io.IOException;
    +import java.util.List;
    +
    +import org.apache.calcite.rel.RelNode;
    +import org.apache.calcite.schema.Table;
    +import org.apache.calcite.sql.SqlIdentifier;
    +import org.apache.calcite.sql.SqlNode;
    +import org.apache.calcite.sql.SqlNodeList;
    +import org.apache.calcite.sql.SqlSelect;
    +import org.apache.calcite.sql.parser.SqlParserPos;
    +import org.apache.calcite.tools.RelConversionException;
    +import org.apache.calcite.tools.ValidationException;
    +import org.apache.drill.common.exceptions.UserException;
    +import org.apache.drill.common.logical.FormatPluginConfig;
    +import org.apache.drill.exec.dotdrill.DotDrillType;
    +import org.apache.drill.exec.physical.PhysicalPlan;
    +import org.apache.drill.exec.physical.base.PhysicalOperator;
    +import org.apache.drill.exec.planner.logical.DrillAnalyzeRel;
    +import org.apache.drill.exec.planner.logical.DrillRel;
    +import org.apache.drill.exec.planner.logical.DrillScreenRel;
    +import org.apache.drill.exec.planner.logical.DrillStoreRel;
    +import org.apache.drill.exec.planner.logical.DrillWriterRel;
    +import org.apache.drill.exec.planner.logical.DrillTable;
    +import org.apache.drill.exec.planner.physical.Prel;
    +import org.apache.drill.exec.planner.sql.DirectPlan;
    +import org.apache.drill.exec.planner.sql.SchemaUtilites;
    +import org.apache.drill.exec.planner.sql.parser.SqlAnalyzeTable;
    +import org.apache.drill.exec.store.AbstractSchema;
    +import org.apache.drill.exec.store.dfs.DrillFileSystem;
    +import org.apache.drill.exec.store.dfs.FileSystemPlugin;
    +import org.apache.drill.exec.store.dfs.FormatSelection;
    +import org.apache.drill.exec.store.dfs.NamedFormatPluginConfig;
    +import org.apache.drill.exec.store.parquet.ParquetFormatConfig;
    +import org.apache.drill.exec.util.Pointer;
    +import org.apache.drill.exec.work.foreman.ForemanSetupException;
    +import org.apache.drill.exec.work.foreman.SqlUnsupportedException;
    +import org.apache.hadoop.fs.Path;
    +import org.apache.hadoop.fs.FileStatus;
    +
    +public class AnalyzeTableHandler extends DefaultSqlHandler {
    +  private static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(AnalyzeTableHandler.class);
    +
    +  public AnalyzeTableHandler(SqlHandlerConfig config, Pointer<String> textPlan) {
    +    super(config, textPlan);
    +  }
    +
    +  @Override
    +  public PhysicalPlan getPlan(SqlNode sqlNode)
    +      throws ValidationException, RelConversionException, IOException, ForemanSetupException {
    +    final SqlAnalyzeTable sqlAnalyzeTable = unwrap(sqlNode, SqlAnalyzeTable.class);
    +
    +    verifyNoUnsupportedFunctions(sqlAnalyzeTable);
    +
    +    SqlIdentifier tableIdentifier = sqlAnalyzeTable.getTableIdentifier();
    +    SqlSelect scanSql = new SqlSelect(
    +        SqlParserPos.ZERO,              /* position */
    +        SqlNodeList.EMPTY,              /* keyword list */
    +        getColumnList(sqlAnalyzeTable), /* select list */
    +        tableIdentifier,                /* from */
    +        null,                           /* where */
    +        null,                           /* group by */
    +        null,                           /* having */
    +        null,                           /* windowDecls */
    +        null,                           /* orderBy */
    +        null,                           /* offset */
    +        null                            /* fetch */
    +    );
    +
    +    final ConvertedRelNode convertedRelNode = validateAndConvert(rewrite(scanSql));
    +    final RelNode relScan = convertedRelNode.getConvertedNode();
    +    final String tableName = sqlAnalyzeTable.getName();
    +    final AbstractSchema drillSchema = SchemaUtilites.resolveToDrillSchema(
    +        config.getConverter().getDefaultSchema(), sqlAnalyzeTable.getSchemaPath());
    +    Table table = SqlHandlerUtil.getTableFromSchema(drillSchema, tableName);
    +
    +    if (table == null) {
    +      throw UserException.validationError()
    +          .message("No table with given name [%s] exists in schema [%s]", tableName,
    +              drillSchema.getFullSchemaName())
    +          .build(logger);
    +    }
    +
    +    if(! (table instanceof DrillTable)) {
    +      return notSupported(tableName);
    +    }
    +
    +    if (table instanceof DrillTable) {
    +      DrillTable drillTable = (DrillTable) table;
    +      final Object selection = drillTable.getSelection();
    +      if (!(selection instanceof FormatSelection)) {
    +        return notSupported(tableName);
    +      }
    +      // Do not support non-parquet tables
    +      FormatSelection formatSelection = (FormatSelection) selection;
    +      FormatPluginConfig formatConfig = formatSelection.getFormat();
    +      if (!((formatConfig instanceof ParquetFormatConfig)
    --- End diff --
    
    For the ANALYZE query we perform this check before building the physical plan. We do not have a GroupScan at the 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 #729: Drill 1328: Support table statistics for Parquet

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

    https://github.com/apache/drill/pull/729#discussion_r102327621
  
    --- Diff: exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/statistics/StatisticsAggBatch.java ---
    @@ -0,0 +1,256 @@
    +/**
    + * 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.impl.statistics;
    +
    +import com.google.common.collect.Lists;
    +import com.sun.codemodel.JExpr;
    +import org.apache.drill.common.expression.ErrorCollector;
    +import org.apache.drill.common.expression.ErrorCollectorImpl;
    +import org.apache.drill.common.expression.FunctionCallFactory;
    +import org.apache.drill.common.expression.LogicalExpression;
    +import org.apache.drill.common.expression.SchemaPath;
    +import org.apache.drill.common.expression.ValueExpressions;
    +import org.apache.drill.exec.exception.ClassTransformationException;
    +import org.apache.drill.exec.exception.OutOfMemoryException;
    +import org.apache.drill.exec.exception.SchemaChangeException;
    +import org.apache.drill.exec.expr.ClassGenerator;
    +import org.apache.drill.exec.expr.CodeGenerator;
    +import org.apache.drill.exec.expr.ExpressionTreeMaterializer;
    +import org.apache.drill.exec.expr.TypeHelper;
    +import org.apache.drill.exec.expr.ValueVectorWriteExpression;
    +import org.apache.drill.exec.ops.FragmentContext;
    +import org.apache.drill.exec.physical.config.StatisticsAggregate;
    +import org.apache.drill.exec.physical.impl.aggregate.StreamingAggBatch;
    +import org.apache.drill.exec.physical.impl.aggregate.StreamingAggTemplate;
    +import org.apache.drill.exec.physical.impl.aggregate.StreamingAggregator;
    +import org.apache.drill.exec.planner.physical.StatsAggPrel.OperatorPhase;
    +import org.apache.drill.exec.record.BatchSchema.SelectionVectorMode;
    +import org.apache.drill.exec.record.MaterializedField;
    +import org.apache.drill.exec.record.RecordBatch;
    +import org.apache.drill.exec.record.TypedFieldId;
    +import org.apache.drill.exec.store.ImplicitColumnExplorer;
    +import org.apache.drill.exec.vector.ValueVector;
    +import org.apache.drill.exec.vector.complex.FieldIdUtil;
    +import org.apache.drill.exec.vector.complex.MapVector;
    +
    +import java.io.IOException;
    +import java.util.GregorianCalendar;
    +import java.util.List;
    +import java.util.TimeZone;
    +
    +/**
    + * TODO: This needs cleanup. Currently the key values are constants and we compare the constants
    + * for every record. Seems unnecessary.
    + *
    + * Example input and output:
    + * Schema of incoming batch: region_id (VARCHAR), sales_city (VARCHAR), cnt (BIGINT)
    + * Schema of output:
    + *    "schema" : BIGINT - Schema number. For each schema change this number is incremented.
    + *    "computed" : BIGINT - What time is it computed?
    + *    "columns"       : MAP - Column names
    + *       "region_id"  : VARCHAR
    + *       "sales_city" : VARCHAR
    + *       "cnt"        : VARCHAR
    + *    "statscount" : MAP
    + *       "region_id"  : BIGINT - statscount(region_id) - aggregation over all values of region_id
    + *                      in incoming batch
    + *       "sales_city" : BIGINT - statscount(sales_city)
    + *       "cnt"        : BIGINT - statscount(cnt)
    + *    "nonnullstatcount" : MAP
    + *       "region_id"  : BIGINT - nonnullstatcount(region_id)
    + *       "sales_city" : BIGINT - nonnullstatcount(sales_city)
    + *       "cnt"        : BIGINT - nonnullstatcount(cnt)
    + *   .... another map for next stats function ....
    + */
    +public class StatisticsAggBatch extends StreamingAggBatch {
    +  private List<String> functions;
    +  private int schema = 0;
    +
    +  public StatisticsAggBatch(StatisticsAggregate popConfig, RecordBatch incoming,
    +      FragmentContext context) throws OutOfMemoryException {
    +    super(popConfig, incoming, context);
    +    this.functions = popConfig.getFunctions();
    +  }
    +
    +  private void createKeyColumn(String name, LogicalExpression expr, List<LogicalExpression> keyExprs,
    +      List<TypedFieldId> keyOutputIds) throws SchemaChangeException {
    +    ErrorCollector collector = new ErrorCollectorImpl();
    +
    +    LogicalExpression mle = ExpressionTreeMaterializer.materialize(expr, incoming, collector,
    +        context.getFunctionRegistry());
    +
    +    MaterializedField outputField = MaterializedField.create(name, mle.getMajorType());
    +    ValueVector vector = TypeHelper.getNewVector(outputField, oContext.getAllocator());
    +
    +    keyExprs.add(mle);
    +    keyOutputIds.add(container.add(vector));
    +
    +    if (collector.hasErrors()) {
    +      throw new SchemaChangeException("Failure while materializing expression. "
    +          + collector.toErrorString());
    +    }
    +  }
    +
    +  private void createNestedKeyColumn(MapVector parent, String name, LogicalExpression expr,
    +      List<LogicalExpression> keyExprs, List<TypedFieldId> keyOutputIds)
    +          throws SchemaChangeException {
    +    ErrorCollector collector = new ErrorCollectorImpl();
    +
    +    LogicalExpression mle = ExpressionTreeMaterializer.materialize(expr, incoming, collector,
    +        context.getFunctionRegistry());
    +
    +    Class<? extends ValueVector> vvc =
    +        TypeHelper.getValueVectorClass(mle.getMajorType().getMinorType(),
    +            mle.getMajorType().getMode());
    +
    +    ValueVector vv = parent.addOrGet(name, mle.getMajorType(), vvc);
    +
    +    TypedFieldId pfid = container.getValueVectorId(SchemaPath.getSimplePath(parent.getField().getPath()));
    +    assert pfid.getFieldIds().length == 1;
    +    TypedFieldId.Builder builder = TypedFieldId.newBuilder();
    +    builder.addId(pfid.getFieldIds()[0]);
    +    TypedFieldId id =
    +        FieldIdUtil.getFieldIdIfMatches(parent, builder, true,
    +            SchemaPath.getSimplePath(vv.getField().getPath()).getRootSegment());
    +
    +    keyExprs.add(mle);
    +    keyOutputIds.add(id);
    +
    +    if (collector.hasErrors()) {
    +      throw new SchemaChangeException("Failure while materializing expression. "
    +          + collector.toErrorString());
    +    }
    +  }
    +
    +  private void addMapVector(String name, MapVector parent, LogicalExpression expr,
    +      List<LogicalExpression> valueExprs) throws SchemaChangeException {
    +    ErrorCollector collector = new ErrorCollectorImpl();
    +
    +    LogicalExpression mle = ExpressionTreeMaterializer.materialize(expr, incoming, collector,
    +        context.getFunctionRegistry());
    +
    +    Class<? extends ValueVector> vvc =
    +        TypeHelper.getValueVectorClass(mle.getMajorType().getMinorType(), mle.getMajorType().getMode());
    +    ValueVector vv = parent.addOrGet(name, mle.getMajorType(), vvc);
    +
    +    TypedFieldId pfid = container.getValueVectorId(SchemaPath.getSimplePath(parent.getField().getPath()));
    +    assert pfid.getFieldIds().length == 1;
    +    TypedFieldId.Builder builder = TypedFieldId.newBuilder();
    +    builder.addId(pfid.getFieldIds()[0]);
    +    TypedFieldId id = FieldIdUtil.getFieldIdIfMatches(parent, builder, true,
    +        SchemaPath.getSimplePath(vv.getField().getPath()).getRootSegment());
    +
    +    valueExprs.add(new ValueVectorWriteExpression(id, mle, true));
    +
    +    if (collector.hasErrors()) {
    +      throw new SchemaChangeException("Failure while materializing expression. "
    +          + collector.toErrorString());
    +    }
    +  }
    +
    +  private StreamingAggregator codegenAggregator(List<LogicalExpression> keyExprs,
    +      List<LogicalExpression> valueExprs, List<TypedFieldId> keyOutputIds)
    +          throws SchemaChangeException, ClassTransformationException, IOException {
    +    ClassGenerator<StreamingAggregator> cg =
    +        CodeGenerator.getRoot(StreamingAggTemplate.TEMPLATE_DEFINITION, context.getFunctionRegistry(),
    +            context.getOptions());
    +
    +    LogicalExpression[] keyExprsArray = new LogicalExpression[keyExprs.size()];
    +    LogicalExpression[] valueExprsArray = new LogicalExpression[valueExprs.size()];
    +    TypedFieldId[] keyOutputIdsArray = new TypedFieldId[keyOutputIds.size()];
    +
    +    keyExprs.toArray(keyExprsArray);
    +    valueExprs.toArray(valueExprsArray);
    +    keyOutputIds.toArray(keyOutputIdsArray);
    +
    +    setupIsSame(cg, keyExprsArray);
    +    setupIsSameApart(cg, keyExprsArray);
    +    addRecordValues(cg, valueExprsArray);
    +    outputRecordKeys(cg, keyOutputIdsArray, keyExprsArray);
    +    outputRecordKeysPrev(cg, keyOutputIdsArray, keyExprsArray);
    +
    +    cg.getBlock("resetValues")._return(JExpr.TRUE);
    +    getIndex(cg);
    +
    +    container.buildSchema(SelectionVectorMode.NONE);
    +    StreamingAggregator agg = context.getImplementationClass(cg);
    +    agg.setup(oContext, incoming, this);
    +    return agg;
    +  }
    +
    +  private boolean isImplicitFileColumn(MaterializedField mf) {
    +    return ImplicitColumnExplorer.initImplicitFileColumns(context.getOptions()).get(mf.getName()) != null;
    +  }
    +
    +  protected StreamingAggregator createAggregatorInternal()
    +      throws SchemaChangeException, ClassTransformationException, IOException {
    +    container.clear();
    +
    +    List<LogicalExpression> keyExprs = Lists.newArrayList();
    +    List<LogicalExpression> valueExprs = Lists.newArrayList();
    +    List<TypedFieldId> keyOutputIds = Lists.newArrayList();
    +    GregorianCalendar calendar = new GregorianCalendar(TimeZone.getTimeZone("UTC"));
    +    calendar.setTimeInMillis(System.currentTimeMillis());
    +
    +    if (this.getPopConfig() instanceof StatisticsAggregate
    +        && (((StatisticsAggregate) this.getPopConfig()).getPhase() == OperatorPhase.PHASE_1of1
    +            || ((StatisticsAggregate) this.getPopConfig()).getPhase() == OperatorPhase.PHASE_2of2)) {
    +      createKeyColumn("schema",
    +          ValueExpressions.getBigInt(schema++),
    +          keyExprs,
    +          keyOutputIds
    +      );
    +      createKeyColumn("computed",
    +          ValueExpressions.getDate(calendar),
    +          keyExprs,
    +          keyOutputIds
    +      );
    +    }
    +
    +    MapVector cparent = new MapVector("column", oContext.getAllocator(), null);
    +    container.add(cparent);
    +    for (MaterializedField mf : incoming.getSchema()) {
    +      // Ignore implicit columns
    +      if (!isImplicitFileColumn(mf)) {
    +        createNestedKeyColumn(
    +            cparent,
    +            mf.getLastName(),
    +            ValueExpressions.getChar(mf.getLastName()),
    +            keyExprs,
    +            keyOutputIds
    +        );
    +      }
    +    }
    +
    +    for (String func : functions) {
    +      MapVector parent = new MapVector(func, oContext.getAllocator(), null);
    +      container.add(parent);
    +
    +      for (MaterializedField mf : incoming.getSchema()) {
    +        if (!isImplicitFileColumn(mf)) {
    +          List<LogicalExpression> args = Lists.newArrayList();
    +          args.add(SchemaPath.getSimplePath(mf.getPath()));
    +          LogicalExpression call = FunctionCallFactory.createExpression(func, args);
    +          addMapVector(mf.getLastName(), parent, call, valueExprs);
    --- End diff --
    
    The current design does not handle nested maps. This limitation would be documented. We throw an error now saying that maps as input are not supported.


---
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 #729: Drill 1328: Support table statistics for Parquet

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

    https://github.com/apache/drill/pull/729#discussion_r102290224
  
    --- Diff: exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/unpivot/UnpivotMapsBatchCreator.java ---
    @@ -0,0 +1,40 @@
    +/**
    + * 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.physical.impl.unpivot;
    +
    +import java.util.List;
    +
    +import org.apache.drill.common.exceptions.ExecutionSetupException;
    +import org.apache.drill.exec.ops.FragmentContext;
    +import org.apache.drill.exec.physical.config.UnpivotMaps;
    +import org.apache.drill.exec.physical.impl.BatchCreator;
    +import org.apache.drill.exec.record.CloseableRecordBatch;
    +import org.apache.drill.exec.record.RecordBatch;
    +
    +import com.google.common.base.Preconditions;
    +
    +@SuppressWarnings("unused")
    --- End diff --
    
    Copy-paste artifact. Removed it.


---
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 #729: Drill 1328: Support table statistics for Parquet

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

    https://github.com/apache/drill/pull/729#discussion_r103796907
  
    --- Diff: exec/java-exec/src/main/java/org/apache/drill/exec/ExecConstants.java ---
    @@ -390,4 +391,15 @@
     
       String DYNAMIC_UDF_SUPPORT_ENABLED = "exec.udf.enable_dynamic_support";
       BooleanValidator DYNAMIC_UDF_SUPPORT_ENABLED_VALIDATOR = new BooleanValidator(DYNAMIC_UDF_SUPPORT_ENABLED, true, true);
    +
    +  /**
    +   * Option whose value is a long value representing the number of bits required for computing ndv (using HLL)
    +   */
    +  LongValidator NDV_MEMORY_LIMIT = new PositiveLongValidator("exec.statistics.ndv_memory_limit", 30, 20);
    +
    +  /**
    +   * Option whose value represents the current version of the statistics. Decreasing the value will generate
    +   * the older version of statistics
    +   */
    +  LongValidator STATISTICS_VERSION = new NonNegativeLongValidator("exec.statistics.capability_version", 1, 1);
    --- End diff --
    
    Agree with @paul-rogers on this. 


---
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 #729: Drill 1328 r4

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/729#discussion_r99393478
  
    --- Diff: exec/java-exec/src/main/java/org/apache/drill/exec/ops/QueryContext.java ---
    @@ -77,6 +78,7 @@
       private final SchemaTreeProvider schemaTreeProvider;
       /** Stores constants and their holders by type */
       private final Map<String, Map<MinorType, ValueHolder>> constantValueHolderCache;
    +  private StatementType stmtType = StatementType.UNKNOWN;
    --- End diff --
    
    Maybe initialize to null (the default) and require that the statement type be set later. Or, pass the statement type into the constructor and make the type a `final` variable showing that the statement type cannot change over 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 #729: Drill 1328: Support table statistics for Parquet

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/729#discussion_r102865583
  
    --- Diff: exec/java-exec/src/main/java/org/apache/drill/exec/ExecConstants.java ---
    @@ -390,4 +391,15 @@
     
       String DYNAMIC_UDF_SUPPORT_ENABLED = "exec.udf.enable_dynamic_support";
       BooleanValidator DYNAMIC_UDF_SUPPORT_ENABLED_VALIDATOR = new BooleanValidator(DYNAMIC_UDF_SUPPORT_ENABLED, true, true);
    +
    +  /**
    +   * Option whose value is a long value representing the number of bits required for computing ndv (using HLL)
    +   */
    +  LongValidator NDV_MEMORY_LIMIT = new PositiveLongValidator("exec.statistics.ndv_memory_limit", 30, 20);
    +
    +  /**
    +   * Option whose value represents the current version of the statistics. Decreasing the value will generate
    +   * the older version of statistics
    +   */
    +  LongValidator STATISTICS_VERSION = new NonNegativeLongValidator("exec.statistics.capability_version", 1, 1);
    --- End diff --
    
    Having a statistics version number makes sense. What I disagree on is how we are managing the version.
    
    The version is defined by the code that gathers and writes the stats. If I'm running a Drill that has version 3 of the implementation, I write version 3 files. That version number should be a constant defined in the code. When we change stats format, we bump the version number.
    
    The reader should handle old versions of the file: at least one older version (to ease software upgrades.) The reader retrieves the version from the file and checks if it is supported by the reader implementation.
    
    This is all very standard practice.
    
    Where, then, is there room for the user to specify a version? What does specifying a version mean? This is the question we need to clarify.


---
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 #729: Drill 1328: Support table statistics for Parquet

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

    https://github.com/apache/drill/pull/729#discussion_r102316247
  
    --- Diff: exec/java-exec/src/main/java/org/apache/drill/exec/physical/config/UnpivotMaps.java ---
    @@ -0,0 +1,59 @@
    +/**
    + * 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.config;
    +
    +import java.util.List;
    +
    +import org.apache.drill.exec.physical.base.AbstractSingle;
    +import org.apache.drill.exec.physical.base.PhysicalOperator;
    +import org.apache.drill.exec.physical.base.PhysicalVisitor;
    +import org.apache.drill.exec.proto.UserBitShared.CoreOperatorType;
    +
    +import com.fasterxml.jackson.annotation.JsonCreator;
    +import com.fasterxml.jackson.annotation.JsonProperty;
    +import com.fasterxml.jackson.annotation.JsonTypeName;
    +
    +@JsonTypeName("unpivot-maps")
    +public class UnpivotMaps extends AbstractSingle {
    --- End diff --
    
    This is generic in the sense that it can unpivot any given set of maps given that they are of
    the same size and contain the same keys(columns).


---
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 #729: Drill 1328: Support table statistics for Parquet

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

    https://github.com/apache/drill/pull/729#discussion_r103383633
  
    --- Diff: exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/statistics/AvgWidthMergedStatistic.java ---
    @@ -0,0 +1,135 @@
    +/*
    + * 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.impl.statistics;
    +
    +import org.apache.drill.common.types.TypeProtos;
    +import org.apache.drill.exec.expr.holders.NullableFloat8Holder;
    +import org.apache.drill.exec.expr.holders.ValueHolder;
    +import org.apache.drill.exec.vector.NullableFloat8Vector;
    +import org.apache.drill.exec.vector.ValueVector;
    +import org.apache.drill.exec.vector.complex.MapVector;
    +
    +import java.util.HashMap;
    +import java.util.List;
    +import java.util.Map;
    +
    +public class AvgWidthMergedStatistic extends AbstractMergedStatistic {
    +
    +  private String name;
    +  private String inputName;
    +  private boolean configureComplete = false;
    +  private boolean mergeComplete = false;
    --- End diff --
    
    Yes, not necessary. However, having them move through different states would make it easier to debug/track issues. Also, incremental statistics would probably increase the run-time states interactions. As suggested added a state enum `enum State {Config, Merge, Complete};`


---
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 #729: Drill 1328: Support table statistics for Parquet

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/729#discussion_r102874299
  
    --- Diff: exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/statistics/MergedStatisticFactory.java ---
    @@ -0,0 +1,46 @@
    +/*
    + * 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.impl.statistics;
    +
    +public class MergedStatisticFactory {
    +  /*
    +   * Creates the appropriate statistics object given the name of the statistics and the input statistic
    +   */
    +  public static MergedStatistic getMergedStatistic(String outputStatName, String inputStatName) {
    +    if (outputStatName == null || inputStatName == null) {
    +      return null;
    --- End diff --
    
    This is probably an error:
    ```
    throw new IllegalArgumentException("Names cannot 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 #729: Drill 1328: Support table statistics for Parquet

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/729#discussion_r102872366
  
    --- Diff: exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/statistics/AbstractMergedStatistic.java ---
    @@ -0,0 +1,52 @@
    +/*
    + * 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.impl.statistics;
    +
    +import org.apache.drill.exec.vector.ValueVector;
    +
    +public abstract class AbstractMergedStatistic extends Statistic implements MergedStatistic {
    +  @Override
    +  public String getName() {
    +    throw new UnsupportedOperationException("getName() not implemented");
    +  }
    +
    +  @Override
    +  public String getInput() {
    +    throw new UnsupportedOperationException("getInput() not implemented");
    +  }
    +
    +  @Override
    +  public void merge(ValueVector input) {
    +    throw new UnsupportedOperationException("merge() not implemented");
    +  }
    +
    +  @Override
    +  public Object getStat(String colName) {
    +    throw new UnsupportedOperationException("getStat() not implemented");
    +  }
    +
    +  @Override
    +  public void setOutput(ValueVector output) {
    +    throw new UnsupportedOperationException("getOutput() not implemented");
    +  }
    +
    +  @Override
    +  public void configure(Object configurations) {
    --- End diff --
    
    See comments below. Types should not be so generic.
    
    Also, given the number of hash lookups, I wonder if we need a class to hold the information about each column?


---
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 #729: Drill 1328: Support table statistics for Parquet

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

    https://github.com/apache/drill/pull/729#discussion_r100866510
  
    --- Diff: exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/statistics/StatisticsMergeBatch.java ---
    @@ -0,0 +1,532 @@
    +/**
    + * 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.impl.statistics;
    +
    +import com.google.common.collect.Lists;
    +import com.google.common.collect.Maps;
    +import org.apache.drill.common.expression.LogicalExpression;
    +import org.apache.drill.common.expression.ErrorCollector;
    +import org.apache.drill.common.expression.ErrorCollectorImpl;
    +import org.apache.drill.common.expression.FunctionCallFactory;
    +import org.apache.drill.common.expression.SchemaPath;
    +import org.apache.drill.common.expression.ValueExpressions;
    +import org.apache.drill.common.types.TypeProtos;
    +import org.apache.drill.common.types.TypeProtos.MajorType;
    +import org.apache.drill.exec.exception.OutOfMemoryException;
    +import org.apache.drill.exec.exception.SchemaChangeException;
    +import org.apache.drill.exec.expr.ExpressionTreeMaterializer;
    +import org.apache.drill.exec.expr.TypeHelper;
    +import org.apache.drill.exec.expr.holders.BigIntHolder;
    +import org.apache.drill.exec.expr.holders.NullableFloat8Holder;
    +import org.apache.drill.exec.expr.holders.ObjectHolder;
    +import org.apache.drill.exec.expr.holders.ValueHolder;
    +import org.apache.drill.exec.ops.FragmentContext;
    +import org.apache.drill.exec.physical.config.StatisticsMerge;
    +import org.apache.drill.exec.record.AbstractSingleRecordBatch;
    +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.VectorContainer;
    +import org.apache.drill.exec.record.VectorWrapper;
    +import org.apache.drill.exec.vector.ValueVector;
    +import org.apache.drill.exec.vector.DateVector;
    +import org.apache.drill.exec.vector.BigIntVector;
    +import org.apache.drill.exec.vector.NullableBigIntVector;
    +import org.apache.drill.exec.vector.NullableFloat8Vector;
    +import org.apache.drill.exec.vector.NullableVarBinaryVector;
    +import org.apache.drill.exec.vector.VarCharVector;
    +import org.apache.drill.exec.vector.complex.MapVector;
    +
    +import java.io.ByteArrayInputStream;
    +import java.io.DataInputStream;
    +import java.io.IOException;
    +import java.util.List;
    +import java.util.GregorianCalendar;
    +import java.util.HashMap;
    +import java.util.Map;
    +import java.util.TimeZone;
    +
    +import com.clearspring.analytics.stream.cardinality.HyperLogLog;
    +
    +public class StatisticsMergeBatch extends AbstractSingleRecordBatch<StatisticsMerge> {
    +  private static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(StatisticsMergeBatch.class);
    +  private Map<String, String> functions;
    +  private boolean first = true;
    +  private boolean finished = false;
    +  private int schema = 0;
    +  private int recordCount = 0;
    +  private List<String> keyList = null;
    +  private Map<MaterializedField, ValueVector> dataSrcVecMap = null;
    +  // Map of non-map fields to VV in the incoming schema
    +  private Map<MaterializedField, ValueVector> copySrcVecMap = null;
    +  private Map<String, Map<String, ValueHolder>> aggregationMap = null;
    +  public StatisticsMergeBatch(StatisticsMerge popConfig, RecordBatch incoming,
    +                              FragmentContext context) throws OutOfMemoryException {
    +    super(popConfig, context, incoming);
    +    this.functions = new HashMap<>();
    +    this.aggregationMap = new HashMap<>();
    +
    +    /*for (String key : popConfig.getFunctions()) {
    --- 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 #729: Drill 1328: Support table statistics for Parquet

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/729#discussion_r102874162
  
    --- Diff: exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/statistics/MergedStatistic.java ---
    @@ -0,0 +1,41 @@
    +/*
    + * 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.impl.statistics;
    +
    +import org.apache.drill.exec.vector.ValueVector;
    +/*
    + * Interface for implementing a merged statistic. A merged statistic can merge
    + * the input statistics to get the overall value. e.g. `rowcount` merged statistic
    + * should merge all `rowcount` input statistic and return the overall `rowcount`.
    + * Given `rowcount`s 10 and 20, the `rowcount` merged statistic will return 30.
    + */
    +public interface MergedStatistic {
    +  // Gets the name of the merged statistic
    --- End diff --
    
    If you use Javadoc comments, then the description will show up in any generated documentation:
    ```
    /** Gets the name ... */
    ```


---
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 #729: Drill 1328 r4

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/729#discussion_r99390012
  
    --- Diff: exec/java-exec/pom.xml ---
    @@ -410,6 +410,11 @@
           <artifactId>joda-time</artifactId>
           <version>2.9</version>
         </dependency>
    +    <dependency>
    +      <groupId>com.clearspring.analytics</groupId>
    +      <artifactId>stream</artifactId>
    +      <version>2.7.0</version>
    +    </dependency>
    --- End diff --
    
    This library must be excluded from the Drill JDBC pom. (Yes, a real nuisance...)


---
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 #729: Drill 1328: Support table statistics for Parquet

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

    https://github.com/apache/drill/pull/729#discussion_r102322555
  
    --- Diff: exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/statistics/StatisticsMergeBatch.java ---
    @@ -0,0 +1,532 @@
    +/**
    + * 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.impl.statistics;
    +
    +import com.google.common.collect.Lists;
    +import com.google.common.collect.Maps;
    +import org.apache.drill.common.expression.LogicalExpression;
    +import org.apache.drill.common.expression.ErrorCollector;
    +import org.apache.drill.common.expression.ErrorCollectorImpl;
    +import org.apache.drill.common.expression.FunctionCallFactory;
    +import org.apache.drill.common.expression.SchemaPath;
    +import org.apache.drill.common.expression.ValueExpressions;
    +import org.apache.drill.common.types.TypeProtos;
    +import org.apache.drill.common.types.TypeProtos.MajorType;
    +import org.apache.drill.exec.exception.OutOfMemoryException;
    +import org.apache.drill.exec.exception.SchemaChangeException;
    +import org.apache.drill.exec.expr.ExpressionTreeMaterializer;
    +import org.apache.drill.exec.expr.TypeHelper;
    +import org.apache.drill.exec.expr.holders.BigIntHolder;
    +import org.apache.drill.exec.expr.holders.NullableFloat8Holder;
    +import org.apache.drill.exec.expr.holders.ObjectHolder;
    +import org.apache.drill.exec.expr.holders.ValueHolder;
    +import org.apache.drill.exec.ops.FragmentContext;
    +import org.apache.drill.exec.physical.config.StatisticsMerge;
    +import org.apache.drill.exec.record.AbstractSingleRecordBatch;
    +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.VectorContainer;
    +import org.apache.drill.exec.record.VectorWrapper;
    +import org.apache.drill.exec.vector.ValueVector;
    +import org.apache.drill.exec.vector.DateVector;
    +import org.apache.drill.exec.vector.BigIntVector;
    +import org.apache.drill.exec.vector.NullableBigIntVector;
    +import org.apache.drill.exec.vector.NullableFloat8Vector;
    +import org.apache.drill.exec.vector.NullableVarBinaryVector;
    +import org.apache.drill.exec.vector.VarCharVector;
    +import org.apache.drill.exec.vector.complex.MapVector;
    +
    +import java.io.ByteArrayInputStream;
    +import java.io.DataInputStream;
    +import java.io.IOException;
    +import java.util.List;
    +import java.util.GregorianCalendar;
    +import java.util.HashMap;
    +import java.util.Map;
    +import java.util.TimeZone;
    +
    +import com.clearspring.analytics.stream.cardinality.HyperLogLog;
    +
    +public class StatisticsMergeBatch extends AbstractSingleRecordBatch<StatisticsMerge> {
    +  private static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(StatisticsMergeBatch.class);
    +  private Map<String, String> functions;
    +  private boolean first = true;
    +  private boolean finished = false;
    +  private int schema = 0;
    +  private int recordCount = 0;
    +  private List<String> keyList = null;
    +  private Map<MaterializedField, ValueVector> dataSrcVecMap = null;
    +  // Map of non-map fields to VV in the incoming schema
    +  private Map<MaterializedField, ValueVector> copySrcVecMap = null;
    +  private Map<String, Map<String, ValueHolder>> aggregationMap = null;
    +  public StatisticsMergeBatch(StatisticsMerge popConfig, RecordBatch incoming,
    +                              FragmentContext context) throws OutOfMemoryException {
    +    super(popConfig, context, incoming);
    +    this.functions = new HashMap<>();
    +    this.aggregationMap = new HashMap<>();
    +
    +    /*for (String key : popConfig.getFunctions()) {
    +      aggregationMap.put(key, new HashMap<String, ValueHolder>());
    +      if (key.equalsIgnoreCase("statcount") || key.equalsIgnoreCase("nonnullstatcount")) {
    +        functions.put(key, "sum");
    +      } else if (key.equalsIgnoreCase("hll")) {
    +        functions.put(key, "hll_merge");
    +      } else if (key.equalsIgnoreCase("sum_width")) {
    +        functions.put(key, "avg_width");
    +      }
    +    }*/
    +    for (String key : popConfig.getFunctions()) {
    +      if (key.equalsIgnoreCase("sum_width")) {
    +        functions.put(key, "avg_width");
    +      } else if (key.equalsIgnoreCase("hll")) {
    +        functions.put(key, "hll_merge");
    +      } else {
    +        functions.put(key, key);
    +      }
    +      aggregationMap.put(functions.get(key), new HashMap<String, ValueHolder>());
    +    }
    +  }
    +
    +  private void createKeyColumn(String name, LogicalExpression expr, Map<MaterializedField, ValueVector> parentMap)
    +      throws SchemaChangeException {
    +    ErrorCollector collector = new ErrorCollectorImpl();
    +
    +    LogicalExpression mle = ExpressionTreeMaterializer.materialize(expr, incoming, collector,
    +        context.getFunctionRegistry());
    +
    +    MaterializedField outputField = MaterializedField.create(name, mle.getMajorType());
    +    ValueVector vector = TypeHelper.getNewVector(outputField, oContext.getAllocator());
    +    container.add(vector);
    +
    +    if (collector.hasErrors()) {
    +      throw new SchemaChangeException("Failure while materializing expression. "
    +          + collector.toErrorString());
    +    }
    +    parentMap.put(outputField, vector);
    +  }
    +
    +  private ValueVector addMapVector(String name, MapVector parent, LogicalExpression expr) throws SchemaChangeException {
    +    ErrorCollector collector = new ErrorCollectorImpl();
    +
    +    LogicalExpression mle = ExpressionTreeMaterializer.materialize(expr, incoming, collector,
    +        context.getFunctionRegistry());
    +
    +    Class<? extends ValueVector> vvc =
    +        TypeHelper.getValueVectorClass(mle.getMajorType().getMinorType(), mle.getMajorType().getMode());
    +    ValueVector vector = parent.addOrGet(name, mle.getMajorType(), vvc);
    +
    +    if (collector.hasErrors()) {
    +      throw new SchemaChangeException("Failure while materializing expression. "
    +          + collector.toErrorString());
    +    }
    +    return vector;
    +  }
    +
    +  /**
    +   * Identify the list of fields within a map which are unpivoted as columns in output
    +   */
    +  private void buildKeyList() {
    +    List<String> lastMapKeyList = null;
    +    for (VectorWrapper<?> vw : incoming) {
    +      if (vw.getField().getType().getMinorType() != TypeProtos.MinorType.MAP) {
    +        continue;
    +      }
    +
    +      keyList = Lists.newArrayList();
    +
    +      for (ValueVector vv : vw.getValueVector()) {
    +        keyList.add(vv.getField().getLastName());
    +      }
    +
    +      if (lastMapKeyList == null) {
    +        lastMapKeyList = keyList;
    +      } else {
    +        if (keyList.size() != lastMapKeyList.size() || !lastMapKeyList.containsAll(keyList)) {
    +          throw new UnsupportedOperationException("Maps have different fields");
    +        }
    +      }
    +    }
    +  }
    +
    +  private void buildOutputContainer() throws SchemaChangeException {
    +    dataSrcVecMap = Maps.newHashMap();
    +    copySrcVecMap = Maps.newHashMap();
    +    MajorType mt = null;
    +
    +    ErrorCollector collector = new ErrorCollectorImpl();
    +    GregorianCalendar calendar = new GregorianCalendar(TimeZone.getTimeZone("UTC"));
    +
    +    calendar.setTimeInMillis(System.currentTimeMillis());
    +    createKeyColumn("schema", ValueExpressions.getBigInt(schema++), copySrcVecMap);
    +    createKeyColumn("computed", ValueExpressions.getDate(calendar), copySrcVecMap);
    +
    +    for (VectorWrapper<?> vw : incoming) {
    +      addVectorToOutgoingContainer(vw.getField().getLastName(), vw, collector);
    +      /*MaterializedField ds = vw.getField();
    +      String field = vw.getField().getLastName();
    +      // Input map vector
    +      MapVector mapVector = (MapVector) vw.getValueVector();
    +      assert mapVector.getPrimitiveVectors().size() > 0;
    +      // Proceed to create output map vector with same name e.g. statcount etc.
    +      mt = mapVector.getField().getType();
    +      MaterializedField mf = MaterializedField.create(functions.get(field), mt);
    +      assert !dataSrcVecMap.containsKey(mf);
    +      ValueVector vector = TypeHelper.getNewVector(mf, oContext.getAllocator());
    +      container.add(vector);
    +      MapVector outputMapVector = (MapVector) vector;
    +
    +      for (ValueVector vv : mapVector) {
    +        String fieldName = vv.getField().getLastName();
    +        if (!keyList.contains(fieldName)) {
    +          throw new UnsupportedOperationException("Unpivot data vector " +
    +              ds + " contains key " + fieldName + " not contained in key source!");
    +        }
    +        if (vv.getField().getType().getMinorType() == TypeProtos.MinorType.MAP) {
    +          throw new UnsupportedOperationException("Unpivot of nested map is not supported!");
    +        }
    +        if (field.equals("column")) {
    +          outputMapVector.addOrGet(fieldName, vv.getField().getType(), vv.getClass());
    +        } else {
    +          List<LogicalExpression> args = Lists.newArrayList();
    +          //TODO: Something else to access value of col such as emp_id?
    +          args.add(SchemaPath.getSimplePath(vv.getField().getPath()));
    +          //TODO: Put in the mapVector
    +          LogicalExpression call = FunctionCallFactory.createExpression(functions.get(field), args);
    +          //TODO: Is this sufficient to add to new Map?
    +          ValueVector vector1 = addMapVector(fieldName, outputMapVector, call);
    +          if (collector.hasErrors()) {
    +            throw new SchemaChangeException("Failure while materializing expression. "
    +                + collector.toErrorString());
    +          }
    +        }
    +      }
    +      dataSrcVecMap.put(ds, outputMapVector);*/
    +    }
    +    //Now create NDV in the outgoing container which was not avaliable in the incoming
    +    for (VectorWrapper<?> vw : incoming) {
    +      if (vw.getField().getLastName().equalsIgnoreCase("sum_width")) {//NullableFloat8 type
    +        addVectorToOutgoingContainer("ndv", vw, collector);
    +        break;
    +      }
    +    }
    +    container.setRecordCount(0);
    +    recordCount = 0;
    +    container.buildSchema(incoming.getSchema().getSelectionVectorMode());
    +  }
    +
    +  private void addVectorToOutgoingContainer(String field, VectorWrapper vw, ErrorCollector collector)
    +     throws SchemaChangeException {
    +    // Input map vector
    +    MapVector mapVector = (MapVector) vw.getValueVector();
    +    MaterializedField mf;
    +    assert mapVector.getPrimitiveVectors().size() > 0;
    +    // Proceed to create output map vector with same name e.g. statcount etc.
    +    MajorType mt = mapVector.getField().getType();
    +    if (functions.get(field) != null) {
    +      mf = MaterializedField.create(functions.get(field), mt);
    +    } else {
    +      mf = MaterializedField.create(field, mt);
    +    }
    +    assert !dataSrcVecMap.containsKey(mf);
    +    ValueVector vector = TypeHelper.getNewVector(mf, oContext.getAllocator());
    +    container.add(vector);
    +    MapVector outputMapVector = (MapVector) vector;
    +
    +    for (ValueVector vv : mapVector) {
    +      String fieldName = vv.getField().getLastName();
    +      if (!keyList.contains(fieldName)) {
    +        throw new UnsupportedOperationException("Unpivot data vector " +
    +                field + " contains key " + fieldName + " not contained in key source!");
    +      }
    +      if (vv.getField().getType().getMinorType() == TypeProtos.MinorType.MAP) {
    +        throw new UnsupportedOperationException("Unpivot of nested map is not supported!");
    +      }
    +      if (field.equals("column")) {
    +        outputMapVector.addOrGet(fieldName, vv.getField().getType(), vv.getClass());
    +      } else {
    +        List<LogicalExpression> args = Lists.newArrayList();
    +        LogicalExpression call;
    +        //TODO: Something else to access value of col such as emp_id?
    +        args.add(SchemaPath.getSimplePath(vv.getField().getPath()));
    +        //TODO: Put in the mapVector
    +        if (functions.get(field) != null) {
    +          call = FunctionCallFactory.createExpression(functions.get(field), args);
    +        } else {
    +          call = FunctionCallFactory.createExpression(field, args);
    +        }
    +        //TODO: Is this sufficient to add to new Map?
    +        ValueVector vector1 = addMapVector(fieldName, outputMapVector, call);
    +        if (collector.hasErrors()) {
    +          throw new SchemaChangeException("Failure while materializing expression. "
    +                  + collector.toErrorString());
    +        }
    +      }
    +    }
    +  }
    +
    +  @Override
    +  protected boolean setupNewSchema() throws SchemaChangeException {
    +    container.clear();
    +    buildKeyList();
    +    buildOutputContainer();
    +    return true;
    +  }
    +
    +  @Override
    +  protected IterOutcome doWork() {
    +    int outRecordCount = incoming.getRecordCount();
    +    HashMap<String, Long> nonNullRows = new HashMap<>();
    +
    +    for (VectorWrapper<?> vw : incoming) {
    +      ValueVector vv = vw.getValueVector();
    +      if (vv.getField().getType().getMinorType() != TypeProtos.MinorType.MAP) {
    +        //We only expect Maps in the incoming. STOP, if this is not the case.
    +        return IterOutcome.STOP;
    +      }
    +      MapVector mapVec = (MapVector) vv;
    +      Map<String, ValueHolder> statMap = aggregationMap.get(vv.getField().getLastName());
    +
    +      for (ValueVector mapElt : mapVec) {
    +        if (vv.getField().getLastName().equalsIgnoreCase("column")) {
    +          BigIntHolder nameHolder;
    +          if (statMap.get(mapElt.getField().getLastName()) != null) {
    +            nameHolder = (BigIntHolder) statMap.get(mapElt.getField().getLastName());
    +          } else {
    +            nameHolder = new BigIntHolder();
    +            statMap.put(mapElt.getField().getLastName(), nameHolder);
    +          }
    +          nameHolder.value = 1;
    +        } else if (vv.getField().getLastName().equalsIgnoreCase("statcount")
    +            || vv.getField().getLastName().equalsIgnoreCase("nonnullstatcount")) {
    +          BigIntHolder sumHolder;
    +          String colName = mapElt.getField().getLastName();
    +          if (statMap.get(colName) != null) {
    +            sumHolder = (BigIntHolder) statMap.get(colName);
    +          } else {
    +            sumHolder = new BigIntHolder();
    +            statMap.put(colName, sumHolder);
    +          }
    +          //TODO: assert size = 1
    +          //TODO: logger
    +          Object val = mapElt.getAccessor().getObject(0);
    +          if (val != null) {
    +            sumHolder.value += (long)val;
    +          }
    +        } else if (vv.getField().getLastName().equalsIgnoreCase("sum_width")) {
    +          NullableFloat8Holder sumHolder;
    +          String colName = mapElt.getField().getLastName();
    +          if (statMap == null) {
    +            statMap = aggregationMap.get(functions.get(vv.getField().getLastName()));
    +          }
    +          if (statMap.get(colName) != null) {
    +            sumHolder = (NullableFloat8Holder) statMap.get(colName);
    +          } else {
    +            sumHolder = new NullableFloat8Holder();
    +            statMap.put(colName, sumHolder);
    +          }
    +          //TODO: assert size = 1
    +          //TODO: logger
    +          Object val = mapElt.getAccessor().getObject(0);
    +          if (val != null) {
    +            sumHolder.value += (double) val;
    +            sumHolder.isSet = 1;
    +          }
    +        } else if (vv.getField().getLastName().equalsIgnoreCase("hll")) {
    +          ObjectHolder hllHolder;
    +          String colName = mapElt.getField().getLastName();
    +          if (statMap == null) {
    +            statMap = aggregationMap.get(functions.get(vv.getField().getLastName()));
    +          }
    +          if (statMap.get(colName) != null) {
    +            hllHolder = (ObjectHolder) statMap.get(colName);
    +          } else {
    +            hllHolder = new ObjectHolder();
    +            hllHolder.obj = new HyperLogLog(context.getContextInformation().getHllAccuracy());
    +            statMap.put(colName, hllHolder);
    +          }
    +          NullableVarBinaryVector hllVector = (NullableVarBinaryVector) mapElt;
    +          try {
    +            if (hllVector.getAccessor().isSet(0) == 1) {
    +              ByteArrayInputStream bais = new ByteArrayInputStream(hllVector.getAccessor().getObject(0), 0,
    +                  mapElt.getBufferSize());
    +              HyperLogLog other = HyperLogLog.Builder.build(new DataInputStream(bais));
    +              ((HyperLogLog) hllHolder.obj).addAll(other);
    +            }
    +          } catch (Exception ex) {
    +            //TODO: Catch IOException/CardinalityMergeException
    +            //TODO: logger
    +            return IterOutcome.STOP;
    +          }
    +        }
    +      }
    +      // Add NDV value vector map using HLL map (since the NDV map is directly generated from HLL and not produced by the underlying
    +      // Statistics Agg)
    +      Map<String, ValueHolder> hllMap = aggregationMap.get("hll");
    +      if (hllMap != null) {
    +        aggregationMap.put("ndv", hllMap);
    +      }
    +    }
    +    return IterOutcome.OK;
    +  }
    +
    +  public VectorContainer getOutgoingContainer() {
    +    return this.container;
    +  }
    +
    +  @Override
    +  public IterOutcome innerNext() {
    +    IterOutcome outcome;
    +    boolean didSomeWork = false;
    +    if (finished) {
    +      return IterOutcome.NONE;
    +    }
    +    try {
    +      outer: while (true) {
    +        outcome = next(incoming);
    +        switch (outcome) {
    +          case NONE:
    +            break outer;
    +          case OUT_OF_MEMORY:
    +          case NOT_YET:
    +          case STOP:
    +            return outcome;
    +          case OK_NEW_SCHEMA:
    +            if (first) {
    +              first =false;
    +              if (!setupNewSchema()) {
    +                outcome = IterOutcome.OK;
    +              }
    +              return outcome;
    +            }
    +            //fall through
    +          case OK:
    +            assert first == false : "First batch should be OK_NEW_SCHEMA";
    +            IterOutcome out = doWork();
    +            didSomeWork = true;
    +            if (out != IterOutcome.OK) {
    +              return out;
    +            }
    +            break;
    +          default:
    +            throw new UnsupportedOperationException("Unsupported upstream state " + outcome);
    +        }
    +      }
    +    } catch (SchemaChangeException ex) {
    +      kill(false);
    +      logger.error("Failure during query", ex);
    +      context.fail(ex);
    +      return IterOutcome.STOP;
    +    }
    +
    +    // We can only get here if upstream is NONE i.e. no more batches. If we did some work prior to
    +    // exhausting all upstream, then return OK. Otherwise, return NONE.
    +    if (didSomeWork) {
    +      IterOutcome out = buildOutgoingRecordBatch();
    +      finished = true;
    +      return out;
    +    } else {
    +      return outcome;
    +    }
    +  }
    +
    +  // Prepare the outgoing container
    +  private IterOutcome buildOutgoingRecordBatch() {
    --- 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 #729: Drill 1328: Support table statistics for Parquet

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/729#discussion_r100681072
  
    --- Diff: exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/statistics/StatisticsMergeBatch.java ---
    @@ -0,0 +1,532 @@
    +/**
    + * 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.impl.statistics;
    +
    +import com.google.common.collect.Lists;
    +import com.google.common.collect.Maps;
    +import org.apache.drill.common.expression.LogicalExpression;
    +import org.apache.drill.common.expression.ErrorCollector;
    +import org.apache.drill.common.expression.ErrorCollectorImpl;
    +import org.apache.drill.common.expression.FunctionCallFactory;
    +import org.apache.drill.common.expression.SchemaPath;
    +import org.apache.drill.common.expression.ValueExpressions;
    +import org.apache.drill.common.types.TypeProtos;
    +import org.apache.drill.common.types.TypeProtos.MajorType;
    +import org.apache.drill.exec.exception.OutOfMemoryException;
    +import org.apache.drill.exec.exception.SchemaChangeException;
    +import org.apache.drill.exec.expr.ExpressionTreeMaterializer;
    +import org.apache.drill.exec.expr.TypeHelper;
    +import org.apache.drill.exec.expr.holders.BigIntHolder;
    +import org.apache.drill.exec.expr.holders.NullableFloat8Holder;
    +import org.apache.drill.exec.expr.holders.ObjectHolder;
    +import org.apache.drill.exec.expr.holders.ValueHolder;
    +import org.apache.drill.exec.ops.FragmentContext;
    +import org.apache.drill.exec.physical.config.StatisticsMerge;
    +import org.apache.drill.exec.record.AbstractSingleRecordBatch;
    +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.VectorContainer;
    +import org.apache.drill.exec.record.VectorWrapper;
    +import org.apache.drill.exec.vector.ValueVector;
    +import org.apache.drill.exec.vector.DateVector;
    +import org.apache.drill.exec.vector.BigIntVector;
    +import org.apache.drill.exec.vector.NullableBigIntVector;
    +import org.apache.drill.exec.vector.NullableFloat8Vector;
    +import org.apache.drill.exec.vector.NullableVarBinaryVector;
    +import org.apache.drill.exec.vector.VarCharVector;
    +import org.apache.drill.exec.vector.complex.MapVector;
    +
    +import java.io.ByteArrayInputStream;
    +import java.io.DataInputStream;
    +import java.io.IOException;
    +import java.util.List;
    +import java.util.GregorianCalendar;
    +import java.util.HashMap;
    +import java.util.Map;
    +import java.util.TimeZone;
    +
    +import com.clearspring.analytics.stream.cardinality.HyperLogLog;
    +
    +public class StatisticsMergeBatch extends AbstractSingleRecordBatch<StatisticsMerge> {
    +  private static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(StatisticsMergeBatch.class);
    +  private Map<String, String> functions;
    +  private boolean first = true;
    +  private boolean finished = false;
    +  private int schema = 0;
    +  private int recordCount = 0;
    +  private List<String> keyList = null;
    +  private Map<MaterializedField, ValueVector> dataSrcVecMap = null;
    +  // Map of non-map fields to VV in the incoming schema
    +  private Map<MaterializedField, ValueVector> copySrcVecMap = null;
    +  private Map<String, Map<String, ValueHolder>> aggregationMap = null;
    +  public StatisticsMergeBatch(StatisticsMerge popConfig, RecordBatch incoming,
    +                              FragmentContext context) throws OutOfMemoryException {
    +    super(popConfig, context, incoming);
    +    this.functions = new HashMap<>();
    +    this.aggregationMap = new HashMap<>();
    +
    +    /*for (String key : popConfig.getFunctions()) {
    +      aggregationMap.put(key, new HashMap<String, ValueHolder>());
    +      if (key.equalsIgnoreCase("statcount") || key.equalsIgnoreCase("nonnullstatcount")) {
    +        functions.put(key, "sum");
    +      } else if (key.equalsIgnoreCase("hll")) {
    +        functions.put(key, "hll_merge");
    +      } else if (key.equalsIgnoreCase("sum_width")) {
    +        functions.put(key, "avg_width");
    +      }
    +    }*/
    +    for (String key : popConfig.getFunctions()) {
    +      if (key.equalsIgnoreCase("sum_width")) {
    +        functions.put(key, "avg_width");
    +      } else if (key.equalsIgnoreCase("hll")) {
    +        functions.put(key, "hll_merge");
    +      } else {
    +        functions.put(key, key);
    +      }
    +      aggregationMap.put(functions.get(key), new HashMap<String, ValueHolder>());
    +    }
    +  }
    +
    +  private void createKeyColumn(String name, LogicalExpression expr, Map<MaterializedField, ValueVector> parentMap)
    +      throws SchemaChangeException {
    +    ErrorCollector collector = new ErrorCollectorImpl();
    +
    +    LogicalExpression mle = ExpressionTreeMaterializer.materialize(expr, incoming, collector,
    +        context.getFunctionRegistry());
    +
    +    MaterializedField outputField = MaterializedField.create(name, mle.getMajorType());
    +    ValueVector vector = TypeHelper.getNewVector(outputField, oContext.getAllocator());
    +    container.add(vector);
    +
    +    if (collector.hasErrors()) {
    +      throw new SchemaChangeException("Failure while materializing expression. "
    +          + collector.toErrorString());
    +    }
    +    parentMap.put(outputField, vector);
    +  }
    +
    +  private ValueVector addMapVector(String name, MapVector parent, LogicalExpression expr) throws SchemaChangeException {
    +    ErrorCollector collector = new ErrorCollectorImpl();
    +
    +    LogicalExpression mle = ExpressionTreeMaterializer.materialize(expr, incoming, collector,
    +        context.getFunctionRegistry());
    +
    +    Class<? extends ValueVector> vvc =
    +        TypeHelper.getValueVectorClass(mle.getMajorType().getMinorType(), mle.getMajorType().getMode());
    +    ValueVector vector = parent.addOrGet(name, mle.getMajorType(), vvc);
    +
    +    if (collector.hasErrors()) {
    +      throw new SchemaChangeException("Failure while materializing expression. "
    +          + collector.toErrorString());
    +    }
    +    return vector;
    +  }
    +
    +  /**
    +   * Identify the list of fields within a map which are unpivoted as columns in output
    +   */
    +  private void buildKeyList() {
    +    List<String> lastMapKeyList = null;
    +    for (VectorWrapper<?> vw : incoming) {
    +      if (vw.getField().getType().getMinorType() != TypeProtos.MinorType.MAP) {
    +        continue;
    +      }
    +
    +      keyList = Lists.newArrayList();
    +
    +      for (ValueVector vv : vw.getValueVector()) {
    +        keyList.add(vv.getField().getLastName());
    +      }
    +
    +      if (lastMapKeyList == null) {
    +        lastMapKeyList = keyList;
    +      } else {
    +        if (keyList.size() != lastMapKeyList.size() || !lastMapKeyList.containsAll(keyList)) {
    +          throw new UnsupportedOperationException("Maps have different fields");
    +        }
    +      }
    +    }
    +  }
    +
    +  private void buildOutputContainer() throws SchemaChangeException {
    +    dataSrcVecMap = Maps.newHashMap();
    +    copySrcVecMap = Maps.newHashMap();
    +    MajorType mt = null;
    +
    +    ErrorCollector collector = new ErrorCollectorImpl();
    +    GregorianCalendar calendar = new GregorianCalendar(TimeZone.getTimeZone("UTC"));
    +
    +    calendar.setTimeInMillis(System.currentTimeMillis());
    +    createKeyColumn("schema", ValueExpressions.getBigInt(schema++), copySrcVecMap);
    +    createKeyColumn("computed", ValueExpressions.getDate(calendar), copySrcVecMap);
    +
    +    for (VectorWrapper<?> vw : incoming) {
    +      addVectorToOutgoingContainer(vw.getField().getLastName(), vw, collector);
    +      /*MaterializedField ds = vw.getField();
    +      String field = vw.getField().getLastName();
    +      // Input map vector
    +      MapVector mapVector = (MapVector) vw.getValueVector();
    +      assert mapVector.getPrimitiveVectors().size() > 0;
    +      // Proceed to create output map vector with same name e.g. statcount etc.
    +      mt = mapVector.getField().getType();
    +      MaterializedField mf = MaterializedField.create(functions.get(field), mt);
    +      assert !dataSrcVecMap.containsKey(mf);
    +      ValueVector vector = TypeHelper.getNewVector(mf, oContext.getAllocator());
    +      container.add(vector);
    +      MapVector outputMapVector = (MapVector) vector;
    +
    +      for (ValueVector vv : mapVector) {
    +        String fieldName = vv.getField().getLastName();
    +        if (!keyList.contains(fieldName)) {
    +          throw new UnsupportedOperationException("Unpivot data vector " +
    +              ds + " contains key " + fieldName + " not contained in key source!");
    +        }
    +        if (vv.getField().getType().getMinorType() == TypeProtos.MinorType.MAP) {
    +          throw new UnsupportedOperationException("Unpivot of nested map is not supported!");
    +        }
    +        if (field.equals("column")) {
    +          outputMapVector.addOrGet(fieldName, vv.getField().getType(), vv.getClass());
    +        } else {
    +          List<LogicalExpression> args = Lists.newArrayList();
    +          //TODO: Something else to access value of col such as emp_id?
    +          args.add(SchemaPath.getSimplePath(vv.getField().getPath()));
    +          //TODO: Put in the mapVector
    +          LogicalExpression call = FunctionCallFactory.createExpression(functions.get(field), args);
    +          //TODO: Is this sufficient to add to new Map?
    +          ValueVector vector1 = addMapVector(fieldName, outputMapVector, call);
    +          if (collector.hasErrors()) {
    +            throw new SchemaChangeException("Failure while materializing expression. "
    +                + collector.toErrorString());
    +          }
    +        }
    +      }
    +      dataSrcVecMap.put(ds, outputMapVector);*/
    +    }
    +    //Now create NDV in the outgoing container which was not avaliable in the incoming
    +    for (VectorWrapper<?> vw : incoming) {
    +      if (vw.getField().getLastName().equalsIgnoreCase("sum_width")) {//NullableFloat8 type
    +        addVectorToOutgoingContainer("ndv", vw, collector);
    +        break;
    +      }
    +    }
    +    container.setRecordCount(0);
    +    recordCount = 0;
    +    container.buildSchema(incoming.getSchema().getSelectionVectorMode());
    +  }
    +
    +  private void addVectorToOutgoingContainer(String field, VectorWrapper vw, ErrorCollector collector)
    +     throws SchemaChangeException {
    +    // Input map vector
    +    MapVector mapVector = (MapVector) vw.getValueVector();
    +    MaterializedField mf;
    +    assert mapVector.getPrimitiveVectors().size() > 0;
    +    // Proceed to create output map vector with same name e.g. statcount etc.
    +    MajorType mt = mapVector.getField().getType();
    +    if (functions.get(field) != null) {
    +      mf = MaterializedField.create(functions.get(field), mt);
    +    } else {
    +      mf = MaterializedField.create(field, mt);
    +    }
    +    assert !dataSrcVecMap.containsKey(mf);
    +    ValueVector vector = TypeHelper.getNewVector(mf, oContext.getAllocator());
    +    container.add(vector);
    +    MapVector outputMapVector = (MapVector) vector;
    +
    +    for (ValueVector vv : mapVector) {
    +      String fieldName = vv.getField().getLastName();
    +      if (!keyList.contains(fieldName)) {
    +        throw new UnsupportedOperationException("Unpivot data vector " +
    +                field + " contains key " + fieldName + " not contained in key source!");
    +      }
    +      if (vv.getField().getType().getMinorType() == TypeProtos.MinorType.MAP) {
    +        throw new UnsupportedOperationException("Unpivot of nested map is not supported!");
    +      }
    +      if (field.equals("column")) {
    +        outputMapVector.addOrGet(fieldName, vv.getField().getType(), vv.getClass());
    +      } else {
    +        List<LogicalExpression> args = Lists.newArrayList();
    +        LogicalExpression call;
    +        //TODO: Something else to access value of col such as emp_id?
    +        args.add(SchemaPath.getSimplePath(vv.getField().getPath()));
    +        //TODO: Put in the mapVector
    +        if (functions.get(field) != null) {
    +          call = FunctionCallFactory.createExpression(functions.get(field), args);
    +        } else {
    +          call = FunctionCallFactory.createExpression(field, args);
    +        }
    +        //TODO: Is this sufficient to add to new Map?
    +        ValueVector vector1 = addMapVector(fieldName, outputMapVector, call);
    +        if (collector.hasErrors()) {
    +          throw new SchemaChangeException("Failure while materializing expression. "
    +                  + collector.toErrorString());
    +        }
    +      }
    +    }
    +  }
    +
    +  @Override
    +  protected boolean setupNewSchema() throws SchemaChangeException {
    +    container.clear();
    +    buildKeyList();
    +    buildOutputContainer();
    +    return true;
    +  }
    +
    +  @Override
    +  protected IterOutcome doWork() {
    +    int outRecordCount = incoming.getRecordCount();
    +    HashMap<String, Long> nonNullRows = new HashMap<>();
    +
    +    for (VectorWrapper<?> vw : incoming) {
    +      ValueVector vv = vw.getValueVector();
    +      if (vv.getField().getType().getMinorType() != TypeProtos.MinorType.MAP) {
    +        //We only expect Maps in the incoming. STOP, if this is not the case.
    +        return IterOutcome.STOP;
    +      }
    +      MapVector mapVec = (MapVector) vv;
    +      Map<String, ValueHolder> statMap = aggregationMap.get(vv.getField().getLastName());
    +
    +      for (ValueVector mapElt : mapVec) {
    +        if (vv.getField().getLastName().equalsIgnoreCase("column")) {
    +          BigIntHolder nameHolder;
    +          if (statMap.get(mapElt.getField().getLastName()) != null) {
    +            nameHolder = (BigIntHolder) statMap.get(mapElt.getField().getLastName());
    +          } else {
    +            nameHolder = new BigIntHolder();
    +            statMap.put(mapElt.getField().getLastName(), nameHolder);
    +          }
    +          nameHolder.value = 1;
    +        } else if (vv.getField().getLastName().equalsIgnoreCase("statcount")
    +            || vv.getField().getLastName().equalsIgnoreCase("nonnullstatcount")) {
    +          BigIntHolder sumHolder;
    +          String colName = mapElt.getField().getLastName();
    +          if (statMap.get(colName) != null) {
    +            sumHolder = (BigIntHolder) statMap.get(colName);
    +          } else {
    +            sumHolder = new BigIntHolder();
    +            statMap.put(colName, sumHolder);
    +          }
    +          //TODO: assert size = 1
    +          //TODO: logger
    +          Object val = mapElt.getAccessor().getObject(0);
    +          if (val != null) {
    +            sumHolder.value += (long)val;
    +          }
    +        } else if (vv.getField().getLastName().equalsIgnoreCase("sum_width")) {
    +          NullableFloat8Holder sumHolder;
    +          String colName = mapElt.getField().getLastName();
    +          if (statMap == null) {
    +            statMap = aggregationMap.get(functions.get(vv.getField().getLastName()));
    +          }
    +          if (statMap.get(colName) != null) {
    +            sumHolder = (NullableFloat8Holder) statMap.get(colName);
    +          } else {
    +            sumHolder = new NullableFloat8Holder();
    +            statMap.put(colName, sumHolder);
    +          }
    +          //TODO: assert size = 1
    +          //TODO: logger
    +          Object val = mapElt.getAccessor().getObject(0);
    +          if (val != null) {
    +            sumHolder.value += (double) val;
    +            sumHolder.isSet = 1;
    +          }
    +        } else if (vv.getField().getLastName().equalsIgnoreCase("hll")) {
    +          ObjectHolder hllHolder;
    +          String colName = mapElt.getField().getLastName();
    +          if (statMap == null) {
    +            statMap = aggregationMap.get(functions.get(vv.getField().getLastName()));
    +          }
    +          if (statMap.get(colName) != null) {
    +            hllHolder = (ObjectHolder) statMap.get(colName);
    +          } else {
    +            hllHolder = new ObjectHolder();
    +            hllHolder.obj = new HyperLogLog(context.getContextInformation().getHllAccuracy());
    +            statMap.put(colName, hllHolder);
    +          }
    +          NullableVarBinaryVector hllVector = (NullableVarBinaryVector) mapElt;
    +          try {
    +            if (hllVector.getAccessor().isSet(0) == 1) {
    +              ByteArrayInputStream bais = new ByteArrayInputStream(hllVector.getAccessor().getObject(0), 0,
    +                  mapElt.getBufferSize());
    +              HyperLogLog other = HyperLogLog.Builder.build(new DataInputStream(bais));
    +              ((HyperLogLog) hllHolder.obj).addAll(other);
    +            }
    +          } catch (Exception ex) {
    +            //TODO: Catch IOException/CardinalityMergeException
    +            //TODO: logger
    +            return IterOutcome.STOP;
    +          }
    +        }
    +      }
    +      // Add NDV value vector map using HLL map (since the NDV map is directly generated from HLL and not produced by the underlying
    +      // Statistics Agg)
    +      Map<String, ValueHolder> hllMap = aggregationMap.get("hll");
    +      if (hllMap != null) {
    +        aggregationMap.put("ndv", hllMap);
    +      }
    +    }
    +    return IterOutcome.OK;
    +  }
    +
    +  public VectorContainer getOutgoingContainer() {
    +    return this.container;
    +  }
    +
    +  @Override
    +  public IterOutcome innerNext() {
    +    IterOutcome outcome;
    +    boolean didSomeWork = false;
    +    if (finished) {
    +      return IterOutcome.NONE;
    +    }
    +    try {
    +      outer: while (true) {
    +        outcome = next(incoming);
    +        switch (outcome) {
    +          case NONE:
    +            break outer;
    +          case OUT_OF_MEMORY:
    +          case NOT_YET:
    +          case STOP:
    +            return outcome;
    +          case OK_NEW_SCHEMA:
    +            if (first) {
    +              first =false;
    +              if (!setupNewSchema()) {
    +                outcome = IterOutcome.OK;
    +              }
    +              return outcome;
    +            }
    +            //fall through
    +          case OK:
    +            assert first == false : "First batch should be OK_NEW_SCHEMA";
    +            IterOutcome out = doWork();
    +            didSomeWork = true;
    +            if (out != IterOutcome.OK) {
    +              return out;
    +            }
    +            break;
    +          default:
    +            throw new UnsupportedOperationException("Unsupported upstream state " + outcome);
    +        }
    +      }
    +    } catch (SchemaChangeException ex) {
    +      kill(false);
    +      logger.error("Failure during query", ex);
    +      context.fail(ex);
    +      return IterOutcome.STOP;
    +    }
    +
    +    // We can only get here if upstream is NONE i.e. no more batches. If we did some work prior to
    +    // exhausting all upstream, then return OK. Otherwise, return NONE.
    +    if (didSomeWork) {
    +      IterOutcome out = buildOutgoingRecordBatch();
    +      finished = true;
    +      return out;
    +    } else {
    +      return outcome;
    +    }
    +  }
    +
    +  // Prepare the outgoing container
    +  private IterOutcome buildOutgoingRecordBatch() {
    --- End diff --
    
    Again, this function is far to complex to review without some explanation. Please add comments. Please break up functionality into smaller per-function methods.


---
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 #729: Drill 1328: Support table statistics for Parquet

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/729#discussion_r102867030
  
    --- Diff: exec/java-exec/src/main/java/org/apache/drill/exec/ops/FragmentContext.java ---
    @@ -245,6 +247,18 @@ public SchemaPlus getRootSchema() {
       }
     
       /**
    +   * Returns the statement type (e.g. SELECT, CTAS, ANALYZE) from the query context.
    +   * @return query statement type {@link SqlStatementType}, if known.
    +   */
    +  public SqlStatementType getSQLStatementType() {
    +    if (queryContext == null) {
    +      fail(new UnsupportedOperationException("Statement type is only valid for root fragment. " +
    --- End diff --
    
    The `fail()` call is for runtime errors due to external causes, user error, etc. Here, we have a programming error. Better ways to handle this:
    
    ```
    if (queryContext == null) {
      throw new IllegalStateException("Statement type...");
    }
    ```
    Or just:
    
    ```
    Preconditions.checkNotNull(queryContext);
    ```


---
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 #729: Drill 1328: Support table statistics for Parquet

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/729#discussion_r102871374
  
    --- Diff: exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/statistics/AvgWidthMergedStatistic.java ---
    @@ -0,0 +1,135 @@
    +/*
    + * 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.impl.statistics;
    +
    +import org.apache.drill.common.types.TypeProtos;
    +import org.apache.drill.exec.expr.holders.NullableFloat8Holder;
    +import org.apache.drill.exec.expr.holders.ValueHolder;
    +import org.apache.drill.exec.vector.NullableFloat8Vector;
    +import org.apache.drill.exec.vector.ValueVector;
    +import org.apache.drill.exec.vector.complex.MapVector;
    +
    +import java.util.HashMap;
    +import java.util.List;
    +import java.util.Map;
    +
    +public class AvgWidthMergedStatistic extends AbstractMergedStatistic {
    +
    +  private String name;
    +  private String inputName;
    +  private boolean configureComplete = false;
    +  private boolean mergeComplete = false;
    +  private Map<String, ValueHolder> sumHolder;
    +  MergedStatistic types, nonNullStatCounts, statCounts;
    +
    +  public AvgWidthMergedStatistic (String name, String inputName) {
    +    this.name = name;
    +    this.inputName = inputName;
    +    this.sumHolder = new HashMap<>();
    +    types = nonNullStatCounts = statCounts = null;
    +  }
    +
    +  @Override
    +  public String getName() {
    +    return name;
    +  }
    +
    +  @Override
    +  public String getInput() {
    +    return inputName;
    +  }
    +
    +  @Override
    +  public void merge(ValueVector input) {
    +    // Check the input is a Map Vector
    +    assert (input.getField().getType().getMinorType() == TypeProtos.MinorType.MAP);
    +    MapVector inputMap = (MapVector) input;
    +    for (ValueVector vv : inputMap) {
    +      String colName = vv.getField().getLastName();
    +      NullableFloat8Holder colSumHolder;
    +      if (sumHolder.get(colName) != null) {
    +        colSumHolder = (NullableFloat8Holder) sumHolder.get(colName);
    +      } else {
    +        colSumHolder = new NullableFloat8Holder();
    +        sumHolder.put(colName, colSumHolder);
    +      }
    +      Object val = vv.getAccessor().getObject(0);
    +      if (val != null) {
    +        colSumHolder.value += (double) val;
    +        colSumHolder.isSet = 1;
    +      }
    +    }
    +  }
    +
    +  @Override
    +  public Object getStat(String colName) {
    +      if (mergeComplete != true) {
    +        throw new IllegalStateException(
    +            String.format("Statistic `%s` has not completed merging statistics", name));
    +      }
    +      NullableFloat8Holder colSumHolder = (NullableFloat8Holder) sumHolder.get(colName);
    +      return (long) (colSumHolder.value/ getRowCount(colName));
    +    }
    +
    +  @Override
    +  public void setOutput(ValueVector output) {
    +    // Check the input is a Map Vector
    +    assert (output.getField().getType().getMinorType() == TypeProtos.MinorType.MAP);
    +    // Dependencies have been configured correctly
    +    assert (configureComplete == true);
    +    MapVector outputMap = (MapVector) output;
    +
    +    for (ValueVector outMapCol : outputMap) {
    +      String colName = outMapCol.getField().getLastName();
    +      NullableFloat8Holder colSumHolder = (NullableFloat8Holder) sumHolder.get(colName);
    +      NullableFloat8Vector vv = (NullableFloat8Vector) outMapCol;
    +      vv.allocateNewSafe();
    +      vv.getMutator().setSafe(0, (colSumHolder.value / getRowCount(colName)));
    +    }
    +    mergeComplete = true;
    +  }
    +
    +  @Override
    +  public void configure(Object configurations) {
    +    List<MergedStatistic> statistics = (List<MergedStatistic>) configurations;
    --- End diff --
    
    ```
    public void configure(List<MergedStatistic> configurations) {
    ```
    
    Please don't use generic Object and casting unless absolutely necessary. (Sometimes it is. This doesn't appear to be one of those times...)


---
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 #729: Drill 1328: Support table statistics for Parquet

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

    https://github.com/apache/drill/pull/729#discussion_r102345408
  
    --- Diff: exec/java-exec/src/main/java/org/apache/drill/exec/planner/cost/DrillRelMdSelectivity.java ---
    @@ -0,0 +1,219 @@
    +/*******************************************************************************
    + * 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.planner.cost;
    +
    +import org.apache.calcite.plan.RelOptUtil;
    +import org.apache.calcite.plan.volcano.RelSubset;
    +import org.apache.calcite.rel.RelNode;
    +import org.apache.calcite.rel.SingleRel;
    +import org.apache.calcite.rel.core.JoinRelType;
    +import org.apache.calcite.rel.core.TableScan;
    +import org.apache.calcite.rel.metadata.ReflectiveRelMetadataProvider;
    +import org.apache.calcite.rel.metadata.RelMdSelectivity;
    +import org.apache.calcite.rel.metadata.RelMdUtil;
    +import org.apache.calcite.rel.metadata.RelMetadataProvider;
    +import org.apache.calcite.rel.metadata.RelMetadataQuery;
    +import org.apache.calcite.rel.type.RelDataType;
    +import org.apache.calcite.rex.RexBuilder;
    +import org.apache.calcite.rex.RexCall;
    +import org.apache.calcite.rex.RexInputRef;
    +import org.apache.calcite.rex.RexNode;
    +import org.apache.calcite.rex.RexUtil;
    +import org.apache.calcite.rex.RexVisitor;
    +import org.apache.calcite.rex.RexVisitorImpl;
    +import org.apache.calcite.sql.SqlKind;
    +import org.apache.calcite.util.BuiltInMethod;
    +import org.apache.calcite.util.Util;
    +import org.apache.drill.exec.planner.common.DrillJoinRelBase;
    +import org.apache.drill.exec.planner.common.DrillRelOptUtil;
    +import org.apache.drill.exec.planner.logical.DrillTable;
    +import org.apache.drill.exec.planner.logical.DrillTranslatableTable;
    +
    +import java.util.ArrayList;
    +import java.util.List;
    +
    +public class DrillRelMdSelectivity extends RelMdSelectivity {
    +  private static final org.slf4j.Logger logger =
    +      org.slf4j.LoggerFactory.getLogger(RelMdSelectivity.class);
    +
    +  private static final DrillRelMdSelectivity INSTANCE =
    +      new DrillRelMdSelectivity();
    +
    +  public static final RelMetadataProvider SOURCE =
    +      ReflectiveRelMetadataProvider.reflectiveSource(
    +          BuiltInMethod.SELECTIVITY.method, INSTANCE);
    +
    +  @Override
    +  public Double getSelectivity(RelNode rel, RexNode predicate) {
    +    if (rel instanceof TableScan) {
    +      return getScanSelectivity((TableScan) rel, predicate);
    +    } else if (rel instanceof DrillJoinRelBase) {
    +      return getJoinSelectivity(((DrillJoinRelBase) rel), predicate);
    +    } else if (rel instanceof SingleRel && !DrillRelOptUtil.guessRows(rel)) {
    +        return RelMetadataQuery.getSelectivity(((SingleRel) rel).getInput(), predicate);
    +    } else if (rel instanceof RelSubset && !DrillRelOptUtil.guessRows(rel)) {
    +      if (((RelSubset) rel).getBest() != null) {
    +        return RelMetadataQuery.getSelectivity(((RelSubset)rel).getBest(), predicate);
    +      } else if (((RelSubset)rel).getOriginal() != null) {
    +        return RelMetadataQuery.getSelectivity(((RelSubset)rel).getOriginal(), predicate);
    +      } else {
    +        return super.getSelectivity(rel, predicate);
    +      }
    +    } else {
    +      return super.getSelectivity(rel, predicate);
    +    }
    +  }
    +
    +  private Double getJoinSelectivity(DrillJoinRelBase rel, RexNode predicate) {
    +    double sel = 1.0;
    +    // determine which filters apply to the left vs right
    +    RexNode leftPred = null;
    +    RexNode rightPred = null;
    +    JoinRelType joinType = rel.getJoinType();
    +    final RexBuilder rexBuilder = rel.getCluster().getRexBuilder();
    +    int[] adjustments = new int[rel.getRowType().getFieldCount()];
    +
    +    if (DrillRelOptUtil.guessRows(rel)) {
    +      return super.getSelectivity(rel, predicate);
    +    }
    +
    +    if (predicate != null) {
    +      RexNode pred;
    +      List<RexNode> leftFilters = new ArrayList<RexNode>();
    +      List<RexNode> rightFilters = new ArrayList<RexNode>();
    +      List<RexNode> joinFilters = new ArrayList<RexNode>();
    +      List<RexNode> predList = RelOptUtil.conjunctions(predicate);
    +
    +      RelOptUtil.classifyFilters(
    +          rel,
    +          predList,
    +          joinType,
    +          joinType == JoinRelType.INNER,
    +          !joinType.generatesNullsOnLeft(),
    +          !joinType.generatesNullsOnRight(),
    +          joinFilters,
    +          leftFilters,
    +          rightFilters);
    +      leftPred =
    +          RexUtil.composeConjunction(rexBuilder, leftFilters, true);
    +      rightPred =
    +          RexUtil.composeConjunction(rexBuilder, rightFilters, true);
    +      for (RelNode child : rel.getInputs()) {
    +        RexNode modifiedPred = null;
    +
    +        if (child == rel.getLeft()) {
    +          pred = leftPred;
    +        } else {
    +          pred = rightPred;
    +        }
    +        if (pred != null) {
    +          // convert the predicate to reference the types of the children
    +          modifiedPred =
    +              pred.accept(new RelOptUtil.RexInputConverter(
    +              rexBuilder,
    +              null,
    +              child.getRowType().getFieldList(),
    +              adjustments));
    +        }
    +        sel *= RelMetadataQuery.getSelectivity(child, modifiedPred);
    +      }
    +      sel *= RelMdUtil.guessSelectivity(
    +          RexUtil.composeConjunction(rexBuilder, joinFilters, true));
    +    }
    +    return sel;
    +  }
    +
    +  private Double getScanSelectivity(TableScan scan, RexNode predicate) {
    +    DrillTable table = scan.getTable().unwrap(DrillTable.class);
    +    if (table == null) {
    +      table = scan.getTable().unwrap(DrillTranslatableTable.class).getDrillTable();
    +    }
    +    if (table == null || table.getStatsTable() == null) {
    +      return super.getSelectivity(scan, predicate);
    +    } else {
    +      return getScanSelectivityInternal(table, predicate, scan.getRowType());
    +    }
    +  }
    +
    +  private Double getScanSelectivityInternal(DrillTable table, RexNode predicate,
    +      RelDataType type) {
    +    double sel = 1.0;
    +
    +    if ((predicate == null) || predicate.isAlwaysTrue()) {
    +      return sel;
    +    }
    +
    +    for (RexNode pred : RelOptUtil.conjunctions(predicate)) {
    +      double orSel = 0;
    +      for (RexNode orPred : RelOptUtil.disjunctions(pred)) {
    +        //CALCITE guess
    +        Double guess = RelMdUtil.guessSelectivity(pred);
    +        if (orPred.isA(SqlKind.EQUALS)) {
    --- End diff --
    
    As discussed, the DRILL-5254 changes need to go in as Calcite changes. So this change can be done only after we get these changes into Calcite. 


---
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 #729: Drill 1328: Support table statistics for Parquet

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/729#discussion_r102863904
  
    --- Diff: exec/java-exec/src/main/codegen/data/Parser.tdd ---
    @@ -39,7 +39,13 @@
         "METADATA",
         "DATABASE",
         "IF",
    -    "JAR"
    +    "JAR",
    +    "ANALYZE",
    +    "COMPUTE",
    +    "ESTIMATE",
    +    "STATISTICS",
    +    "SAMPLE",
    +    "PERCENT"
    --- End diff --
    
    We need a solution to this problem. For this feature, we probably just have to live with breaking existing queries. Please be sure the list of new keywords goes into both release notes and a [list of reserved words](http://drill.apache.org/docs/reserved-keywords/) on the Apache Drill web site.


---
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 #729: Drill 1328: Support table statistics for Parquet

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

    https://github.com/apache/drill/pull/729#discussion_r103366732
  
    --- Diff: exec/java-exec/src/main/java/org/apache/drill/exec/ops/QueryContext.java ---
    @@ -283,4 +288,22 @@ public void close() throws Exception {
           closed = true;
         }
       }
    +
    +  /**
    +  * @param stmtType : Sets the type {@link SqlStatementType} of the statement e.g. CTAS, ANALYZE
    +  */
    +  public void setSQLStatementType(SqlStatementType stmtType) {
    +    if (this.stmtType == null) {
    +      this.stmtType = stmtType;
    +    } else {
    +      throw new UnsupportedOperationException("SQL Statement type is already set");
    --- End diff --
    
    Changed to `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 #729: Drill 1328: Support table statistics for Parquet

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/729#discussion_r100680855
  
    --- Diff: exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/statistics/StatisticsMergeBatch.java ---
    @@ -0,0 +1,532 @@
    +/**
    + * 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.impl.statistics;
    +
    +import com.google.common.collect.Lists;
    +import com.google.common.collect.Maps;
    +import org.apache.drill.common.expression.LogicalExpression;
    +import org.apache.drill.common.expression.ErrorCollector;
    +import org.apache.drill.common.expression.ErrorCollectorImpl;
    +import org.apache.drill.common.expression.FunctionCallFactory;
    +import org.apache.drill.common.expression.SchemaPath;
    +import org.apache.drill.common.expression.ValueExpressions;
    +import org.apache.drill.common.types.TypeProtos;
    +import org.apache.drill.common.types.TypeProtos.MajorType;
    +import org.apache.drill.exec.exception.OutOfMemoryException;
    +import org.apache.drill.exec.exception.SchemaChangeException;
    +import org.apache.drill.exec.expr.ExpressionTreeMaterializer;
    +import org.apache.drill.exec.expr.TypeHelper;
    +import org.apache.drill.exec.expr.holders.BigIntHolder;
    +import org.apache.drill.exec.expr.holders.NullableFloat8Holder;
    +import org.apache.drill.exec.expr.holders.ObjectHolder;
    +import org.apache.drill.exec.expr.holders.ValueHolder;
    +import org.apache.drill.exec.ops.FragmentContext;
    +import org.apache.drill.exec.physical.config.StatisticsMerge;
    +import org.apache.drill.exec.record.AbstractSingleRecordBatch;
    +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.VectorContainer;
    +import org.apache.drill.exec.record.VectorWrapper;
    +import org.apache.drill.exec.vector.ValueVector;
    +import org.apache.drill.exec.vector.DateVector;
    +import org.apache.drill.exec.vector.BigIntVector;
    +import org.apache.drill.exec.vector.NullableBigIntVector;
    +import org.apache.drill.exec.vector.NullableFloat8Vector;
    +import org.apache.drill.exec.vector.NullableVarBinaryVector;
    +import org.apache.drill.exec.vector.VarCharVector;
    +import org.apache.drill.exec.vector.complex.MapVector;
    +
    +import java.io.ByteArrayInputStream;
    +import java.io.DataInputStream;
    +import java.io.IOException;
    +import java.util.List;
    +import java.util.GregorianCalendar;
    +import java.util.HashMap;
    +import java.util.Map;
    +import java.util.TimeZone;
    +
    +import com.clearspring.analytics.stream.cardinality.HyperLogLog;
    +
    +public class StatisticsMergeBatch extends AbstractSingleRecordBatch<StatisticsMerge> {
    +  private static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(StatisticsMergeBatch.class);
    +  private Map<String, String> functions;
    +  private boolean first = true;
    +  private boolean finished = false;
    +  private int schema = 0;
    +  private int recordCount = 0;
    +  private List<String> keyList = null;
    +  private Map<MaterializedField, ValueVector> dataSrcVecMap = null;
    +  // Map of non-map fields to VV in the incoming schema
    +  private Map<MaterializedField, ValueVector> copySrcVecMap = null;
    +  private Map<String, Map<String, ValueHolder>> aggregationMap = null;
    +  public StatisticsMergeBatch(StatisticsMerge popConfig, RecordBatch incoming,
    +                              FragmentContext context) throws OutOfMemoryException {
    +    super(popConfig, context, incoming);
    +    this.functions = new HashMap<>();
    +    this.aggregationMap = new HashMap<>();
    +
    +    /*for (String key : popConfig.getFunctions()) {
    +      aggregationMap.put(key, new HashMap<String, ValueHolder>());
    +      if (key.equalsIgnoreCase("statcount") || key.equalsIgnoreCase("nonnullstatcount")) {
    +        functions.put(key, "sum");
    +      } else if (key.equalsIgnoreCase("hll")) {
    +        functions.put(key, "hll_merge");
    +      } else if (key.equalsIgnoreCase("sum_width")) {
    +        functions.put(key, "avg_width");
    +      }
    +    }*/
    +    for (String key : popConfig.getFunctions()) {
    +      if (key.equalsIgnoreCase("sum_width")) {
    +        functions.put(key, "avg_width");
    +      } else if (key.equalsIgnoreCase("hll")) {
    +        functions.put(key, "hll_merge");
    +      } else {
    +        functions.put(key, key);
    +      }
    +      aggregationMap.put(functions.get(key), new HashMap<String, ValueHolder>());
    +    }
    +  }
    +
    +  private void createKeyColumn(String name, LogicalExpression expr, Map<MaterializedField, ValueVector> parentMap)
    +      throws SchemaChangeException {
    +    ErrorCollector collector = new ErrorCollectorImpl();
    +
    +    LogicalExpression mle = ExpressionTreeMaterializer.materialize(expr, incoming, collector,
    +        context.getFunctionRegistry());
    +
    +    MaterializedField outputField = MaterializedField.create(name, mle.getMajorType());
    +    ValueVector vector = TypeHelper.getNewVector(outputField, oContext.getAllocator());
    +    container.add(vector);
    +
    +    if (collector.hasErrors()) {
    +      throw new SchemaChangeException("Failure while materializing expression. "
    +          + collector.toErrorString());
    +    }
    +    parentMap.put(outputField, vector);
    +  }
    +
    +  private ValueVector addMapVector(String name, MapVector parent, LogicalExpression expr) throws SchemaChangeException {
    +    ErrorCollector collector = new ErrorCollectorImpl();
    +
    +    LogicalExpression mle = ExpressionTreeMaterializer.materialize(expr, incoming, collector,
    +        context.getFunctionRegistry());
    +
    +    Class<? extends ValueVector> vvc =
    +        TypeHelper.getValueVectorClass(mle.getMajorType().getMinorType(), mle.getMajorType().getMode());
    +    ValueVector vector = parent.addOrGet(name, mle.getMajorType(), vvc);
    +
    +    if (collector.hasErrors()) {
    +      throw new SchemaChangeException("Failure while materializing expression. "
    +          + collector.toErrorString());
    +    }
    +    return vector;
    +  }
    +
    +  /**
    +   * Identify the list of fields within a map which are unpivoted as columns in output
    +   */
    +  private void buildKeyList() {
    +    List<String> lastMapKeyList = null;
    +    for (VectorWrapper<?> vw : incoming) {
    +      if (vw.getField().getType().getMinorType() != TypeProtos.MinorType.MAP) {
    +        continue;
    +      }
    +
    +      keyList = Lists.newArrayList();
    +
    +      for (ValueVector vv : vw.getValueVector()) {
    +        keyList.add(vv.getField().getLastName());
    +      }
    +
    +      if (lastMapKeyList == null) {
    +        lastMapKeyList = keyList;
    +      } else {
    +        if (keyList.size() != lastMapKeyList.size() || !lastMapKeyList.containsAll(keyList)) {
    +          throw new UnsupportedOperationException("Maps have different fields");
    +        }
    +      }
    +    }
    +  }
    +
    +  private void buildOutputContainer() throws SchemaChangeException {
    +    dataSrcVecMap = Maps.newHashMap();
    +    copySrcVecMap = Maps.newHashMap();
    +    MajorType mt = null;
    +
    +    ErrorCollector collector = new ErrorCollectorImpl();
    +    GregorianCalendar calendar = new GregorianCalendar(TimeZone.getTimeZone("UTC"));
    +
    +    calendar.setTimeInMillis(System.currentTimeMillis());
    +    createKeyColumn("schema", ValueExpressions.getBigInt(schema++), copySrcVecMap);
    +    createKeyColumn("computed", ValueExpressions.getDate(calendar), copySrcVecMap);
    +
    +    for (VectorWrapper<?> vw : incoming) {
    +      addVectorToOutgoingContainer(vw.getField().getLastName(), vw, collector);
    +      /*MaterializedField ds = vw.getField();
    +      String field = vw.getField().getLastName();
    +      // Input map vector
    +      MapVector mapVector = (MapVector) vw.getValueVector();
    +      assert mapVector.getPrimitiveVectors().size() > 0;
    +      // Proceed to create output map vector with same name e.g. statcount etc.
    +      mt = mapVector.getField().getType();
    +      MaterializedField mf = MaterializedField.create(functions.get(field), mt);
    +      assert !dataSrcVecMap.containsKey(mf);
    +      ValueVector vector = TypeHelper.getNewVector(mf, oContext.getAllocator());
    +      container.add(vector);
    +      MapVector outputMapVector = (MapVector) vector;
    +
    +      for (ValueVector vv : mapVector) {
    +        String fieldName = vv.getField().getLastName();
    +        if (!keyList.contains(fieldName)) {
    +          throw new UnsupportedOperationException("Unpivot data vector " +
    +              ds + " contains key " + fieldName + " not contained in key source!");
    +        }
    +        if (vv.getField().getType().getMinorType() == TypeProtos.MinorType.MAP) {
    +          throw new UnsupportedOperationException("Unpivot of nested map is not supported!");
    +        }
    +        if (field.equals("column")) {
    +          outputMapVector.addOrGet(fieldName, vv.getField().getType(), vv.getClass());
    +        } else {
    +          List<LogicalExpression> args = Lists.newArrayList();
    +          //TODO: Something else to access value of col such as emp_id?
    +          args.add(SchemaPath.getSimplePath(vv.getField().getPath()));
    +          //TODO: Put in the mapVector
    +          LogicalExpression call = FunctionCallFactory.createExpression(functions.get(field), args);
    +          //TODO: Is this sufficient to add to new Map?
    +          ValueVector vector1 = addMapVector(fieldName, outputMapVector, call);
    +          if (collector.hasErrors()) {
    +            throw new SchemaChangeException("Failure while materializing expression. "
    +                + collector.toErrorString());
    +          }
    +        }
    +      }
    +      dataSrcVecMap.put(ds, outputMapVector);*/
    +    }
    +    //Now create NDV in the outgoing container which was not avaliable in the incoming
    +    for (VectorWrapper<?> vw : incoming) {
    +      if (vw.getField().getLastName().equalsIgnoreCase("sum_width")) {//NullableFloat8 type
    +        addVectorToOutgoingContainer("ndv", vw, collector);
    --- End diff --
    
    Again, we are hard-coding knowledge of certain functions here in the code, makes it hard to extend the set of functions.
    
    See comment below about using hard-coded strings rather than declared constants.


---
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 #729: Drill 1328: Support table statistics for Parquet

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

    https://github.com/apache/drill/pull/729#discussion_r100705962
  
    --- Diff: exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/aggregate/InternalBatch.java ---
    @@ -54,6 +56,7 @@ public InternalBatch(RecordBatch incoming, VectorWrapper[] ignoreWrappers, Opera
           this.sv2 = null;
         }
         this.schema = incoming.getSchema();
    +    this.incoming = incoming;
    --- End diff --
    
    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 #729: Drill 1328: Support table statistics for Parquet

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/729#discussion_r100677240
  
    --- Diff: exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/statistics/StatisticsAggBatch.java ---
    @@ -0,0 +1,256 @@
    +/**
    + * 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.impl.statistics;
    +
    +import com.google.common.collect.Lists;
    +import com.sun.codemodel.JExpr;
    +import org.apache.drill.common.expression.ErrorCollector;
    +import org.apache.drill.common.expression.ErrorCollectorImpl;
    +import org.apache.drill.common.expression.FunctionCallFactory;
    +import org.apache.drill.common.expression.LogicalExpression;
    +import org.apache.drill.common.expression.SchemaPath;
    +import org.apache.drill.common.expression.ValueExpressions;
    +import org.apache.drill.exec.exception.ClassTransformationException;
    +import org.apache.drill.exec.exception.OutOfMemoryException;
    +import org.apache.drill.exec.exception.SchemaChangeException;
    +import org.apache.drill.exec.expr.ClassGenerator;
    +import org.apache.drill.exec.expr.CodeGenerator;
    +import org.apache.drill.exec.expr.ExpressionTreeMaterializer;
    +import org.apache.drill.exec.expr.TypeHelper;
    +import org.apache.drill.exec.expr.ValueVectorWriteExpression;
    +import org.apache.drill.exec.ops.FragmentContext;
    +import org.apache.drill.exec.physical.config.StatisticsAggregate;
    +import org.apache.drill.exec.physical.impl.aggregate.StreamingAggBatch;
    +import org.apache.drill.exec.physical.impl.aggregate.StreamingAggTemplate;
    +import org.apache.drill.exec.physical.impl.aggregate.StreamingAggregator;
    +import org.apache.drill.exec.planner.physical.StatsAggPrel.OperatorPhase;
    +import org.apache.drill.exec.record.BatchSchema.SelectionVectorMode;
    +import org.apache.drill.exec.record.MaterializedField;
    +import org.apache.drill.exec.record.RecordBatch;
    +import org.apache.drill.exec.record.TypedFieldId;
    +import org.apache.drill.exec.store.ImplicitColumnExplorer;
    +import org.apache.drill.exec.vector.ValueVector;
    +import org.apache.drill.exec.vector.complex.FieldIdUtil;
    +import org.apache.drill.exec.vector.complex.MapVector;
    +
    +import java.io.IOException;
    +import java.util.GregorianCalendar;
    +import java.util.List;
    +import java.util.TimeZone;
    +
    +/**
    + * TODO: This needs cleanup. Currently the key values are constants and we compare the constants
    + * for every record. Seems unnecessary.
    + *
    + * Example input and output:
    + * Schema of incoming batch: region_id (VARCHAR), sales_city (VARCHAR), cnt (BIGINT)
    + * Schema of output:
    + *    "schema" : BIGINT - Schema number. For each schema change this number is incremented.
    + *    "computed" : BIGINT - What time is it computed?
    + *    "columns"       : MAP - Column names
    + *       "region_id"  : VARCHAR
    + *       "sales_city" : VARCHAR
    + *       "cnt"        : VARCHAR
    + *    "statscount" : MAP
    + *       "region_id"  : BIGINT - statscount(region_id) - aggregation over all values of region_id
    + *                      in incoming batch
    + *       "sales_city" : BIGINT - statscount(sales_city)
    + *       "cnt"        : BIGINT - statscount(cnt)
    + *    "nonnullstatcount" : MAP
    + *       "region_id"  : BIGINT - nonnullstatcount(region_id)
    + *       "sales_city" : BIGINT - nonnullstatcount(sales_city)
    + *       "cnt"        : BIGINT - nonnullstatcount(cnt)
    + *   .... another map for next stats function ....
    + */
    +public class StatisticsAggBatch extends StreamingAggBatch {
    +  private List<String> functions;
    +  private int schema = 0;
    +
    +  public StatisticsAggBatch(StatisticsAggregate popConfig, RecordBatch incoming,
    +      FragmentContext context) throws OutOfMemoryException {
    +    super(popConfig, incoming, context);
    +    this.functions = popConfig.getFunctions();
    +  }
    +
    +  private void createKeyColumn(String name, LogicalExpression expr, List<LogicalExpression> keyExprs,
    +      List<TypedFieldId> keyOutputIds) throws SchemaChangeException {
    +    ErrorCollector collector = new ErrorCollectorImpl();
    +
    +    LogicalExpression mle = ExpressionTreeMaterializer.materialize(expr, incoming, collector,
    +        context.getFunctionRegistry());
    +
    +    MaterializedField outputField = MaterializedField.create(name, mle.getMajorType());
    +    ValueVector vector = TypeHelper.getNewVector(outputField, oContext.getAllocator());
    +
    +    keyExprs.add(mle);
    +    keyOutputIds.add(container.add(vector));
    +
    +    if (collector.hasErrors()) {
    +      throw new SchemaChangeException("Failure while materializing expression. "
    +          + collector.toErrorString());
    +    }
    +  }
    +
    +  private void createNestedKeyColumn(MapVector parent, String name, LogicalExpression expr,
    +      List<LogicalExpression> keyExprs, List<TypedFieldId> keyOutputIds)
    +          throws SchemaChangeException {
    +    ErrorCollector collector = new ErrorCollectorImpl();
    +
    +    LogicalExpression mle = ExpressionTreeMaterializer.materialize(expr, incoming, collector,
    +        context.getFunctionRegistry());
    +
    +    Class<? extends ValueVector> vvc =
    +        TypeHelper.getValueVectorClass(mle.getMajorType().getMinorType(),
    +            mle.getMajorType().getMode());
    +
    +    ValueVector vv = parent.addOrGet(name, mle.getMajorType(), vvc);
    +
    +    TypedFieldId pfid = container.getValueVectorId(SchemaPath.getSimplePath(parent.getField().getPath()));
    +    assert pfid.getFieldIds().length == 1;
    +    TypedFieldId.Builder builder = TypedFieldId.newBuilder();
    +    builder.addId(pfid.getFieldIds()[0]);
    +    TypedFieldId id =
    +        FieldIdUtil.getFieldIdIfMatches(parent, builder, true,
    +            SchemaPath.getSimplePath(vv.getField().getPath()).getRootSegment());
    +
    +    keyExprs.add(mle);
    +    keyOutputIds.add(id);
    +
    +    if (collector.hasErrors()) {
    +      throw new SchemaChangeException("Failure while materializing expression. "
    +          + collector.toErrorString());
    +    }
    +  }
    +
    +  private void addMapVector(String name, MapVector parent, LogicalExpression expr,
    +      List<LogicalExpression> valueExprs) throws SchemaChangeException {
    +    ErrorCollector collector = new ErrorCollectorImpl();
    +
    +    LogicalExpression mle = ExpressionTreeMaterializer.materialize(expr, incoming, collector,
    +        context.getFunctionRegistry());
    +
    +    Class<? extends ValueVector> vvc =
    +        TypeHelper.getValueVectorClass(mle.getMajorType().getMinorType(), mle.getMajorType().getMode());
    +    ValueVector vv = parent.addOrGet(name, mle.getMajorType(), vvc);
    +
    +    TypedFieldId pfid = container.getValueVectorId(SchemaPath.getSimplePath(parent.getField().getPath()));
    +    assert pfid.getFieldIds().length == 1;
    +    TypedFieldId.Builder builder = TypedFieldId.newBuilder();
    +    builder.addId(pfid.getFieldIds()[0]);
    +    TypedFieldId id = FieldIdUtil.getFieldIdIfMatches(parent, builder, true,
    +        SchemaPath.getSimplePath(vv.getField().getPath()).getRootSegment());
    +
    +    valueExprs.add(new ValueVectorWriteExpression(id, mle, true));
    +
    +    if (collector.hasErrors()) {
    --- End diff --
    
    Same comment as above.


---
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 #729: Drill 1328: Support table statistics for Parquet

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/729#discussion_r100677812
  
    --- Diff: exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/statistics/StatisticsAggBatchCreator.java ---
    @@ -0,0 +1,40 @@
    +/**
    + * 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.physical.impl.statistics;
    +
    +import java.util.List;
    +
    +import org.apache.drill.common.exceptions.ExecutionSetupException;
    +import org.apache.drill.exec.ops.FragmentContext;
    +import org.apache.drill.exec.physical.config.StatisticsAggregate;
    +import org.apache.drill.exec.physical.impl.BatchCreator;
    +import org.apache.drill.exec.record.CloseableRecordBatch;
    +import org.apache.drill.exec.record.RecordBatch;
    +
    +import com.google.common.base.Preconditions;
    +
    +@SuppressWarnings("unused")
    --- End diff --
    
    Is it the class that is unused? If so, why create it? Or, is it some method or field? If so, please move the annotation to that specific item.


---
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 #729: Drill 1328: Support table statistics for Parquet

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

    https://github.com/apache/drill/pull/729#discussion_r103606914
  
    --- Diff: exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/statistics/NNStatCountMergedStatistic.java ---
    @@ -0,0 +1,98 @@
    +/*
    + * 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.impl.statistics;
    +
    +import org.apache.drill.common.types.TypeProtos;
    +import org.apache.drill.exec.expr.holders.BigIntHolder;
    +import org.apache.drill.exec.expr.holders.ValueHolder;
    +import org.apache.drill.exec.vector.NullableBigIntVector;
    +import org.apache.drill.exec.vector.ValueVector;
    +import org.apache.drill.exec.vector.complex.MapVector;
    +
    +import java.util.HashMap;
    +import java.util.Map;
    +
    +public class NNStatCountMergedStatistic extends AbstractMergedStatistic {
    +
    +  private String name;
    +  private String inputName;
    +  private boolean mergeComplete = false;
    +  private Map<String, ValueHolder> sumHolder;
    +
    +  public NNStatCountMergedStatistic (String name, String inputName) {
    +    this.name = name;
    +    this.inputName = inputName;
    +    this.sumHolder = new HashMap<>();
    +  }
    +
    +  @Override
    +  public String getName() {
    +      return name;
    +  }
    +
    +  @Override
    +  public String getInput() {
    +    return inputName;
    +  }
    +
    +  @Override
    +  public void merge(ValueVector input) {
    +    // Check the input is a Map Vector
    +    assert (input.getField().getType().getMinorType() == TypeProtos.MinorType.MAP);
    +    MapVector inputMap = (MapVector) input;
    +    for (ValueVector vv : inputMap) {
    +      String colName = vv.getField().getLastName();
    +      BigIntHolder colSumHolder;
    +      if (sumHolder.get(colName) != null) {
    +        colSumHolder = (BigIntHolder) sumHolder.get(colName);
    +      } else {
    +        colSumHolder = new BigIntHolder();
    +        sumHolder.put(colName, colSumHolder);
    +      }
    +      Object val = vv.getAccessor().getObject(0);
    +      if (val != null) {
    +        colSumHolder.value += (long) val;
    +      }
    +    }
    +  }
    +
    +  @Override
    +  public Object getStat(String colName) {
    +    if (mergeComplete != true) {
    +      throw new IllegalStateException(String.format("Statistic `%s` has not completed merging statistics",
    +          name));
    +    }
    +    BigIntHolder colSumHolder = (BigIntHolder) sumHolder.get(colName);
    +    return colSumHolder.value;
    +  }
    +
    +  @Override
    +  public void setOutput(ValueVector output) {
    +    // Check the input is a Map Vector
    +    assert (output.getField().getType().getMinorType() == TypeProtos.MinorType.MAP);
    +    MapVector outputMap = (MapVector) output;
    +    for (ValueVector outMapCol : outputMap) {
    +      String colName = outMapCol.getField().getLastName();
    +      BigIntHolder holder = (BigIntHolder) sumHolder.get(colName);
    +      NullableBigIntVector vv = (NullableBigIntVector) outMapCol;
    +      vv.allocateNewSafe();
    +      vv.getMutator().setSafe(0, holder);
    +    }
    +    mergeComplete = true;
    --- End diff --
    
    The value vector and the results map are all type specific. I do not see a way to factor it out to a common function. The only common portion is 2 lines - the `for` loop and getting the column name. Same goes for the `merge()` function. Hence, leaving as-is.


---
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 #729: Drill 1328: Support table statistics for Parquet

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

    https://github.com/apache/drill/pull/729#discussion_r102604228
  
    --- Diff: exec/java-exec/src/main/java/org/apache/drill/exec/ExecConstants.java ---
    @@ -390,4 +391,15 @@
     
       String DYNAMIC_UDF_SUPPORT_ENABLED = "exec.udf.enable_dynamic_support";
       BooleanValidator DYNAMIC_UDF_SUPPORT_ENABLED_VALIDATOR = new BooleanValidator(DYNAMIC_UDF_SUPPORT_ENABLED, true, true);
    +
    +  /**
    +   * Option whose value is a long value representing the number of bits required for computing ndv (using HLL)
    +   */
    +  LongValidator NDV_MEMORY_LIMIT = new PositiveLongValidator("exec.statistics.ndv_memory_limit", 30, 20);
    --- End diff --
    
    We are not mixing different lengths during the same run. The session setting at the foreman would be passed along in the plan fragment - so non-foreman fragments will use the same value. Also, we do not mix lengths across different runs. So this should not be an issue.


---
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 #729: Drill 1328 r4

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

    https://github.com/apache/drill/pull/729#discussion_r98077612
  
    --- Diff: exec/java-exec/src/main/java/org/apache/drill/exec/planner/physical/NestedLoopJoinPrule.java ---
    @@ -84,8 +88,14 @@ public void onMatch(RelOptRuleCall call) {
         if (!settings.isNestedLoopJoinEnabled()) {
           return;
         }
    -
    -    final DrillJoinRel join = (DrillJoinRel) call.rel(0);
    +    int[] joinFields = new int[2];
    +    DrillJoinRel join = (DrillJoinRel) call.rel(0);
    +    // If right outer join on simply equi join convert it to left outer join. We only support left outer NLJ as of now
    +    if (join.getJoinType() == JoinRelType.RIGHT
    --- End diff --
    
    Can you do this change as part of a separate JIRA since it is unrelated to statistics. 


---
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 #729: Drill 1328: Support table statistics for Parquet

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

    https://github.com/apache/drill/pull/729#discussion_r102291099
  
    --- Diff: exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/unpivot/UnpivotMapsRecordBatch.java ---
    @@ -0,0 +1,276 @@
    +/**
    + * 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.impl.unpivot;
    +
    +import java.util.List;
    +import java.util.Map;
    +
    +import org.apache.drill.common.types.TypeProtos.MajorType;
    +import org.apache.drill.common.types.TypeProtos.MinorType;
    +import org.apache.drill.exec.exception.OutOfMemoryException;
    +import org.apache.drill.exec.exception.SchemaChangeException;
    +import org.apache.drill.exec.expr.TypeHelper;
    +import org.apache.drill.exec.ops.FragmentContext;
    +import org.apache.drill.exec.physical.config.UnpivotMaps;
    +import org.apache.drill.exec.record.AbstractSingleRecordBatch;
    +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.VectorContainer;
    +import org.apache.drill.exec.record.VectorWrapper;
    +import org.apache.drill.exec.vector.ValueVector;
    +import org.apache.drill.exec.vector.complex.MapVector;
    +
    +import com.google.common.collect.Lists;
    +import com.google.common.collect.Maps;
    +
    +/**
    + * Unpivot maps. Assumptions are:
    + *  1) all child vectors in a map are of same type.
    + *  2) Each map contains the same number of fields and field names are also same (types could be different).
    + *
    + * Example input and output:
    + * Schema of input:
    + *    "schema"        : BIGINT - Schema number. For each schema change this number is incremented.
    + *    "computed"      : BIGINT - What time is it computed?
    + *    "columns" : MAP - Column names
    + *       "region_id"  : VARCHAR
    + *       "sales_city" : VARCHAR
    + *       "cnt"        : VARCHAR
    + *    "statscount" : MAP
    + *       "region_id"  : BIGINT - statscount(region_id) - aggregation over all values of region_id
    + *                      in incoming batch
    + *       "sales_city" : BIGINT - statscount(sales_city)
    + *       "cnt"        : BIGINT - statscount(cnt)
    + *    "nonnullstatcount" : MAP
    + *       "region_id"  : BIGINT - nonnullstatcount(region_id)
    + *       "sales_city" : BIGINT - nonnullstatcount(sales_city)
    + *       "cnt"        : BIGINT - nonnullstatcount(cnt)
    + *   .... another map for next stats function ....
    + *
    + * Schema of output:
    --- End diff --
    
    For now, we stick to the original reviewed design. We can revisit the design later, if required. 


---
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 #729: Drill 1328: Support table statistics for Parquet

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

    https://github.com/apache/drill/pull/729#discussion_r102290203
  
    --- Diff: exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/statistics/StatisticsMergeBatchCreator.java ---
    @@ -0,0 +1,39 @@
    +/**
    + * 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.physical.impl.statistics;
    +
    +import com.google.common.base.Preconditions;
    +import org.apache.drill.common.exceptions.ExecutionSetupException;
    +import org.apache.drill.exec.ops.FragmentContext;
    +import org.apache.drill.exec.physical.config.StatisticsMerge;
    +import org.apache.drill.exec.physical.impl.BatchCreator;
    +import org.apache.drill.exec.record.CloseableRecordBatch;
    +import org.apache.drill.exec.record.RecordBatch;
    +
    +import java.util.List;
    +
    +@SuppressWarnings("unused")
    --- End diff --
    
    Copy-paste artifact. Removed it.


---
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 #729: Drill 1328: Support table statistics for Parquet

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/729#discussion_r100680643
  
    --- Diff: exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/statistics/StatisticsMergeBatch.java ---
    @@ -0,0 +1,532 @@
    +/**
    + * 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.impl.statistics;
    +
    +import com.google.common.collect.Lists;
    +import com.google.common.collect.Maps;
    +import org.apache.drill.common.expression.LogicalExpression;
    +import org.apache.drill.common.expression.ErrorCollector;
    +import org.apache.drill.common.expression.ErrorCollectorImpl;
    +import org.apache.drill.common.expression.FunctionCallFactory;
    +import org.apache.drill.common.expression.SchemaPath;
    +import org.apache.drill.common.expression.ValueExpressions;
    +import org.apache.drill.common.types.TypeProtos;
    +import org.apache.drill.common.types.TypeProtos.MajorType;
    +import org.apache.drill.exec.exception.OutOfMemoryException;
    +import org.apache.drill.exec.exception.SchemaChangeException;
    +import org.apache.drill.exec.expr.ExpressionTreeMaterializer;
    +import org.apache.drill.exec.expr.TypeHelper;
    +import org.apache.drill.exec.expr.holders.BigIntHolder;
    +import org.apache.drill.exec.expr.holders.NullableFloat8Holder;
    +import org.apache.drill.exec.expr.holders.ObjectHolder;
    +import org.apache.drill.exec.expr.holders.ValueHolder;
    +import org.apache.drill.exec.ops.FragmentContext;
    +import org.apache.drill.exec.physical.config.StatisticsMerge;
    +import org.apache.drill.exec.record.AbstractSingleRecordBatch;
    +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.VectorContainer;
    +import org.apache.drill.exec.record.VectorWrapper;
    +import org.apache.drill.exec.vector.ValueVector;
    +import org.apache.drill.exec.vector.DateVector;
    +import org.apache.drill.exec.vector.BigIntVector;
    +import org.apache.drill.exec.vector.NullableBigIntVector;
    +import org.apache.drill.exec.vector.NullableFloat8Vector;
    +import org.apache.drill.exec.vector.NullableVarBinaryVector;
    +import org.apache.drill.exec.vector.VarCharVector;
    +import org.apache.drill.exec.vector.complex.MapVector;
    +
    +import java.io.ByteArrayInputStream;
    +import java.io.DataInputStream;
    +import java.io.IOException;
    +import java.util.List;
    +import java.util.GregorianCalendar;
    +import java.util.HashMap;
    +import java.util.Map;
    +import java.util.TimeZone;
    +
    +import com.clearspring.analytics.stream.cardinality.HyperLogLog;
    +
    +public class StatisticsMergeBatch extends AbstractSingleRecordBatch<StatisticsMerge> {
    +  private static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(StatisticsMergeBatch.class);
    +  private Map<String, String> functions;
    +  private boolean first = true;
    +  private boolean finished = false;
    +  private int schema = 0;
    +  private int recordCount = 0;
    +  private List<String> keyList = null;
    +  private Map<MaterializedField, ValueVector> dataSrcVecMap = null;
    +  // Map of non-map fields to VV in the incoming schema
    +  private Map<MaterializedField, ValueVector> copySrcVecMap = null;
    +  private Map<String, Map<String, ValueHolder>> aggregationMap = null;
    +  public StatisticsMergeBatch(StatisticsMerge popConfig, RecordBatch incoming,
    +                              FragmentContext context) throws OutOfMemoryException {
    +    super(popConfig, context, incoming);
    +    this.functions = new HashMap<>();
    +    this.aggregationMap = new HashMap<>();
    +
    +    /*for (String key : popConfig.getFunctions()) {
    +      aggregationMap.put(key, new HashMap<String, ValueHolder>());
    +      if (key.equalsIgnoreCase("statcount") || key.equalsIgnoreCase("nonnullstatcount")) {
    +        functions.put(key, "sum");
    +      } else if (key.equalsIgnoreCase("hll")) {
    +        functions.put(key, "hll_merge");
    +      } else if (key.equalsIgnoreCase("sum_width")) {
    +        functions.put(key, "avg_width");
    +      }
    +    }*/
    +    for (String key : popConfig.getFunctions()) {
    +      if (key.equalsIgnoreCase("sum_width")) {
    +        functions.put(key, "avg_width");
    +      } else if (key.equalsIgnoreCase("hll")) {
    +        functions.put(key, "hll_merge");
    +      } else {
    +        functions.put(key, key);
    +      }
    +      aggregationMap.put(functions.get(key), new HashMap<String, ValueHolder>());
    +    }
    +  }
    +
    +  private void createKeyColumn(String name, LogicalExpression expr, Map<MaterializedField, ValueVector> parentMap)
    +      throws SchemaChangeException {
    +    ErrorCollector collector = new ErrorCollectorImpl();
    +
    +    LogicalExpression mle = ExpressionTreeMaterializer.materialize(expr, incoming, collector,
    +        context.getFunctionRegistry());
    +
    +    MaterializedField outputField = MaterializedField.create(name, mle.getMajorType());
    +    ValueVector vector = TypeHelper.getNewVector(outputField, oContext.getAllocator());
    +    container.add(vector);
    +
    +    if (collector.hasErrors()) {
    +      throw new SchemaChangeException("Failure while materializing expression. "
    +          + collector.toErrorString());
    +    }
    +    parentMap.put(outputField, vector);
    +  }
    +
    +  private ValueVector addMapVector(String name, MapVector parent, LogicalExpression expr) throws SchemaChangeException {
    +    ErrorCollector collector = new ErrorCollectorImpl();
    +
    +    LogicalExpression mle = ExpressionTreeMaterializer.materialize(expr, incoming, collector,
    +        context.getFunctionRegistry());
    +
    +    Class<? extends ValueVector> vvc =
    +        TypeHelper.getValueVectorClass(mle.getMajorType().getMinorType(), mle.getMajorType().getMode());
    +    ValueVector vector = parent.addOrGet(name, mle.getMajorType(), vvc);
    +
    +    if (collector.hasErrors()) {
    +      throw new SchemaChangeException("Failure while materializing expression. "
    +          + collector.toErrorString());
    +    }
    +    return vector;
    +  }
    +
    +  /**
    +   * Identify the list of fields within a map which are unpivoted as columns in output
    +   */
    +  private void buildKeyList() {
    +    List<String> lastMapKeyList = null;
    +    for (VectorWrapper<?> vw : incoming) {
    +      if (vw.getField().getType().getMinorType() != TypeProtos.MinorType.MAP) {
    +        continue;
    +      }
    +
    +      keyList = Lists.newArrayList();
    +
    +      for (ValueVector vv : vw.getValueVector()) {
    +        keyList.add(vv.getField().getLastName());
    +      }
    +
    +      if (lastMapKeyList == null) {
    +        lastMapKeyList = keyList;
    +      } else {
    +        if (keyList.size() != lastMapKeyList.size() || !lastMapKeyList.containsAll(keyList)) {
    +          throw new UnsupportedOperationException("Maps have different fields");
    +        }
    +      }
    +    }
    +  }
    +
    +  private void buildOutputContainer() throws SchemaChangeException {
    +    dataSrcVecMap = Maps.newHashMap();
    +    copySrcVecMap = Maps.newHashMap();
    +    MajorType mt = null;
    +
    +    ErrorCollector collector = new ErrorCollectorImpl();
    +    GregorianCalendar calendar = new GregorianCalendar(TimeZone.getTimeZone("UTC"));
    +
    +    calendar.setTimeInMillis(System.currentTimeMillis());
    +    createKeyColumn("schema", ValueExpressions.getBigInt(schema++), copySrcVecMap);
    +    createKeyColumn("computed", ValueExpressions.getDate(calendar), copySrcVecMap);
    +
    +    for (VectorWrapper<?> vw : incoming) {
    +      addVectorToOutgoingContainer(vw.getField().getLastName(), vw, collector);
    +      /*MaterializedField ds = vw.getField();
    +      String field = vw.getField().getLastName();
    +      // Input map vector
    +      MapVector mapVector = (MapVector) vw.getValueVector();
    +      assert mapVector.getPrimitiveVectors().size() > 0;
    +      // Proceed to create output map vector with same name e.g. statcount etc.
    +      mt = mapVector.getField().getType();
    +      MaterializedField mf = MaterializedField.create(functions.get(field), mt);
    +      assert !dataSrcVecMap.containsKey(mf);
    +      ValueVector vector = TypeHelper.getNewVector(mf, oContext.getAllocator());
    +      container.add(vector);
    +      MapVector outputMapVector = (MapVector) vector;
    +
    +      for (ValueVector vv : mapVector) {
    +        String fieldName = vv.getField().getLastName();
    +        if (!keyList.contains(fieldName)) {
    +          throw new UnsupportedOperationException("Unpivot data vector " +
    +              ds + " contains key " + fieldName + " not contained in key source!");
    +        }
    +        if (vv.getField().getType().getMinorType() == TypeProtos.MinorType.MAP) {
    +          throw new UnsupportedOperationException("Unpivot of nested map is not supported!");
    +        }
    +        if (field.equals("column")) {
    +          outputMapVector.addOrGet(fieldName, vv.getField().getType(), vv.getClass());
    +        } else {
    +          List<LogicalExpression> args = Lists.newArrayList();
    +          //TODO: Something else to access value of col such as emp_id?
    +          args.add(SchemaPath.getSimplePath(vv.getField().getPath()));
    +          //TODO: Put in the mapVector
    +          LogicalExpression call = FunctionCallFactory.createExpression(functions.get(field), args);
    +          //TODO: Is this sufficient to add to new Map?
    +          ValueVector vector1 = addMapVector(fieldName, outputMapVector, call);
    +          if (collector.hasErrors()) {
    +            throw new SchemaChangeException("Failure while materializing expression. "
    +                + collector.toErrorString());
    +          }
    +        }
    +      }
    +      dataSrcVecMap.put(ds, outputMapVector);*/
    +    }
    +    //Now create NDV in the outgoing container which was not avaliable in the incoming
    +    for (VectorWrapper<?> vw : incoming) {
    +      if (vw.getField().getLastName().equalsIgnoreCase("sum_width")) {//NullableFloat8 type
    +        addVectorToOutgoingContainer("ndv", vw, collector);
    +        break;
    +      }
    +    }
    +    container.setRecordCount(0);
    +    recordCount = 0;
    +    container.buildSchema(incoming.getSchema().getSelectionVectorMode());
    +  }
    +
    +  private void addVectorToOutgoingContainer(String field, VectorWrapper vw, ErrorCollector collector)
    +     throws SchemaChangeException {
    +    // Input map vector
    +    MapVector mapVector = (MapVector) vw.getValueVector();
    +    MaterializedField mf;
    +    assert mapVector.getPrimitiveVectors().size() > 0;
    +    // Proceed to create output map vector with same name e.g. statcount etc.
    +    MajorType mt = mapVector.getField().getType();
    +    if (functions.get(field) != null) {
    +      mf = MaterializedField.create(functions.get(field), mt);
    +    } else {
    +      mf = MaterializedField.create(field, mt);
    +    }
    +    assert !dataSrcVecMap.containsKey(mf);
    +    ValueVector vector = TypeHelper.getNewVector(mf, oContext.getAllocator());
    +    container.add(vector);
    +    MapVector outputMapVector = (MapVector) vector;
    +
    +    for (ValueVector vv : mapVector) {
    +      String fieldName = vv.getField().getLastName();
    +      if (!keyList.contains(fieldName)) {
    +        throw new UnsupportedOperationException("Unpivot data vector " +
    +                field + " contains key " + fieldName + " not contained in key source!");
    +      }
    +      if (vv.getField().getType().getMinorType() == TypeProtos.MinorType.MAP) {
    +        throw new UnsupportedOperationException("Unpivot of nested map is not supported!");
    +      }
    +      if (field.equals("column")) {
    +        outputMapVector.addOrGet(fieldName, vv.getField().getType(), vv.getClass());
    +      } else {
    +        List<LogicalExpression> args = Lists.newArrayList();
    +        LogicalExpression call;
    +        //TODO: Something else to access value of col such as emp_id?
    +        args.add(SchemaPath.getSimplePath(vv.getField().getPath()));
    +        //TODO: Put in the mapVector
    +        if (functions.get(field) != null) {
    +          call = FunctionCallFactory.createExpression(functions.get(field), args);
    +        } else {
    +          call = FunctionCallFactory.createExpression(field, args);
    +        }
    +        //TODO: Is this sufficient to add to new Map?
    +        ValueVector vector1 = addMapVector(fieldName, outputMapVector, call);
    +        if (collector.hasErrors()) {
    +          throw new SchemaChangeException("Failure while materializing expression. "
    +                  + collector.toErrorString());
    +        }
    +      }
    +    }
    +  }
    +
    +  @Override
    +  protected boolean setupNewSchema() throws SchemaChangeException {
    +    container.clear();
    +    buildKeyList();
    +    buildOutputContainer();
    +    return true;
    +  }
    +
    +  @Override
    +  protected IterOutcome doWork() {
    +    int outRecordCount = incoming.getRecordCount();
    +    HashMap<String, Long> nonNullRows = new HashMap<>();
    +
    +    for (VectorWrapper<?> vw : incoming) {
    +      ValueVector vv = vw.getValueVector();
    +      if (vv.getField().getType().getMinorType() != TypeProtos.MinorType.MAP) {
    +        //We only expect Maps in the incoming. STOP, if this is not the case.
    +        return IterOutcome.STOP;
    --- End diff --
    
    This will fail with no good information. Throw an `IllegalStateException` or `UserException` instead.


---
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 #729: Drill 1328: Support table statistics for Parquet

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

    https://github.com/apache/drill/pull/729#discussion_r102602855
  
    --- Diff: exec/java-exec/src/main/codegen/data/Parser.tdd ---
    @@ -39,7 +39,13 @@
         "METADATA",
         "DATABASE",
         "IF",
    -    "JAR"
    +    "JAR",
    +    "ANALYZE",
    +    "COMPUTE",
    +    "ESTIMATE",
    +    "STATISTICS",
    +    "SAMPLE",
    +    "PERCENT"
    --- End diff --
    
    @sudheeshkatkam mentioned
    > Something like this came up before where a list of non reserved keyword might result in some ambiguous queries. See DRILL-2116. Also DRILL-3875.
    
    Hence, these keywords were not added to the non-reserved keyword list. Also, I am not sure how we can preserve backward compatibility 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 #729: Drill 1328: Support table statistics for Parquet

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/729#discussion_r100680994
  
    --- Diff: exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/statistics/StatisticsMergeBatch.java ---
    @@ -0,0 +1,532 @@
    +/**
    + * 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.impl.statistics;
    +
    +import com.google.common.collect.Lists;
    +import com.google.common.collect.Maps;
    +import org.apache.drill.common.expression.LogicalExpression;
    +import org.apache.drill.common.expression.ErrorCollector;
    +import org.apache.drill.common.expression.ErrorCollectorImpl;
    +import org.apache.drill.common.expression.FunctionCallFactory;
    +import org.apache.drill.common.expression.SchemaPath;
    +import org.apache.drill.common.expression.ValueExpressions;
    +import org.apache.drill.common.types.TypeProtos;
    +import org.apache.drill.common.types.TypeProtos.MajorType;
    +import org.apache.drill.exec.exception.OutOfMemoryException;
    +import org.apache.drill.exec.exception.SchemaChangeException;
    +import org.apache.drill.exec.expr.ExpressionTreeMaterializer;
    +import org.apache.drill.exec.expr.TypeHelper;
    +import org.apache.drill.exec.expr.holders.BigIntHolder;
    +import org.apache.drill.exec.expr.holders.NullableFloat8Holder;
    +import org.apache.drill.exec.expr.holders.ObjectHolder;
    +import org.apache.drill.exec.expr.holders.ValueHolder;
    +import org.apache.drill.exec.ops.FragmentContext;
    +import org.apache.drill.exec.physical.config.StatisticsMerge;
    +import org.apache.drill.exec.record.AbstractSingleRecordBatch;
    +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.VectorContainer;
    +import org.apache.drill.exec.record.VectorWrapper;
    +import org.apache.drill.exec.vector.ValueVector;
    +import org.apache.drill.exec.vector.DateVector;
    +import org.apache.drill.exec.vector.BigIntVector;
    +import org.apache.drill.exec.vector.NullableBigIntVector;
    +import org.apache.drill.exec.vector.NullableFloat8Vector;
    +import org.apache.drill.exec.vector.NullableVarBinaryVector;
    +import org.apache.drill.exec.vector.VarCharVector;
    +import org.apache.drill.exec.vector.complex.MapVector;
    +
    +import java.io.ByteArrayInputStream;
    +import java.io.DataInputStream;
    +import java.io.IOException;
    +import java.util.List;
    +import java.util.GregorianCalendar;
    +import java.util.HashMap;
    +import java.util.Map;
    +import java.util.TimeZone;
    +
    +import com.clearspring.analytics.stream.cardinality.HyperLogLog;
    +
    +public class StatisticsMergeBatch extends AbstractSingleRecordBatch<StatisticsMerge> {
    +  private static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(StatisticsMergeBatch.class);
    +  private Map<String, String> functions;
    +  private boolean first = true;
    +  private boolean finished = false;
    +  private int schema = 0;
    +  private int recordCount = 0;
    +  private List<String> keyList = null;
    +  private Map<MaterializedField, ValueVector> dataSrcVecMap = null;
    +  // Map of non-map fields to VV in the incoming schema
    +  private Map<MaterializedField, ValueVector> copySrcVecMap = null;
    +  private Map<String, Map<String, ValueHolder>> aggregationMap = null;
    +  public StatisticsMergeBatch(StatisticsMerge popConfig, RecordBatch incoming,
    +                              FragmentContext context) throws OutOfMemoryException {
    +    super(popConfig, context, incoming);
    +    this.functions = new HashMap<>();
    +    this.aggregationMap = new HashMap<>();
    +
    +    /*for (String key : popConfig.getFunctions()) {
    +      aggregationMap.put(key, new HashMap<String, ValueHolder>());
    +      if (key.equalsIgnoreCase("statcount") || key.equalsIgnoreCase("nonnullstatcount")) {
    +        functions.put(key, "sum");
    +      } else if (key.equalsIgnoreCase("hll")) {
    +        functions.put(key, "hll_merge");
    +      } else if (key.equalsIgnoreCase("sum_width")) {
    +        functions.put(key, "avg_width");
    +      }
    +    }*/
    +    for (String key : popConfig.getFunctions()) {
    +      if (key.equalsIgnoreCase("sum_width")) {
    +        functions.put(key, "avg_width");
    +      } else if (key.equalsIgnoreCase("hll")) {
    +        functions.put(key, "hll_merge");
    +      } else {
    +        functions.put(key, key);
    +      }
    +      aggregationMap.put(functions.get(key), new HashMap<String, ValueHolder>());
    +    }
    +  }
    +
    +  private void createKeyColumn(String name, LogicalExpression expr, Map<MaterializedField, ValueVector> parentMap)
    +      throws SchemaChangeException {
    +    ErrorCollector collector = new ErrorCollectorImpl();
    +
    +    LogicalExpression mle = ExpressionTreeMaterializer.materialize(expr, incoming, collector,
    +        context.getFunctionRegistry());
    +
    +    MaterializedField outputField = MaterializedField.create(name, mle.getMajorType());
    +    ValueVector vector = TypeHelper.getNewVector(outputField, oContext.getAllocator());
    +    container.add(vector);
    +
    +    if (collector.hasErrors()) {
    +      throw new SchemaChangeException("Failure while materializing expression. "
    +          + collector.toErrorString());
    +    }
    +    parentMap.put(outputField, vector);
    +  }
    +
    +  private ValueVector addMapVector(String name, MapVector parent, LogicalExpression expr) throws SchemaChangeException {
    +    ErrorCollector collector = new ErrorCollectorImpl();
    +
    +    LogicalExpression mle = ExpressionTreeMaterializer.materialize(expr, incoming, collector,
    +        context.getFunctionRegistry());
    +
    +    Class<? extends ValueVector> vvc =
    +        TypeHelper.getValueVectorClass(mle.getMajorType().getMinorType(), mle.getMajorType().getMode());
    +    ValueVector vector = parent.addOrGet(name, mle.getMajorType(), vvc);
    +
    +    if (collector.hasErrors()) {
    +      throw new SchemaChangeException("Failure while materializing expression. "
    +          + collector.toErrorString());
    +    }
    +    return vector;
    +  }
    +
    +  /**
    +   * Identify the list of fields within a map which are unpivoted as columns in output
    +   */
    +  private void buildKeyList() {
    +    List<String> lastMapKeyList = null;
    +    for (VectorWrapper<?> vw : incoming) {
    +      if (vw.getField().getType().getMinorType() != TypeProtos.MinorType.MAP) {
    +        continue;
    +      }
    +
    +      keyList = Lists.newArrayList();
    +
    +      for (ValueVector vv : vw.getValueVector()) {
    +        keyList.add(vv.getField().getLastName());
    +      }
    +
    +      if (lastMapKeyList == null) {
    +        lastMapKeyList = keyList;
    +      } else {
    +        if (keyList.size() != lastMapKeyList.size() || !lastMapKeyList.containsAll(keyList)) {
    +          throw new UnsupportedOperationException("Maps have different fields");
    +        }
    +      }
    +    }
    +  }
    +
    +  private void buildOutputContainer() throws SchemaChangeException {
    +    dataSrcVecMap = Maps.newHashMap();
    +    copySrcVecMap = Maps.newHashMap();
    +    MajorType mt = null;
    +
    +    ErrorCollector collector = new ErrorCollectorImpl();
    +    GregorianCalendar calendar = new GregorianCalendar(TimeZone.getTimeZone("UTC"));
    +
    +    calendar.setTimeInMillis(System.currentTimeMillis());
    +    createKeyColumn("schema", ValueExpressions.getBigInt(schema++), copySrcVecMap);
    +    createKeyColumn("computed", ValueExpressions.getDate(calendar), copySrcVecMap);
    +
    +    for (VectorWrapper<?> vw : incoming) {
    +      addVectorToOutgoingContainer(vw.getField().getLastName(), vw, collector);
    +      /*MaterializedField ds = vw.getField();
    +      String field = vw.getField().getLastName();
    +      // Input map vector
    +      MapVector mapVector = (MapVector) vw.getValueVector();
    +      assert mapVector.getPrimitiveVectors().size() > 0;
    +      // Proceed to create output map vector with same name e.g. statcount etc.
    +      mt = mapVector.getField().getType();
    +      MaterializedField mf = MaterializedField.create(functions.get(field), mt);
    +      assert !dataSrcVecMap.containsKey(mf);
    +      ValueVector vector = TypeHelper.getNewVector(mf, oContext.getAllocator());
    +      container.add(vector);
    +      MapVector outputMapVector = (MapVector) vector;
    +
    +      for (ValueVector vv : mapVector) {
    +        String fieldName = vv.getField().getLastName();
    +        if (!keyList.contains(fieldName)) {
    +          throw new UnsupportedOperationException("Unpivot data vector " +
    +              ds + " contains key " + fieldName + " not contained in key source!");
    +        }
    +        if (vv.getField().getType().getMinorType() == TypeProtos.MinorType.MAP) {
    +          throw new UnsupportedOperationException("Unpivot of nested map is not supported!");
    +        }
    +        if (field.equals("column")) {
    +          outputMapVector.addOrGet(fieldName, vv.getField().getType(), vv.getClass());
    +        } else {
    +          List<LogicalExpression> args = Lists.newArrayList();
    +          //TODO: Something else to access value of col such as emp_id?
    +          args.add(SchemaPath.getSimplePath(vv.getField().getPath()));
    +          //TODO: Put in the mapVector
    +          LogicalExpression call = FunctionCallFactory.createExpression(functions.get(field), args);
    +          //TODO: Is this sufficient to add to new Map?
    +          ValueVector vector1 = addMapVector(fieldName, outputMapVector, call);
    +          if (collector.hasErrors()) {
    +            throw new SchemaChangeException("Failure while materializing expression. "
    +                + collector.toErrorString());
    +          }
    +        }
    +      }
    +      dataSrcVecMap.put(ds, outputMapVector);*/
    +    }
    +    //Now create NDV in the outgoing container which was not avaliable in the incoming
    +    for (VectorWrapper<?> vw : incoming) {
    +      if (vw.getField().getLastName().equalsIgnoreCase("sum_width")) {//NullableFloat8 type
    +        addVectorToOutgoingContainer("ndv", vw, collector);
    +        break;
    +      }
    +    }
    +    container.setRecordCount(0);
    +    recordCount = 0;
    +    container.buildSchema(incoming.getSchema().getSelectionVectorMode());
    +  }
    +
    +  private void addVectorToOutgoingContainer(String field, VectorWrapper vw, ErrorCollector collector)
    +     throws SchemaChangeException {
    +    // Input map vector
    +    MapVector mapVector = (MapVector) vw.getValueVector();
    +    MaterializedField mf;
    +    assert mapVector.getPrimitiveVectors().size() > 0;
    +    // Proceed to create output map vector with same name e.g. statcount etc.
    +    MajorType mt = mapVector.getField().getType();
    +    if (functions.get(field) != null) {
    +      mf = MaterializedField.create(functions.get(field), mt);
    +    } else {
    +      mf = MaterializedField.create(field, mt);
    +    }
    +    assert !dataSrcVecMap.containsKey(mf);
    +    ValueVector vector = TypeHelper.getNewVector(mf, oContext.getAllocator());
    +    container.add(vector);
    +    MapVector outputMapVector = (MapVector) vector;
    +
    +    for (ValueVector vv : mapVector) {
    +      String fieldName = vv.getField().getLastName();
    +      if (!keyList.contains(fieldName)) {
    +        throw new UnsupportedOperationException("Unpivot data vector " +
    +                field + " contains key " + fieldName + " not contained in key source!");
    +      }
    +      if (vv.getField().getType().getMinorType() == TypeProtos.MinorType.MAP) {
    +        throw new UnsupportedOperationException("Unpivot of nested map is not supported!");
    +      }
    +      if (field.equals("column")) {
    +        outputMapVector.addOrGet(fieldName, vv.getField().getType(), vv.getClass());
    +      } else {
    +        List<LogicalExpression> args = Lists.newArrayList();
    +        LogicalExpression call;
    +        //TODO: Something else to access value of col such as emp_id?
    +        args.add(SchemaPath.getSimplePath(vv.getField().getPath()));
    +        //TODO: Put in the mapVector
    +        if (functions.get(field) != null) {
    +          call = FunctionCallFactory.createExpression(functions.get(field), args);
    +        } else {
    +          call = FunctionCallFactory.createExpression(field, args);
    +        }
    +        //TODO: Is this sufficient to add to new Map?
    +        ValueVector vector1 = addMapVector(fieldName, outputMapVector, call);
    +        if (collector.hasErrors()) {
    +          throw new SchemaChangeException("Failure while materializing expression. "
    +                  + collector.toErrorString());
    +        }
    +      }
    +    }
    +  }
    +
    +  @Override
    +  protected boolean setupNewSchema() throws SchemaChangeException {
    +    container.clear();
    +    buildKeyList();
    +    buildOutputContainer();
    +    return true;
    +  }
    +
    +  @Override
    +  protected IterOutcome doWork() {
    +    int outRecordCount = incoming.getRecordCount();
    +    HashMap<String, Long> nonNullRows = new HashMap<>();
    +
    +    for (VectorWrapper<?> vw : incoming) {
    +      ValueVector vv = vw.getValueVector();
    +      if (vv.getField().getType().getMinorType() != TypeProtos.MinorType.MAP) {
    +        //We only expect Maps in the incoming. STOP, if this is not the case.
    +        return IterOutcome.STOP;
    +      }
    +      MapVector mapVec = (MapVector) vv;
    +      Map<String, ValueHolder> statMap = aggregationMap.get(vv.getField().getLastName());
    +
    +      for (ValueVector mapElt : mapVec) {
    +        if (vv.getField().getLastName().equalsIgnoreCase("column")) {
    +          BigIntHolder nameHolder;
    +          if (statMap.get(mapElt.getField().getLastName()) != null) {
    +            nameHolder = (BigIntHolder) statMap.get(mapElt.getField().getLastName());
    +          } else {
    +            nameHolder = new BigIntHolder();
    +            statMap.put(mapElt.getField().getLastName(), nameHolder);
    +          }
    +          nameHolder.value = 1;
    +        } else if (vv.getField().getLastName().equalsIgnoreCase("statcount")
    +            || vv.getField().getLastName().equalsIgnoreCase("nonnullstatcount")) {
    +          BigIntHolder sumHolder;
    +          String colName = mapElt.getField().getLastName();
    +          if (statMap.get(colName) != null) {
    +            sumHolder = (BigIntHolder) statMap.get(colName);
    +          } else {
    +            sumHolder = new BigIntHolder();
    +            statMap.put(colName, sumHolder);
    +          }
    +          //TODO: assert size = 1
    +          //TODO: logger
    +          Object val = mapElt.getAccessor().getObject(0);
    +          if (val != null) {
    +            sumHolder.value += (long)val;
    +          }
    +        } else if (vv.getField().getLastName().equalsIgnoreCase("sum_width")) {
    +          NullableFloat8Holder sumHolder;
    +          String colName = mapElt.getField().getLastName();
    +          if (statMap == null) {
    +            statMap = aggregationMap.get(functions.get(vv.getField().getLastName()));
    +          }
    +          if (statMap.get(colName) != null) {
    +            sumHolder = (NullableFloat8Holder) statMap.get(colName);
    +          } else {
    +            sumHolder = new NullableFloat8Holder();
    +            statMap.put(colName, sumHolder);
    +          }
    +          //TODO: assert size = 1
    +          //TODO: logger
    +          Object val = mapElt.getAccessor().getObject(0);
    +          if (val != null) {
    +            sumHolder.value += (double) val;
    +            sumHolder.isSet = 1;
    +          }
    +        } else if (vv.getField().getLastName().equalsIgnoreCase("hll")) {
    +          ObjectHolder hllHolder;
    +          String colName = mapElt.getField().getLastName();
    +          if (statMap == null) {
    +            statMap = aggregationMap.get(functions.get(vv.getField().getLastName()));
    +          }
    +          if (statMap.get(colName) != null) {
    +            hllHolder = (ObjectHolder) statMap.get(colName);
    +          } else {
    +            hllHolder = new ObjectHolder();
    +            hllHolder.obj = new HyperLogLog(context.getContextInformation().getHllAccuracy());
    +            statMap.put(colName, hllHolder);
    +          }
    +          NullableVarBinaryVector hllVector = (NullableVarBinaryVector) mapElt;
    +          try {
    +            if (hllVector.getAccessor().isSet(0) == 1) {
    +              ByteArrayInputStream bais = new ByteArrayInputStream(hllVector.getAccessor().getObject(0), 0,
    +                  mapElt.getBufferSize());
    +              HyperLogLog other = HyperLogLog.Builder.build(new DataInputStream(bais));
    +              ((HyperLogLog) hllHolder.obj).addAll(other);
    +            }
    +          } catch (Exception ex) {
    +            //TODO: Catch IOException/CardinalityMergeException
    +            //TODO: logger
    +            return IterOutcome.STOP;
    +          }
    +        }
    +      }
    +      // Add NDV value vector map using HLL map (since the NDV map is directly generated from HLL and not produced by the underlying
    +      // Statistics Agg)
    +      Map<String, ValueHolder> hllMap = aggregationMap.get("hll");
    +      if (hllMap != null) {
    +        aggregationMap.put("ndv", hllMap);
    +      }
    +    }
    +    return IterOutcome.OK;
    +  }
    +
    +  public VectorContainer getOutgoingContainer() {
    +    return this.container;
    +  }
    +
    +  @Override
    +  public IterOutcome innerNext() {
    +    IterOutcome outcome;
    +    boolean didSomeWork = false;
    +    if (finished) {
    +      return IterOutcome.NONE;
    +    }
    +    try {
    +      outer: while (true) {
    +        outcome = next(incoming);
    +        switch (outcome) {
    +          case NONE:
    +            break outer;
    +          case OUT_OF_MEMORY:
    +          case NOT_YET:
    +          case STOP:
    +            return outcome;
    +          case OK_NEW_SCHEMA:
    --- End diff --
    
    Confused. The schema of the underlying data source might change. Does that then trigger a schema change for the stats? I suppose it must, if a new field appears in the original data, we add a field to the maps, which triggers a schema change.
    
    But, should this occur? Do we need a schema change for stats when the underlying schema changes? (See comment about a list-based schema earlier.)


---
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 #729: Drill 1328: Support table statistics for Parquet

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/729#discussion_r100680446
  
    --- Diff: exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/statistics/StatisticsMergeBatch.java ---
    @@ -0,0 +1,532 @@
    +/**
    + * 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.impl.statistics;
    +
    +import com.google.common.collect.Lists;
    +import com.google.common.collect.Maps;
    +import org.apache.drill.common.expression.LogicalExpression;
    +import org.apache.drill.common.expression.ErrorCollector;
    +import org.apache.drill.common.expression.ErrorCollectorImpl;
    +import org.apache.drill.common.expression.FunctionCallFactory;
    +import org.apache.drill.common.expression.SchemaPath;
    +import org.apache.drill.common.expression.ValueExpressions;
    +import org.apache.drill.common.types.TypeProtos;
    +import org.apache.drill.common.types.TypeProtos.MajorType;
    +import org.apache.drill.exec.exception.OutOfMemoryException;
    +import org.apache.drill.exec.exception.SchemaChangeException;
    +import org.apache.drill.exec.expr.ExpressionTreeMaterializer;
    +import org.apache.drill.exec.expr.TypeHelper;
    +import org.apache.drill.exec.expr.holders.BigIntHolder;
    +import org.apache.drill.exec.expr.holders.NullableFloat8Holder;
    +import org.apache.drill.exec.expr.holders.ObjectHolder;
    +import org.apache.drill.exec.expr.holders.ValueHolder;
    +import org.apache.drill.exec.ops.FragmentContext;
    +import org.apache.drill.exec.physical.config.StatisticsMerge;
    +import org.apache.drill.exec.record.AbstractSingleRecordBatch;
    +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.VectorContainer;
    +import org.apache.drill.exec.record.VectorWrapper;
    +import org.apache.drill.exec.vector.ValueVector;
    +import org.apache.drill.exec.vector.DateVector;
    +import org.apache.drill.exec.vector.BigIntVector;
    +import org.apache.drill.exec.vector.NullableBigIntVector;
    +import org.apache.drill.exec.vector.NullableFloat8Vector;
    +import org.apache.drill.exec.vector.NullableVarBinaryVector;
    +import org.apache.drill.exec.vector.VarCharVector;
    +import org.apache.drill.exec.vector.complex.MapVector;
    +
    +import java.io.ByteArrayInputStream;
    +import java.io.DataInputStream;
    +import java.io.IOException;
    +import java.util.List;
    +import java.util.GregorianCalendar;
    +import java.util.HashMap;
    +import java.util.Map;
    +import java.util.TimeZone;
    +
    +import com.clearspring.analytics.stream.cardinality.HyperLogLog;
    +
    --- End diff --
    
    Perhaps a brief comment to explain what this is, what it does, and how data flows through it?


---
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 #729: Drill 1328 r4

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/729#discussion_r99392948
  
    --- Diff: exec/java-exec/src/main/java/org/apache/drill/exec/dotdrill/DotDrillUtil.java ---
    @@ -30,6 +32,9 @@
       static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(DotDrillUtil.class);
     
       private static List<DotDrillFile> getDrillFiles(DrillFileSystem fs, FileStatus[] statuses, DotDrillType... types){
    +    if (statuses == null) {
    +      return Collections.emptyList();
    +    }
    --- End diff --
    
    Is this a legal case? Should this, instead, be:
    
    ```
    if (statuses == null) { throw new IllegalArgumentException( ... ); }
    ```
    
    (Or the `Preconditions` equivalent...)


---
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 #729: Drill 1328: Support table statistics for Parquet

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/729#discussion_r100677774
  
    --- Diff: exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/statistics/StatisticsAggBatch.java ---
    @@ -0,0 +1,256 @@
    +/**
    + * 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.impl.statistics;
    +
    +import com.google.common.collect.Lists;
    +import com.sun.codemodel.JExpr;
    +import org.apache.drill.common.expression.ErrorCollector;
    +import org.apache.drill.common.expression.ErrorCollectorImpl;
    +import org.apache.drill.common.expression.FunctionCallFactory;
    +import org.apache.drill.common.expression.LogicalExpression;
    +import org.apache.drill.common.expression.SchemaPath;
    +import org.apache.drill.common.expression.ValueExpressions;
    +import org.apache.drill.exec.exception.ClassTransformationException;
    +import org.apache.drill.exec.exception.OutOfMemoryException;
    +import org.apache.drill.exec.exception.SchemaChangeException;
    +import org.apache.drill.exec.expr.ClassGenerator;
    +import org.apache.drill.exec.expr.CodeGenerator;
    +import org.apache.drill.exec.expr.ExpressionTreeMaterializer;
    +import org.apache.drill.exec.expr.TypeHelper;
    +import org.apache.drill.exec.expr.ValueVectorWriteExpression;
    +import org.apache.drill.exec.ops.FragmentContext;
    +import org.apache.drill.exec.physical.config.StatisticsAggregate;
    +import org.apache.drill.exec.physical.impl.aggregate.StreamingAggBatch;
    +import org.apache.drill.exec.physical.impl.aggregate.StreamingAggTemplate;
    +import org.apache.drill.exec.physical.impl.aggregate.StreamingAggregator;
    +import org.apache.drill.exec.planner.physical.StatsAggPrel.OperatorPhase;
    +import org.apache.drill.exec.record.BatchSchema.SelectionVectorMode;
    +import org.apache.drill.exec.record.MaterializedField;
    +import org.apache.drill.exec.record.RecordBatch;
    +import org.apache.drill.exec.record.TypedFieldId;
    +import org.apache.drill.exec.store.ImplicitColumnExplorer;
    +import org.apache.drill.exec.vector.ValueVector;
    +import org.apache.drill.exec.vector.complex.FieldIdUtil;
    +import org.apache.drill.exec.vector.complex.MapVector;
    +
    +import java.io.IOException;
    +import java.util.GregorianCalendar;
    +import java.util.List;
    +import java.util.TimeZone;
    +
    +/**
    + * TODO: This needs cleanup. Currently the key values are constants and we compare the constants
    + * for every record. Seems unnecessary.
    + *
    + * Example input and output:
    + * Schema of incoming batch: region_id (VARCHAR), sales_city (VARCHAR), cnt (BIGINT)
    + * Schema of output:
    + *    "schema" : BIGINT - Schema number. For each schema change this number is incremented.
    + *    "computed" : BIGINT - What time is it computed?
    + *    "columns"       : MAP - Column names
    + *       "region_id"  : VARCHAR
    + *       "sales_city" : VARCHAR
    + *       "cnt"        : VARCHAR
    + *    "statscount" : MAP
    + *       "region_id"  : BIGINT - statscount(region_id) - aggregation over all values of region_id
    + *                      in incoming batch
    + *       "sales_city" : BIGINT - statscount(sales_city)
    + *       "cnt"        : BIGINT - statscount(cnt)
    + *    "nonnullstatcount" : MAP
    + *       "region_id"  : BIGINT - nonnullstatcount(region_id)
    + *       "sales_city" : BIGINT - nonnullstatcount(sales_city)
    + *       "cnt"        : BIGINT - nonnullstatcount(cnt)
    + *   .... another map for next stats function ....
    + */
    +public class StatisticsAggBatch extends StreamingAggBatch {
    +  private List<String> functions;
    +  private int schema = 0;
    +
    +  public StatisticsAggBatch(StatisticsAggregate popConfig, RecordBatch incoming,
    +      FragmentContext context) throws OutOfMemoryException {
    +    super(popConfig, incoming, context);
    +    this.functions = popConfig.getFunctions();
    +  }
    +
    +  private void createKeyColumn(String name, LogicalExpression expr, List<LogicalExpression> keyExprs,
    +      List<TypedFieldId> keyOutputIds) throws SchemaChangeException {
    +    ErrorCollector collector = new ErrorCollectorImpl();
    +
    +    LogicalExpression mle = ExpressionTreeMaterializer.materialize(expr, incoming, collector,
    +        context.getFunctionRegistry());
    +
    +    MaterializedField outputField = MaterializedField.create(name, mle.getMajorType());
    +    ValueVector vector = TypeHelper.getNewVector(outputField, oContext.getAllocator());
    +
    +    keyExprs.add(mle);
    +    keyOutputIds.add(container.add(vector));
    +
    +    if (collector.hasErrors()) {
    +      throw new SchemaChangeException("Failure while materializing expression. "
    +          + collector.toErrorString());
    +    }
    +  }
    +
    +  private void createNestedKeyColumn(MapVector parent, String name, LogicalExpression expr,
    +      List<LogicalExpression> keyExprs, List<TypedFieldId> keyOutputIds)
    +          throws SchemaChangeException {
    +    ErrorCollector collector = new ErrorCollectorImpl();
    +
    +    LogicalExpression mle = ExpressionTreeMaterializer.materialize(expr, incoming, collector,
    +        context.getFunctionRegistry());
    +
    +    Class<? extends ValueVector> vvc =
    +        TypeHelper.getValueVectorClass(mle.getMajorType().getMinorType(),
    +            mle.getMajorType().getMode());
    +
    +    ValueVector vv = parent.addOrGet(name, mle.getMajorType(), vvc);
    +
    +    TypedFieldId pfid = container.getValueVectorId(SchemaPath.getSimplePath(parent.getField().getPath()));
    +    assert pfid.getFieldIds().length == 1;
    +    TypedFieldId.Builder builder = TypedFieldId.newBuilder();
    +    builder.addId(pfid.getFieldIds()[0]);
    +    TypedFieldId id =
    +        FieldIdUtil.getFieldIdIfMatches(parent, builder, true,
    +            SchemaPath.getSimplePath(vv.getField().getPath()).getRootSegment());
    +
    +    keyExprs.add(mle);
    +    keyOutputIds.add(id);
    +
    +    if (collector.hasErrors()) {
    +      throw new SchemaChangeException("Failure while materializing expression. "
    +          + collector.toErrorString());
    +    }
    +  }
    +
    +  private void addMapVector(String name, MapVector parent, LogicalExpression expr,
    +      List<LogicalExpression> valueExprs) throws SchemaChangeException {
    +    ErrorCollector collector = new ErrorCollectorImpl();
    +
    +    LogicalExpression mle = ExpressionTreeMaterializer.materialize(expr, incoming, collector,
    +        context.getFunctionRegistry());
    +
    +    Class<? extends ValueVector> vvc =
    +        TypeHelper.getValueVectorClass(mle.getMajorType().getMinorType(), mle.getMajorType().getMode());
    +    ValueVector vv = parent.addOrGet(name, mle.getMajorType(), vvc);
    +
    +    TypedFieldId pfid = container.getValueVectorId(SchemaPath.getSimplePath(parent.getField().getPath()));
    +    assert pfid.getFieldIds().length == 1;
    +    TypedFieldId.Builder builder = TypedFieldId.newBuilder();
    +    builder.addId(pfid.getFieldIds()[0]);
    +    TypedFieldId id = FieldIdUtil.getFieldIdIfMatches(parent, builder, true,
    +        SchemaPath.getSimplePath(vv.getField().getPath()).getRootSegment());
    +
    +    valueExprs.add(new ValueVectorWriteExpression(id, mle, true));
    +
    +    if (collector.hasErrors()) {
    +      throw new SchemaChangeException("Failure while materializing expression. "
    +          + collector.toErrorString());
    +    }
    +  }
    +
    +  private StreamingAggregator codegenAggregator(List<LogicalExpression> keyExprs,
    +      List<LogicalExpression> valueExprs, List<TypedFieldId> keyOutputIds)
    +          throws SchemaChangeException, ClassTransformationException, IOException {
    +    ClassGenerator<StreamingAggregator> cg =
    +        CodeGenerator.getRoot(StreamingAggTemplate.TEMPLATE_DEFINITION, context.getFunctionRegistry(),
    +            context.getOptions());
    +
    +    LogicalExpression[] keyExprsArray = new LogicalExpression[keyExprs.size()];
    +    LogicalExpression[] valueExprsArray = new LogicalExpression[valueExprs.size()];
    +    TypedFieldId[] keyOutputIdsArray = new TypedFieldId[keyOutputIds.size()];
    +
    +    keyExprs.toArray(keyExprsArray);
    +    valueExprs.toArray(valueExprsArray);
    +    keyOutputIds.toArray(keyOutputIdsArray);
    +
    +    setupIsSame(cg, keyExprsArray);
    +    setupIsSameApart(cg, keyExprsArray);
    +    addRecordValues(cg, valueExprsArray);
    +    outputRecordKeys(cg, keyOutputIdsArray, keyExprsArray);
    +    outputRecordKeysPrev(cg, keyOutputIdsArray, keyExprsArray);
    +
    +    cg.getBlock("resetValues")._return(JExpr.TRUE);
    +    getIndex(cg);
    +
    +    container.buildSchema(SelectionVectorMode.NONE);
    +    StreamingAggregator agg = context.getImplementationClass(cg);
    +    agg.setup(oContext, incoming, this);
    +    return agg;
    +  }
    +
    +  private boolean isImplicitFileColumn(MaterializedField mf) {
    +    return ImplicitColumnExplorer.initImplicitFileColumns(context.getOptions()).get(mf.getName()) != null;
    +  }
    +
    +  protected StreamingAggregator createAggregatorInternal()
    --- End diff --
    
    The method is "create aggregator internal" which sounds like it creates an aggregator (generates a class.) At the end of this method, we do, in fact, generate code.
    
    Should we make all this a bit clearer? Isn't all this stuff a per-schema setup? Or, are we trying to override an existing method? Probably not since no `@Override` on this method. Maybe rename "setupNewSchema" and break into parts?


---
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 #729: Drill 1328: Support table statistics for Parquet

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

    https://github.com/apache/drill/pull/729#discussion_r103609111
  
    --- Diff: exec/java-exec/src/main/java/org/apache/drill/exec/ops/ContextInformation.java ---
    @@ -63,4 +65,11 @@ public long getQueryStartTime() {
       public int getRootFragmentTimeZone() {
         return rootFragmentTimeZone;
       }
    +
    +  /**
    +   * @return HLL accuracy parameter
    +   */
    +  public int getHllMemoryLimit() {
    --- 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 #729: Drill 1328 r4

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/729#discussion_r99391697
  
    --- Diff: exec/java-exec/src/main/java/org/apache/drill/exec/ops/ContextInformation.java ---
    @@ -28,12 +28,14 @@
       private final String currentDefaultSchema;
       private final long queryStartTime;
       private final int rootFragmentTimeZone;
    +  private final int hllMemoryLimit;
     
       public ContextInformation(final UserCredentials userCredentials, final QueryContextInformation queryContextInfo) {
         this.queryUser = userCredentials.getUserName();
         this.currentDefaultSchema = queryContextInfo.getDefaultSchemaName();
         this.queryStartTime = queryContextInfo.getQueryStartTime();
         this.rootFragmentTimeZone = queryContextInfo.getTimeZone();
    +    this.hllMemoryLimit = queryContextInfo.getHllMemoryLimit();
    --- End diff --
    
    This is labeled as a memory limit here, but the description of the option speaks of bit counts. This is confusing. A memory limit is usually expressed in terms of byte (or MB) that some operator may consume. A bit count might give rise to a memory usage, but is not really a limit per-se...


---
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 #729: Drill 1328: Support table statistics for Parquet

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/729#discussion_r100681080
  
    --- Diff: exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/statistics/StatisticsMergeBatchCreator.java ---
    @@ -0,0 +1,39 @@
    +/**
    + * 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.physical.impl.statistics;
    +
    +import com.google.common.base.Preconditions;
    +import org.apache.drill.common.exceptions.ExecutionSetupException;
    +import org.apache.drill.exec.ops.FragmentContext;
    +import org.apache.drill.exec.physical.config.StatisticsMerge;
    +import org.apache.drill.exec.physical.impl.BatchCreator;
    +import org.apache.drill.exec.record.CloseableRecordBatch;
    +import org.apache.drill.exec.record.RecordBatch;
    +
    +import java.util.List;
    +
    +@SuppressWarnings("unused")
    --- End diff --
    
    Why is this class unused?


---
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 #729: Drill 1328: Support table statistics for Parquet

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

    https://github.com/apache/drill/pull/729#discussion_r102294356
  
    --- Diff: exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/statistics/StatisticsMergeBatch.java ---
    @@ -0,0 +1,532 @@
    +/**
    + * 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.impl.statistics;
    +
    +import com.google.common.collect.Lists;
    +import com.google.common.collect.Maps;
    +import org.apache.drill.common.expression.LogicalExpression;
    +import org.apache.drill.common.expression.ErrorCollector;
    +import org.apache.drill.common.expression.ErrorCollectorImpl;
    +import org.apache.drill.common.expression.FunctionCallFactory;
    +import org.apache.drill.common.expression.SchemaPath;
    +import org.apache.drill.common.expression.ValueExpressions;
    +import org.apache.drill.common.types.TypeProtos;
    +import org.apache.drill.common.types.TypeProtos.MajorType;
    +import org.apache.drill.exec.exception.OutOfMemoryException;
    +import org.apache.drill.exec.exception.SchemaChangeException;
    +import org.apache.drill.exec.expr.ExpressionTreeMaterializer;
    +import org.apache.drill.exec.expr.TypeHelper;
    +import org.apache.drill.exec.expr.holders.BigIntHolder;
    +import org.apache.drill.exec.expr.holders.NullableFloat8Holder;
    +import org.apache.drill.exec.expr.holders.ObjectHolder;
    +import org.apache.drill.exec.expr.holders.ValueHolder;
    +import org.apache.drill.exec.ops.FragmentContext;
    +import org.apache.drill.exec.physical.config.StatisticsMerge;
    +import org.apache.drill.exec.record.AbstractSingleRecordBatch;
    +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.VectorContainer;
    +import org.apache.drill.exec.record.VectorWrapper;
    +import org.apache.drill.exec.vector.ValueVector;
    +import org.apache.drill.exec.vector.DateVector;
    +import org.apache.drill.exec.vector.BigIntVector;
    +import org.apache.drill.exec.vector.NullableBigIntVector;
    +import org.apache.drill.exec.vector.NullableFloat8Vector;
    +import org.apache.drill.exec.vector.NullableVarBinaryVector;
    +import org.apache.drill.exec.vector.VarCharVector;
    +import org.apache.drill.exec.vector.complex.MapVector;
    +
    +import java.io.ByteArrayInputStream;
    +import java.io.DataInputStream;
    +import java.io.IOException;
    +import java.util.List;
    +import java.util.GregorianCalendar;
    +import java.util.HashMap;
    +import java.util.Map;
    +import java.util.TimeZone;
    +
    +import com.clearspring.analytics.stream.cardinality.HyperLogLog;
    +
    +public class StatisticsMergeBatch extends AbstractSingleRecordBatch<StatisticsMerge> {
    +  private static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(StatisticsMergeBatch.class);
    +  private Map<String, String> functions;
    +  private boolean first = true;
    +  private boolean finished = false;
    +  private int schema = 0;
    +  private int recordCount = 0;
    +  private List<String> keyList = null;
    +  private Map<MaterializedField, ValueVector> dataSrcVecMap = null;
    +  // Map of non-map fields to VV in the incoming schema
    +  private Map<MaterializedField, ValueVector> copySrcVecMap = null;
    +  private Map<String, Map<String, ValueHolder>> aggregationMap = null;
    +  public StatisticsMergeBatch(StatisticsMerge popConfig, RecordBatch incoming,
    +                              FragmentContext context) throws OutOfMemoryException {
    +    super(popConfig, context, incoming);
    +    this.functions = new HashMap<>();
    +    this.aggregationMap = new HashMap<>();
    +
    +    /*for (String key : popConfig.getFunctions()) {
    +      aggregationMap.put(key, new HashMap<String, ValueHolder>());
    +      if (key.equalsIgnoreCase("statcount") || key.equalsIgnoreCase("nonnullstatcount")) {
    +        functions.put(key, "sum");
    +      } else if (key.equalsIgnoreCase("hll")) {
    +        functions.put(key, "hll_merge");
    +      } else if (key.equalsIgnoreCase("sum_width")) {
    +        functions.put(key, "avg_width");
    +      }
    +    }*/
    +    for (String key : popConfig.getFunctions()) {
    +      if (key.equalsIgnoreCase("sum_width")) {
    +        functions.put(key, "avg_width");
    +      } else if (key.equalsIgnoreCase("hll")) {
    +        functions.put(key, "hll_merge");
    +      } else {
    +        functions.put(key, key);
    +      }
    +      aggregationMap.put(functions.get(key), new HashMap<String, ValueHolder>());
    +    }
    +  }
    +
    +  private void createKeyColumn(String name, LogicalExpression expr, Map<MaterializedField, ValueVector> parentMap)
    +      throws SchemaChangeException {
    +    ErrorCollector collector = new ErrorCollectorImpl();
    +
    +    LogicalExpression mle = ExpressionTreeMaterializer.materialize(expr, incoming, collector,
    +        context.getFunctionRegistry());
    +
    +    MaterializedField outputField = MaterializedField.create(name, mle.getMajorType());
    +    ValueVector vector = TypeHelper.getNewVector(outputField, oContext.getAllocator());
    +    container.add(vector);
    +
    +    if (collector.hasErrors()) {
    +      throw new SchemaChangeException("Failure while materializing expression. "
    +          + collector.toErrorString());
    +    }
    +    parentMap.put(outputField, vector);
    +  }
    +
    +  private ValueVector addMapVector(String name, MapVector parent, LogicalExpression expr) throws SchemaChangeException {
    +    ErrorCollector collector = new ErrorCollectorImpl();
    +
    +    LogicalExpression mle = ExpressionTreeMaterializer.materialize(expr, incoming, collector,
    +        context.getFunctionRegistry());
    +
    +    Class<? extends ValueVector> vvc =
    +        TypeHelper.getValueVectorClass(mle.getMajorType().getMinorType(), mle.getMajorType().getMode());
    +    ValueVector vector = parent.addOrGet(name, mle.getMajorType(), vvc);
    +
    +    if (collector.hasErrors()) {
    +      throw new SchemaChangeException("Failure while materializing expression. "
    +          + collector.toErrorString());
    +    }
    +    return vector;
    +  }
    +
    +  /**
    +   * Identify the list of fields within a map which are unpivoted as columns in output
    +   */
    +  private void buildKeyList() {
    +    List<String> lastMapKeyList = null;
    +    for (VectorWrapper<?> vw : incoming) {
    +      if (vw.getField().getType().getMinorType() != TypeProtos.MinorType.MAP) {
    +        continue;
    +      }
    +
    +      keyList = Lists.newArrayList();
    +
    +      for (ValueVector vv : vw.getValueVector()) {
    +        keyList.add(vv.getField().getLastName());
    +      }
    +
    +      if (lastMapKeyList == null) {
    +        lastMapKeyList = keyList;
    +      } else {
    +        if (keyList.size() != lastMapKeyList.size() || !lastMapKeyList.containsAll(keyList)) {
    +          throw new UnsupportedOperationException("Maps have different fields");
    +        }
    +      }
    +    }
    +  }
    +
    +  private void buildOutputContainer() throws SchemaChangeException {
    +    dataSrcVecMap = Maps.newHashMap();
    +    copySrcVecMap = Maps.newHashMap();
    +    MajorType mt = null;
    +
    +    ErrorCollector collector = new ErrorCollectorImpl();
    +    GregorianCalendar calendar = new GregorianCalendar(TimeZone.getTimeZone("UTC"));
    +
    +    calendar.setTimeInMillis(System.currentTimeMillis());
    +    createKeyColumn("schema", ValueExpressions.getBigInt(schema++), copySrcVecMap);
    +    createKeyColumn("computed", ValueExpressions.getDate(calendar), copySrcVecMap);
    +
    +    for (VectorWrapper<?> vw : incoming) {
    +      addVectorToOutgoingContainer(vw.getField().getLastName(), vw, collector);
    +      /*MaterializedField ds = vw.getField();
    +      String field = vw.getField().getLastName();
    +      // Input map vector
    +      MapVector mapVector = (MapVector) vw.getValueVector();
    +      assert mapVector.getPrimitiveVectors().size() > 0;
    +      // Proceed to create output map vector with same name e.g. statcount etc.
    +      mt = mapVector.getField().getType();
    +      MaterializedField mf = MaterializedField.create(functions.get(field), mt);
    +      assert !dataSrcVecMap.containsKey(mf);
    +      ValueVector vector = TypeHelper.getNewVector(mf, oContext.getAllocator());
    +      container.add(vector);
    +      MapVector outputMapVector = (MapVector) vector;
    +
    +      for (ValueVector vv : mapVector) {
    +        String fieldName = vv.getField().getLastName();
    +        if (!keyList.contains(fieldName)) {
    +          throw new UnsupportedOperationException("Unpivot data vector " +
    +              ds + " contains key " + fieldName + " not contained in key source!");
    +        }
    +        if (vv.getField().getType().getMinorType() == TypeProtos.MinorType.MAP) {
    +          throw new UnsupportedOperationException("Unpivot of nested map is not supported!");
    +        }
    +        if (field.equals("column")) {
    +          outputMapVector.addOrGet(fieldName, vv.getField().getType(), vv.getClass());
    +        } else {
    +          List<LogicalExpression> args = Lists.newArrayList();
    +          //TODO: Something else to access value of col such as emp_id?
    +          args.add(SchemaPath.getSimplePath(vv.getField().getPath()));
    +          //TODO: Put in the mapVector
    +          LogicalExpression call = FunctionCallFactory.createExpression(functions.get(field), args);
    +          //TODO: Is this sufficient to add to new Map?
    +          ValueVector vector1 = addMapVector(fieldName, outputMapVector, call);
    +          if (collector.hasErrors()) {
    +            throw new SchemaChangeException("Failure while materializing expression. "
    +                + collector.toErrorString());
    +          }
    +        }
    +      }
    +      dataSrcVecMap.put(ds, outputMapVector);*/
    +    }
    +    //Now create NDV in the outgoing container which was not avaliable in the incoming
    +    for (VectorWrapper<?> vw : incoming) {
    +      if (vw.getField().getLastName().equalsIgnoreCase("sum_width")) {//NullableFloat8 type
    +        addVectorToOutgoingContainer("ndv", vw, collector);
    +        break;
    +      }
    +    }
    +    container.setRecordCount(0);
    +    recordCount = 0;
    +    container.buildSchema(incoming.getSchema().getSelectionVectorMode());
    +  }
    +
    +  private void addVectorToOutgoingContainer(String field, VectorWrapper vw, ErrorCollector collector)
    +     throws SchemaChangeException {
    +    // Input map vector
    +    MapVector mapVector = (MapVector) vw.getValueVector();
    +    MaterializedField mf;
    +    assert mapVector.getPrimitiveVectors().size() > 0;
    +    // Proceed to create output map vector with same name e.g. statcount etc.
    +    MajorType mt = mapVector.getField().getType();
    +    if (functions.get(field) != null) {
    +      mf = MaterializedField.create(functions.get(field), mt);
    +    } else {
    +      mf = MaterializedField.create(field, mt);
    +    }
    +    assert !dataSrcVecMap.containsKey(mf);
    +    ValueVector vector = TypeHelper.getNewVector(mf, oContext.getAllocator());
    +    container.add(vector);
    +    MapVector outputMapVector = (MapVector) vector;
    +
    +    for (ValueVector vv : mapVector) {
    +      String fieldName = vv.getField().getLastName();
    +      if (!keyList.contains(fieldName)) {
    +        throw new UnsupportedOperationException("Unpivot data vector " +
    +                field + " contains key " + fieldName + " not contained in key source!");
    +      }
    +      if (vv.getField().getType().getMinorType() == TypeProtos.MinorType.MAP) {
    +        throw new UnsupportedOperationException("Unpivot of nested map is not supported!");
    +      }
    +      if (field.equals("column")) {
    +        outputMapVector.addOrGet(fieldName, vv.getField().getType(), vv.getClass());
    +      } else {
    +        List<LogicalExpression> args = Lists.newArrayList();
    +        LogicalExpression call;
    +        //TODO: Something else to access value of col such as emp_id?
    +        args.add(SchemaPath.getSimplePath(vv.getField().getPath()));
    +        //TODO: Put in the mapVector
    +        if (functions.get(field) != null) {
    +          call = FunctionCallFactory.createExpression(functions.get(field), args);
    +        } else {
    +          call = FunctionCallFactory.createExpression(field, args);
    +        }
    +        //TODO: Is this sufficient to add to new Map?
    +        ValueVector vector1 = addMapVector(fieldName, outputMapVector, call);
    +        if (collector.hasErrors()) {
    +          throw new SchemaChangeException("Failure while materializing expression. "
    +                  + collector.toErrorString());
    +        }
    +      }
    +    }
    +  }
    +
    +  @Override
    +  protected boolean setupNewSchema() throws SchemaChangeException {
    +    container.clear();
    +    buildKeyList();
    +    buildOutputContainer();
    +    return true;
    +  }
    +
    +  @Override
    +  protected IterOutcome doWork() {
    +    int outRecordCount = incoming.getRecordCount();
    +    HashMap<String, Long> nonNullRows = new HashMap<>();
    +
    +    for (VectorWrapper<?> vw : incoming) {
    +      ValueVector vv = vw.getValueVector();
    +      if (vv.getField().getType().getMinorType() != TypeProtos.MinorType.MAP) {
    +        //We only expect Maps in the incoming. STOP, if this is not the case.
    +        return IterOutcome.STOP;
    +      }
    +      MapVector mapVec = (MapVector) vv;
    +      Map<String, ValueHolder> statMap = aggregationMap.get(vv.getField().getLastName());
    +
    +      for (ValueVector mapElt : mapVec) {
    +        if (vv.getField().getLastName().equalsIgnoreCase("column")) {
    +          BigIntHolder nameHolder;
    +          if (statMap.get(mapElt.getField().getLastName()) != null) {
    +            nameHolder = (BigIntHolder) statMap.get(mapElt.getField().getLastName());
    --- End diff --
    
    Yes, this was a hacky way of doing it. Removed it.


---
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 #729: Drill 1328: Support table statistics for Parquet

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

    https://github.com/apache/drill/pull/729#discussion_r100446883
  
    --- Diff: exec/java-exec/src/main/java/org/apache/drill/exec/ops/FragmentContext.java ---
    @@ -392,6 +402,11 @@ public String getQueryUserName() {
         return fragment.getCredentials().getUserName();
       }
     
    +  /**
    +   * @return ID {@link java.util.UUID} of the current query
    +   */
    +  public String getQueryId() { return QueryIdHelper.getQueryId(fragment.getHandle().getQueryId());}
    --- 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 #729: Drill 1328: Support table statistics for Parquet

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/729#discussion_r100676572
  
    --- Diff: exec/java-exec/src/main/java/org/apache/drill/exec/physical/config/StatisticsAggregate.java ---
    @@ -0,0 +1,73 @@
    +/**
    + * 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.config;
    +
    +import org.apache.drill.exec.physical.base.PhysicalOperator;
    +import org.apache.drill.exec.physical.base.PhysicalVisitor;
    +import org.apache.drill.exec.planner.physical.StatsAggPrel.OperatorPhase;
    +import org.apache.drill.exec.proto.UserBitShared.CoreOperatorType;
    +
    +import com.fasterxml.jackson.annotation.JsonCreator;
    +import com.fasterxml.jackson.annotation.JsonProperty;
    +import com.fasterxml.jackson.annotation.JsonTypeName;
    +
    +import com.google.common.collect.ImmutableList;
    +
    +import java.util.List;
    +
    +@JsonTypeName("statistics-aggregate")
    +public class StatisticsAggregate extends StreamingAggregate {
    +  // private static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(StatisticsAggregate.class);
    +  protected OperatorPhase phase = OperatorPhase.PHASE_1of1;  // default phase
    --- End diff --
    
    Can we devise a better name? What does it mean to be "Phase 1 of 1"? If there is only a single phase, do we need an enum? If we do need an enum, can we choose descriptive 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 #729: Drill 1328: Support table statistics for Parquet

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/729#discussion_r102873722
  
    --- Diff: exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/statistics/AvgWidthMergedStatistic.java ---
    @@ -0,0 +1,135 @@
    +/*
    + * 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.impl.statistics;
    +
    +import org.apache.drill.common.types.TypeProtos;
    +import org.apache.drill.exec.expr.holders.NullableFloat8Holder;
    +import org.apache.drill.exec.expr.holders.ValueHolder;
    +import org.apache.drill.exec.vector.NullableFloat8Vector;
    +import org.apache.drill.exec.vector.ValueVector;
    +import org.apache.drill.exec.vector.complex.MapVector;
    +
    +import java.util.HashMap;
    +import java.util.List;
    +import java.util.Map;
    +
    +public class AvgWidthMergedStatistic extends AbstractMergedStatistic {
    +
    +  private String name;
    +  private String inputName;
    +  private boolean configureComplete = false;
    +  private boolean mergeComplete = false;
    --- End diff --
    
    Not sure you really need these state flags. My guess is that you can rely on an NPE to tell us we are calling things wrong. For example, asking for config before it is set will cause an NPE which tells the programmer he did something wrong.
    
    If state was important (and could vary), then having a state enum would be better:
    
    ```
    enum State {Config, Merge, Complete};
    ```
    
    But, again, we are just catching programming errors, so the flags are probably not necessary.


---
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 #729: Drill 1328: Support table statistics for Parquet

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

    https://github.com/apache/drill/pull/729#discussion_r103611163
  
    --- Diff: exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/statistics/StatisticsAggBatch.java ---
    @@ -0,0 +1,256 @@
    +/**
    + * 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.impl.statistics;
    +
    +import com.google.common.collect.Lists;
    +import com.sun.codemodel.JExpr;
    +import org.apache.drill.common.expression.ErrorCollector;
    +import org.apache.drill.common.expression.ErrorCollectorImpl;
    +import org.apache.drill.common.expression.FunctionCallFactory;
    +import org.apache.drill.common.expression.LogicalExpression;
    +import org.apache.drill.common.expression.SchemaPath;
    +import org.apache.drill.common.expression.ValueExpressions;
    +import org.apache.drill.exec.exception.ClassTransformationException;
    +import org.apache.drill.exec.exception.OutOfMemoryException;
    +import org.apache.drill.exec.exception.SchemaChangeException;
    +import org.apache.drill.exec.expr.ClassGenerator;
    +import org.apache.drill.exec.expr.CodeGenerator;
    +import org.apache.drill.exec.expr.ExpressionTreeMaterializer;
    +import org.apache.drill.exec.expr.TypeHelper;
    +import org.apache.drill.exec.expr.ValueVectorWriteExpression;
    +import org.apache.drill.exec.ops.FragmentContext;
    +import org.apache.drill.exec.physical.config.StatisticsAggregate;
    +import org.apache.drill.exec.physical.impl.aggregate.StreamingAggBatch;
    +import org.apache.drill.exec.physical.impl.aggregate.StreamingAggTemplate;
    +import org.apache.drill.exec.physical.impl.aggregate.StreamingAggregator;
    +import org.apache.drill.exec.planner.physical.StatsAggPrel.OperatorPhase;
    +import org.apache.drill.exec.record.BatchSchema.SelectionVectorMode;
    +import org.apache.drill.exec.record.MaterializedField;
    +import org.apache.drill.exec.record.RecordBatch;
    +import org.apache.drill.exec.record.TypedFieldId;
    +import org.apache.drill.exec.store.ImplicitColumnExplorer;
    +import org.apache.drill.exec.vector.ValueVector;
    +import org.apache.drill.exec.vector.complex.FieldIdUtil;
    +import org.apache.drill.exec.vector.complex.MapVector;
    +
    +import java.io.IOException;
    +import java.util.GregorianCalendar;
    +import java.util.List;
    +import java.util.TimeZone;
    +
    +/**
    + * TODO: This needs cleanup. Currently the key values are constants and we compare the constants
    + * for every record. Seems unnecessary.
    + *
    + * Example input and output:
    + * Schema of incoming batch: region_id (VARCHAR), sales_city (VARCHAR), cnt (BIGINT)
    + * Schema of output:
    + *    "schema" : BIGINT - Schema number. For each schema change this number is incremented.
    + *    "computed" : BIGINT - What time is it computed?
    --- 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 #729: Drill 1328: Support table statistics for Parquet

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

    https://github.com/apache/drill/pull/729#discussion_r103608020
  
    --- Diff: exec/java-exec/src/main/java/org/apache/drill/exec/planner/common/DrillRelOptUtil.java ---
    @@ -169,4 +182,99 @@ private static boolean containIdentity(List<? extends RexNode> exps,
         }
         return true;
       }
    +
    +  /**
    +   * Returns whether statistics-based estimates or guesses are used by the optimizer
    +   * */
    +  public static boolean guessRows(RelNode rel) {
    +    final PlannerSettings settings =
    +        rel.getCluster().getPlanner().getContext().unwrap(PlannerSettings.class);
    +    if (!settings.useStatistics()) {
    +      return true;
    +    }
    +    if (rel instanceof RelSubset) {
    +      if (((RelSubset) rel).getBest() != null) {
    +        return guessRows(((RelSubset) rel).getBest());
    +      } else if (((RelSubset) rel).getOriginal() != null) {
    +        return guessRows(((RelSubset) rel).getOriginal());
    +      }
    +    } else if (rel instanceof HepRelVertex) {
    +      if (((HepRelVertex) rel).getCurrentRel() != null) {
    +        return guessRows(((HepRelVertex) rel).getCurrentRel());
    +      }
    +    } else if (rel instanceof TableScan) {
    +      DrillTable table = rel.getTable().unwrap(DrillTable.class);
    +      if (table == null) {
    +        table = rel.getTable().unwrap(DrillTranslatableTable.class).getDrillTable();
    +      }
    +      if (table != null && table.getStatsTable() != null) {
    +        return false;
    +      } else {
    +        return true;
    +      }
    +    } else {
    +      for (RelNode child : rel.getInputs()) {
    +        if (guessRows(child)) { // at least one child is a guess
    +          return true;
    +        }
    +      }
    +    }
    +    return false;
    +  }
    +
    +  private static boolean findLikeOrRangePredicate(RexNode predicate) {
    +    if ((predicate == null) || predicate.isAlwaysTrue()) {
    +      return false;
    +    }
    +    for (RexNode pred : RelOptUtil.conjunctions(predicate)) {
    +      for (RexNode orPred : RelOptUtil.disjunctions(pred)) {
    +        if (!orPred.isA(SqlKind.EQUALS) ||
    +             orPred.isA(SqlKind.LIKE)) {
    +          return true;
    +        }
    +      }
    +    }
    +    return false;
    +  }
    +
    +  public static boolean analyzeSimpleEquiJoin(Join join, int[] joinFieldOrdinals) {
    --- 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 #729: Drill 1328: Support table statistics for Parquet

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

    https://github.com/apache/drill/pull/729#discussion_r103619932
  
    --- Diff: exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/statistics/AbstractMergedStatistic.java ---
    @@ -0,0 +1,52 @@
    +/*
    + * 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.impl.statistics;
    +
    +import org.apache.drill.exec.vector.ValueVector;
    +
    +public abstract class AbstractMergedStatistic extends Statistic implements MergedStatistic {
    +  @Override
    +  public String getName() {
    +    throw new UnsupportedOperationException("getName() not implemented");
    +  }
    +
    +  @Override
    +  public String getInput() {
    +    throw new UnsupportedOperationException("getInput() not implemented");
    +  }
    +
    +  @Override
    +  public void merge(ValueVector input) {
    +    throw new UnsupportedOperationException("merge() not implemented");
    +  }
    +
    +  @Override
    +  public Object getStat(String colName) {
    +    throw new UnsupportedOperationException("getStat() not implemented");
    +  }
    +
    +  @Override
    +  public void setOutput(ValueVector output) {
    +    throw new UnsupportedOperationException("getOutput() not implemented");
    +  }
    +
    +  @Override
    +  public void configure(Object configurations) {
    --- End diff --
    
    Changed the types to be specific. This config does not do hash lookups. However, for functions like `merge()` and `setOutput()` do hash lookups which might not be as expensive given the alternative (columns mappings for `MapVector` which needs to be recomputed for each incoming batch). Also, all the `MergedStatistic` and `StatisticsMergeBatch` contain a single map and given that both the incoming(no. of minor fragments) and outgoing batches(1) would not be many we can keep the code simple 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 #729: Drill 1328 r4

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

    https://github.com/apache/drill/pull/729#discussion_r98108433
  
    --- Diff: exec/java-exec/src/main/java/org/apache/drill/exec/planner/sql/handlers/AnalyzeTableHandler.java ---
    @@ -0,0 +1,256 @@
    +/**
    + * 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.planner.sql.handlers;
    +
    +import java.io.IOException;
    +import java.util.List;
    +
    +import org.apache.calcite.rel.RelNode;
    +import org.apache.calcite.schema.Table;
    +import org.apache.calcite.sql.SqlIdentifier;
    +import org.apache.calcite.sql.SqlNode;
    +import org.apache.calcite.sql.SqlNodeList;
    +import org.apache.calcite.sql.SqlSelect;
    +import org.apache.calcite.sql.parser.SqlParserPos;
    +import org.apache.calcite.tools.RelConversionException;
    +import org.apache.calcite.tools.ValidationException;
    +import org.apache.drill.common.exceptions.UserException;
    +import org.apache.drill.common.logical.FormatPluginConfig;
    +import org.apache.drill.exec.dotdrill.DotDrillType;
    +import org.apache.drill.exec.physical.PhysicalPlan;
    +import org.apache.drill.exec.physical.base.PhysicalOperator;
    +import org.apache.drill.exec.planner.logical.DrillAnalyzeRel;
    +import org.apache.drill.exec.planner.logical.DrillRel;
    +import org.apache.drill.exec.planner.logical.DrillScreenRel;
    +import org.apache.drill.exec.planner.logical.DrillStoreRel;
    +import org.apache.drill.exec.planner.logical.DrillWriterRel;
    +import org.apache.drill.exec.planner.logical.DrillTable;
    +import org.apache.drill.exec.planner.physical.Prel;
    +import org.apache.drill.exec.planner.sql.DirectPlan;
    +import org.apache.drill.exec.planner.sql.SchemaUtilites;
    +import org.apache.drill.exec.planner.sql.parser.SqlAnalyzeTable;
    +import org.apache.drill.exec.store.AbstractSchema;
    +import org.apache.drill.exec.store.dfs.DrillFileSystem;
    +import org.apache.drill.exec.store.dfs.FileSystemPlugin;
    +import org.apache.drill.exec.store.dfs.FormatSelection;
    +import org.apache.drill.exec.store.dfs.NamedFormatPluginConfig;
    +import org.apache.drill.exec.store.parquet.ParquetFormatConfig;
    +import org.apache.drill.exec.util.Pointer;
    +import org.apache.drill.exec.work.foreman.ForemanSetupException;
    +import org.apache.drill.exec.work.foreman.SqlUnsupportedException;
    +import org.apache.hadoop.fs.Path;
    +import org.apache.hadoop.fs.FileStatus;
    +
    +public class AnalyzeTableHandler extends DefaultSqlHandler {
    +  private static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(AnalyzeTableHandler.class);
    +
    +  public AnalyzeTableHandler(SqlHandlerConfig config, Pointer<String> textPlan) {
    +    super(config, textPlan);
    +  }
    +
    +  @Override
    +  public PhysicalPlan getPlan(SqlNode sqlNode)
    +      throws ValidationException, RelConversionException, IOException, ForemanSetupException {
    +    final SqlAnalyzeTable sqlAnalyzeTable = unwrap(sqlNode, SqlAnalyzeTable.class);
    +
    +    verifyNoUnsupportedFunctions(sqlAnalyzeTable);
    +
    +    SqlIdentifier tableIdentifier = sqlAnalyzeTable.getTableIdentifier();
    +    SqlSelect scanSql = new SqlSelect(
    +        SqlParserPos.ZERO,              /* position */
    +        SqlNodeList.EMPTY,              /* keyword list */
    +        getColumnList(sqlAnalyzeTable), /* select list */
    +        tableIdentifier,                /* from */
    +        null,                           /* where */
    +        null,                           /* group by */
    +        null,                           /* having */
    +        null,                           /* windowDecls */
    +        null,                           /* orderBy */
    +        null,                           /* offset */
    +        null                            /* fetch */
    +    );
    +
    +    final ConvertedRelNode convertedRelNode = validateAndConvert(rewrite(scanSql));
    +    final RelNode relScan = convertedRelNode.getConvertedNode();
    +    final String tableName = sqlAnalyzeTable.getName();
    +    final AbstractSchema drillSchema = SchemaUtilites.resolveToDrillSchema(
    +        config.getConverter().getDefaultSchema(), sqlAnalyzeTable.getSchemaPath());
    +    Table table = SqlHandlerUtil.getTableFromSchema(drillSchema, tableName);
    +
    +    if (table == null) {
    +      throw UserException.validationError()
    +          .message("No table with given name [%s] exists in schema [%s]", tableName,
    +              drillSchema.getFullSchemaName())
    +          .build(logger);
    +    }
    +
    +    if(! (table instanceof DrillTable)) {
    +      return notSupported(tableName);
    +    }
    +
    +    if (table instanceof DrillTable) {
    +      DrillTable drillTable = (DrillTable) table;
    +      final Object selection = drillTable.getSelection();
    +      if (!(selection instanceof FormatSelection)) {
    +        return notSupported(tableName);
    +      }
    +      // Do not support non-parquet tables
    +      FormatSelection formatSelection = (FormatSelection) selection;
    +      FormatPluginConfig formatConfig = formatSelection.getFormat();
    +      if (!((formatConfig instanceof ParquetFormatConfig)
    --- End diff --
    
    It would be better to add an API to GroupScan interface for checking if the underlying GroupScan supports statistics collection.   For now, only ParquetGroupScan will return true for that. 


---
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 #729: Drill 1328: Support table statistics for Parquet

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/729#discussion_r100680528
  
    --- Diff: exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/statistics/StatisticsMergeBatch.java ---
    @@ -0,0 +1,532 @@
    +/**
    + * 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.impl.statistics;
    +
    +import com.google.common.collect.Lists;
    +import com.google.common.collect.Maps;
    +import org.apache.drill.common.expression.LogicalExpression;
    +import org.apache.drill.common.expression.ErrorCollector;
    +import org.apache.drill.common.expression.ErrorCollectorImpl;
    +import org.apache.drill.common.expression.FunctionCallFactory;
    +import org.apache.drill.common.expression.SchemaPath;
    +import org.apache.drill.common.expression.ValueExpressions;
    +import org.apache.drill.common.types.TypeProtos;
    +import org.apache.drill.common.types.TypeProtos.MajorType;
    +import org.apache.drill.exec.exception.OutOfMemoryException;
    +import org.apache.drill.exec.exception.SchemaChangeException;
    +import org.apache.drill.exec.expr.ExpressionTreeMaterializer;
    +import org.apache.drill.exec.expr.TypeHelper;
    +import org.apache.drill.exec.expr.holders.BigIntHolder;
    +import org.apache.drill.exec.expr.holders.NullableFloat8Holder;
    +import org.apache.drill.exec.expr.holders.ObjectHolder;
    +import org.apache.drill.exec.expr.holders.ValueHolder;
    +import org.apache.drill.exec.ops.FragmentContext;
    +import org.apache.drill.exec.physical.config.StatisticsMerge;
    +import org.apache.drill.exec.record.AbstractSingleRecordBatch;
    +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.VectorContainer;
    +import org.apache.drill.exec.record.VectorWrapper;
    +import org.apache.drill.exec.vector.ValueVector;
    +import org.apache.drill.exec.vector.DateVector;
    +import org.apache.drill.exec.vector.BigIntVector;
    +import org.apache.drill.exec.vector.NullableBigIntVector;
    +import org.apache.drill.exec.vector.NullableFloat8Vector;
    +import org.apache.drill.exec.vector.NullableVarBinaryVector;
    +import org.apache.drill.exec.vector.VarCharVector;
    +import org.apache.drill.exec.vector.complex.MapVector;
    +
    +import java.io.ByteArrayInputStream;
    +import java.io.DataInputStream;
    +import java.io.IOException;
    +import java.util.List;
    +import java.util.GregorianCalendar;
    +import java.util.HashMap;
    +import java.util.Map;
    +import java.util.TimeZone;
    +
    +import com.clearspring.analytics.stream.cardinality.HyperLogLog;
    +
    +public class StatisticsMergeBatch extends AbstractSingleRecordBatch<StatisticsMerge> {
    +  private static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(StatisticsMergeBatch.class);
    +  private Map<String, String> functions;
    +  private boolean first = true;
    +  private boolean finished = false;
    +  private int schema = 0;
    +  private int recordCount = 0;
    +  private List<String> keyList = null;
    +  private Map<MaterializedField, ValueVector> dataSrcVecMap = null;
    +  // Map of non-map fields to VV in the incoming schema
    +  private Map<MaterializedField, ValueVector> copySrcVecMap = null;
    +  private Map<String, Map<String, ValueHolder>> aggregationMap = null;
    +  public StatisticsMergeBatch(StatisticsMerge popConfig, RecordBatch incoming,
    +                              FragmentContext context) throws OutOfMemoryException {
    +    super(popConfig, context, incoming);
    +    this.functions = new HashMap<>();
    +    this.aggregationMap = new HashMap<>();
    +
    +    /*for (String key : popConfig.getFunctions()) {
    +      aggregationMap.put(key, new HashMap<String, ValueHolder>());
    +      if (key.equalsIgnoreCase("statcount") || key.equalsIgnoreCase("nonnullstatcount")) {
    +        functions.put(key, "sum");
    +      } else if (key.equalsIgnoreCase("hll")) {
    +        functions.put(key, "hll_merge");
    +      } else if (key.equalsIgnoreCase("sum_width")) {
    +        functions.put(key, "avg_width");
    +      }
    +    }*/
    +    for (String key : popConfig.getFunctions()) {
    +      if (key.equalsIgnoreCase("sum_width")) {
    +        functions.put(key, "avg_width");
    +      } else if (key.equalsIgnoreCase("hll")) {
    +        functions.put(key, "hll_merge");
    +      } else {
    +        functions.put(key, key);
    +      }
    +      aggregationMap.put(functions.get(key), new HashMap<String, ValueHolder>());
    +    }
    +  }
    +
    +  private void createKeyColumn(String name, LogicalExpression expr, Map<MaterializedField, ValueVector> parentMap)
    +      throws SchemaChangeException {
    +    ErrorCollector collector = new ErrorCollectorImpl();
    +
    +    LogicalExpression mle = ExpressionTreeMaterializer.materialize(expr, incoming, collector,
    +        context.getFunctionRegistry());
    +
    +    MaterializedField outputField = MaterializedField.create(name, mle.getMajorType());
    +    ValueVector vector = TypeHelper.getNewVector(outputField, oContext.getAllocator());
    +    container.add(vector);
    +
    +    if (collector.hasErrors()) {
    +      throw new SchemaChangeException("Failure while materializing expression. "
    +          + collector.toErrorString());
    +    }
    +    parentMap.put(outputField, vector);
    +  }
    +
    +  private ValueVector addMapVector(String name, MapVector parent, LogicalExpression expr) throws SchemaChangeException {
    +    ErrorCollector collector = new ErrorCollectorImpl();
    +
    +    LogicalExpression mle = ExpressionTreeMaterializer.materialize(expr, incoming, collector,
    +        context.getFunctionRegistry());
    +
    +    Class<? extends ValueVector> vvc =
    +        TypeHelper.getValueVectorClass(mle.getMajorType().getMinorType(), mle.getMajorType().getMode());
    +    ValueVector vector = parent.addOrGet(name, mle.getMajorType(), vvc);
    +
    +    if (collector.hasErrors()) {
    +      throw new SchemaChangeException("Failure while materializing expression. "
    +          + collector.toErrorString());
    +    }
    +    return vector;
    +  }
    +
    +  /**
    +   * Identify the list of fields within a map which are unpivoted as columns in output
    +   */
    +  private void buildKeyList() {
    +    List<String> lastMapKeyList = null;
    +    for (VectorWrapper<?> vw : incoming) {
    +      if (vw.getField().getType().getMinorType() != TypeProtos.MinorType.MAP) {
    +        continue;
    +      }
    +
    +      keyList = Lists.newArrayList();
    +
    +      for (ValueVector vv : vw.getValueVector()) {
    +        keyList.add(vv.getField().getLastName());
    +      }
    +
    +      if (lastMapKeyList == null) {
    +        lastMapKeyList = keyList;
    +      } else {
    +        if (keyList.size() != lastMapKeyList.size() || !lastMapKeyList.containsAll(keyList)) {
    +          throw new UnsupportedOperationException("Maps have different fields");
    +        }
    +      }
    +    }
    +  }
    +
    +  private void buildOutputContainer() throws SchemaChangeException {
    +    dataSrcVecMap = Maps.newHashMap();
    +    copySrcVecMap = Maps.newHashMap();
    +    MajorType mt = null;
    +
    +    ErrorCollector collector = new ErrorCollectorImpl();
    --- End diff --
    
    This code is very similar to the first-level aggregation code reviewed above. Again, it would be helpful to factor out common code rather than using copy & paste. Easier to review and easier to maintain.


---
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 #729: Drill 1328: Support table statistics for Parquet

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

    https://github.com/apache/drill/pull/729#discussion_r100420583
  
    --- Diff: exec/java-exec/src/main/java/org/apache/drill/exec/ExecConstants.java ---
    @@ -390,4 +391,15 @@
     
       String DYNAMIC_UDF_SUPPORT_ENABLED = "exec.udf.enable_dynamic_support";
       BooleanValidator DYNAMIC_UDF_SUPPORT_ENABLED_VALIDATOR = new BooleanValidator(DYNAMIC_UDF_SUPPORT_ENABLED, true, true);
    +
    +  /**
    +   * Option whose value is a long value representing the number of bits required for computing ndv (using HLL)
    +   */
    +  LongValidator NDV_MEMORY_LIMIT = new PositiveLongValidator("exec.statistics.ndv_memory_limit", 30, 20);
    +
    +  /**
    +   * Option whose value represents the current version of the statistics. Decreasing the value will generate
    +   * the older version of statistics
    +   */
    +  LongValidator STATISTICS_VERSION = new NonNegativeLongValidator("exec.statistics.capability_version", 1, 1);
    --- End diff --
    
    Yes. By default, the stats version computed is the latest version. However, if the user chooses to compute an older version (maybe the performance is not optimal) they can set this option to an older version. When planning a query, Drill would interpret the statistics based on the version in the stats 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 issue #729: Drill 1328: Support table statistics for Parquet

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

    https://github.com/apache/drill/pull/729
  
    This PR has been sitting idle for a while. Does it still make sense to continue with the reviews to get this into master? If so, @gparai, can you rebase onto the latest master and update with any fixes made since the last update?


---
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 #729: Drill 1328: Support table statistics for Parquet

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

    https://github.com/apache/drill/pull/729#discussion_r103417821
  
    --- Diff: exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/statistics/MergedStatisticFactory.java ---
    @@ -0,0 +1,46 @@
    +/*
    + * 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.impl.statistics;
    +
    +public class MergedStatisticFactory {
    +  /*
    +   * Creates the appropriate statistics object given the name of the statistics and the input statistic
    +   */
    +  public static MergedStatistic getMergedStatistic(String outputStatName, String inputStatName) {
    +    if (outputStatName == null || inputStatName == null) {
    +      return null;
    +    } else if (outputStatName.equals(Statistic.COLNAME)) {
    +      return new ColumnMergedStatistic(outputStatName, inputStatName);
    +    } else if (outputStatName.equals(Statistic.COLTYPE)) {
    +      return new ColTypeMergedStatistic(outputStatName, inputStatName);
    +    } else if (outputStatName.equals(Statistic.STATCOUNT)) {
    +      return new StatCountMergedStatistic(outputStatName, inputStatName);
    +    } else if (outputStatName.equals(Statistic.NNSTATCOUNT)) {
    +      return new NNStatCountMergedStatistic(outputStatName, inputStatName);
    +    } else if (outputStatName.equals(Statistic.AVG_WIDTH)) {
    +      return new AvgWidthMergedStatistic(outputStatName, inputStatName);
    +    } else if (outputStatName.equals(Statistic.HLL_MERGE)) {
    +      return new HLLMergedStatistic(outputStatName, inputStatName);
    --- End diff --
    
    Yes, the output name can be hard-coded in the class. I want to avoid hard-coding the input name because then for each new statistic we would have to define/duplicate the mapping in multiple places. Right now the mapping is defined only once in AnalyzePRule.java. Also there might be a many:1 mapping e.g. if the input implementation for a statistic changes but the interface remains the same. I don't see this happening now but maybe in the future.


---
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 #729: Drill 1328: Support table statistics for Parquet

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

    https://github.com/apache/drill/pull/729#discussion_r101181131
  
    --- Diff: exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/statistics/StatisticsMergeBatch.java ---
    @@ -0,0 +1,532 @@
    +/**
    + * 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.impl.statistics;
    +
    +import com.google.common.collect.Lists;
    +import com.google.common.collect.Maps;
    +import org.apache.drill.common.expression.LogicalExpression;
    +import org.apache.drill.common.expression.ErrorCollector;
    +import org.apache.drill.common.expression.ErrorCollectorImpl;
    +import org.apache.drill.common.expression.FunctionCallFactory;
    +import org.apache.drill.common.expression.SchemaPath;
    +import org.apache.drill.common.expression.ValueExpressions;
    +import org.apache.drill.common.types.TypeProtos;
    +import org.apache.drill.common.types.TypeProtos.MajorType;
    +import org.apache.drill.exec.exception.OutOfMemoryException;
    +import org.apache.drill.exec.exception.SchemaChangeException;
    +import org.apache.drill.exec.expr.ExpressionTreeMaterializer;
    +import org.apache.drill.exec.expr.TypeHelper;
    +import org.apache.drill.exec.expr.holders.BigIntHolder;
    +import org.apache.drill.exec.expr.holders.NullableFloat8Holder;
    +import org.apache.drill.exec.expr.holders.ObjectHolder;
    +import org.apache.drill.exec.expr.holders.ValueHolder;
    +import org.apache.drill.exec.ops.FragmentContext;
    +import org.apache.drill.exec.physical.config.StatisticsMerge;
    +import org.apache.drill.exec.record.AbstractSingleRecordBatch;
    +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.VectorContainer;
    +import org.apache.drill.exec.record.VectorWrapper;
    +import org.apache.drill.exec.vector.ValueVector;
    +import org.apache.drill.exec.vector.DateVector;
    +import org.apache.drill.exec.vector.BigIntVector;
    +import org.apache.drill.exec.vector.NullableBigIntVector;
    +import org.apache.drill.exec.vector.NullableFloat8Vector;
    +import org.apache.drill.exec.vector.NullableVarBinaryVector;
    +import org.apache.drill.exec.vector.VarCharVector;
    +import org.apache.drill.exec.vector.complex.MapVector;
    +
    +import java.io.ByteArrayInputStream;
    +import java.io.DataInputStream;
    +import java.io.IOException;
    +import java.util.List;
    +import java.util.GregorianCalendar;
    +import java.util.HashMap;
    +import java.util.Map;
    +import java.util.TimeZone;
    +
    +import com.clearspring.analytics.stream.cardinality.HyperLogLog;
    +
    +public class StatisticsMergeBatch extends AbstractSingleRecordBatch<StatisticsMerge> {
    +  private static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(StatisticsMergeBatch.class);
    +  private Map<String, String> functions;
    +  private boolean first = true;
    +  private boolean finished = false;
    +  private int schema = 0;
    +  private int recordCount = 0;
    +  private List<String> keyList = null;
    +  private Map<MaterializedField, ValueVector> dataSrcVecMap = null;
    +  // Map of non-map fields to VV in the incoming schema
    +  private Map<MaterializedField, ValueVector> copySrcVecMap = null;
    +  private Map<String, Map<String, ValueHolder>> aggregationMap = null;
    +  public StatisticsMergeBatch(StatisticsMerge popConfig, RecordBatch incoming,
    +                              FragmentContext context) throws OutOfMemoryException {
    +    super(popConfig, context, incoming);
    +    this.functions = new HashMap<>();
    +    this.aggregationMap = new HashMap<>();
    +
    +    /*for (String key : popConfig.getFunctions()) {
    +      aggregationMap.put(key, new HashMap<String, ValueHolder>());
    +      if (key.equalsIgnoreCase("statcount") || key.equalsIgnoreCase("nonnullstatcount")) {
    +        functions.put(key, "sum");
    +      } else if (key.equalsIgnoreCase("hll")) {
    +        functions.put(key, "hll_merge");
    +      } else if (key.equalsIgnoreCase("sum_width")) {
    +        functions.put(key, "avg_width");
    +      }
    +    }*/
    +    for (String key : popConfig.getFunctions()) {
    +      if (key.equalsIgnoreCase("sum_width")) {
    +        functions.put(key, "avg_width");
    +      } else if (key.equalsIgnoreCase("hll")) {
    +        functions.put(key, "hll_merge");
    +      } else {
    +        functions.put(key, key);
    +      }
    +      aggregationMap.put(functions.get(key), new HashMap<String, ValueHolder>());
    +    }
    +  }
    +
    +  private void createKeyColumn(String name, LogicalExpression expr, Map<MaterializedField, ValueVector> parentMap)
    +      throws SchemaChangeException {
    +    ErrorCollector collector = new ErrorCollectorImpl();
    +
    +    LogicalExpression mle = ExpressionTreeMaterializer.materialize(expr, incoming, collector,
    +        context.getFunctionRegistry());
    +
    +    MaterializedField outputField = MaterializedField.create(name, mle.getMajorType());
    +    ValueVector vector = TypeHelper.getNewVector(outputField, oContext.getAllocator());
    +    container.add(vector);
    +
    +    if (collector.hasErrors()) {
    +      throw new SchemaChangeException("Failure while materializing expression. "
    +          + collector.toErrorString());
    +    }
    +    parentMap.put(outputField, vector);
    +  }
    +
    +  private ValueVector addMapVector(String name, MapVector parent, LogicalExpression expr) throws SchemaChangeException {
    +    ErrorCollector collector = new ErrorCollectorImpl();
    --- 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 #729: Drill 1328: Support table statistics for Parquet

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

    https://github.com/apache/drill/pull/729#discussion_r103395251
  
    --- Diff: exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/statistics/AvgWidthMergedStatistic.java ---
    @@ -0,0 +1,135 @@
    +/*
    + * 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.impl.statistics;
    +
    +import org.apache.drill.common.types.TypeProtos;
    +import org.apache.drill.exec.expr.holders.NullableFloat8Holder;
    +import org.apache.drill.exec.expr.holders.ValueHolder;
    +import org.apache.drill.exec.vector.NullableFloat8Vector;
    +import org.apache.drill.exec.vector.ValueVector;
    +import org.apache.drill.exec.vector.complex.MapVector;
    +
    +import java.util.HashMap;
    +import java.util.List;
    +import java.util.Map;
    +
    +public class AvgWidthMergedStatistic extends AbstractMergedStatistic {
    +
    +  private String name;
    +  private String inputName;
    +  private boolean configureComplete = false;
    +  private boolean mergeComplete = false;
    +  private Map<String, ValueHolder> sumHolder;
    +  MergedStatistic types, nonNullStatCounts, statCounts;
    +
    +  public AvgWidthMergedStatistic (String name, String inputName) {
    +    this.name = name;
    +    this.inputName = inputName;
    +    this.sumHolder = new HashMap<>();
    +    types = nonNullStatCounts = statCounts = null;
    +  }
    +
    +  @Override
    +  public String getName() {
    +    return name;
    +  }
    +
    +  @Override
    +  public String getInput() {
    +    return inputName;
    +  }
    +
    +  @Override
    +  public void merge(ValueVector input) {
    +    // Check the input is a Map Vector
    +    assert (input.getField().getType().getMinorType() == TypeProtos.MinorType.MAP);
    +    MapVector inputMap = (MapVector) input;
    +    for (ValueVector vv : inputMap) {
    +      String colName = vv.getField().getLastName();
    +      NullableFloat8Holder colSumHolder;
    +      if (sumHolder.get(colName) != null) {
    +        colSumHolder = (NullableFloat8Holder) sumHolder.get(colName);
    +      } else {
    +        colSumHolder = new NullableFloat8Holder();
    +        sumHolder.put(colName, colSumHolder);
    +      }
    +      Object val = vv.getAccessor().getObject(0);
    +      if (val != null) {
    +        colSumHolder.value += (double) val;
    +        colSumHolder.isSet = 1;
    +      }
    +    }
    +  }
    +
    +  @Override
    +  public Object getStat(String colName) {
    +      if (mergeComplete != true) {
    +        throw new IllegalStateException(
    +            String.format("Statistic `%s` has not completed merging statistics", name));
    +      }
    +      NullableFloat8Holder colSumHolder = (NullableFloat8Holder) sumHolder.get(colName);
    +      return (long) (colSumHolder.value/ getRowCount(colName));
    --- End diff --
    
    Yes, for variable length columns it returns nonnullrowcount. Added an 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 #729: Drill 1328: Support table statistics for Parquet

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

    https://github.com/apache/drill/pull/729#discussion_r102306890
  
    --- Diff: exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/statistics/StatisticsMergeBatch.java ---
    @@ -0,0 +1,532 @@
    +/**
    + * 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.impl.statistics;
    +
    +import com.google.common.collect.Lists;
    +import com.google.common.collect.Maps;
    +import org.apache.drill.common.expression.LogicalExpression;
    +import org.apache.drill.common.expression.ErrorCollector;
    +import org.apache.drill.common.expression.ErrorCollectorImpl;
    +import org.apache.drill.common.expression.FunctionCallFactory;
    +import org.apache.drill.common.expression.SchemaPath;
    +import org.apache.drill.common.expression.ValueExpressions;
    +import org.apache.drill.common.types.TypeProtos;
    +import org.apache.drill.common.types.TypeProtos.MajorType;
    +import org.apache.drill.exec.exception.OutOfMemoryException;
    +import org.apache.drill.exec.exception.SchemaChangeException;
    +import org.apache.drill.exec.expr.ExpressionTreeMaterializer;
    +import org.apache.drill.exec.expr.TypeHelper;
    +import org.apache.drill.exec.expr.holders.BigIntHolder;
    +import org.apache.drill.exec.expr.holders.NullableFloat8Holder;
    +import org.apache.drill.exec.expr.holders.ObjectHolder;
    +import org.apache.drill.exec.expr.holders.ValueHolder;
    +import org.apache.drill.exec.ops.FragmentContext;
    +import org.apache.drill.exec.physical.config.StatisticsMerge;
    +import org.apache.drill.exec.record.AbstractSingleRecordBatch;
    +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.VectorContainer;
    +import org.apache.drill.exec.record.VectorWrapper;
    +import org.apache.drill.exec.vector.ValueVector;
    +import org.apache.drill.exec.vector.DateVector;
    +import org.apache.drill.exec.vector.BigIntVector;
    +import org.apache.drill.exec.vector.NullableBigIntVector;
    +import org.apache.drill.exec.vector.NullableFloat8Vector;
    +import org.apache.drill.exec.vector.NullableVarBinaryVector;
    +import org.apache.drill.exec.vector.VarCharVector;
    +import org.apache.drill.exec.vector.complex.MapVector;
    +
    +import java.io.ByteArrayInputStream;
    +import java.io.DataInputStream;
    +import java.io.IOException;
    +import java.util.List;
    +import java.util.GregorianCalendar;
    +import java.util.HashMap;
    +import java.util.Map;
    +import java.util.TimeZone;
    +
    +import com.clearspring.analytics.stream.cardinality.HyperLogLog;
    +
    +public class StatisticsMergeBatch extends AbstractSingleRecordBatch<StatisticsMerge> {
    +  private static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(StatisticsMergeBatch.class);
    +  private Map<String, String> functions;
    +  private boolean first = true;
    +  private boolean finished = false;
    +  private int schema = 0;
    +  private int recordCount = 0;
    +  private List<String> keyList = null;
    +  private Map<MaterializedField, ValueVector> dataSrcVecMap = null;
    +  // Map of non-map fields to VV in the incoming schema
    +  private Map<MaterializedField, ValueVector> copySrcVecMap = null;
    +  private Map<String, Map<String, ValueHolder>> aggregationMap = null;
    +  public StatisticsMergeBatch(StatisticsMerge popConfig, RecordBatch incoming,
    +                              FragmentContext context) throws OutOfMemoryException {
    +    super(popConfig, context, incoming);
    +    this.functions = new HashMap<>();
    +    this.aggregationMap = new HashMap<>();
    +
    +    /*for (String key : popConfig.getFunctions()) {
    +      aggregationMap.put(key, new HashMap<String, ValueHolder>());
    +      if (key.equalsIgnoreCase("statcount") || key.equalsIgnoreCase("nonnullstatcount")) {
    +        functions.put(key, "sum");
    +      } else if (key.equalsIgnoreCase("hll")) {
    +        functions.put(key, "hll_merge");
    +      } else if (key.equalsIgnoreCase("sum_width")) {
    +        functions.put(key, "avg_width");
    +      }
    +    }*/
    +    for (String key : popConfig.getFunctions()) {
    +      if (key.equalsIgnoreCase("sum_width")) {
    +        functions.put(key, "avg_width");
    +      } else if (key.equalsIgnoreCase("hll")) {
    +        functions.put(key, "hll_merge");
    +      } else {
    +        functions.put(key, key);
    +      }
    +      aggregationMap.put(functions.get(key), new HashMap<String, ValueHolder>());
    +    }
    +  }
    +
    +  private void createKeyColumn(String name, LogicalExpression expr, Map<MaterializedField, ValueVector> parentMap)
    +      throws SchemaChangeException {
    +    ErrorCollector collector = new ErrorCollectorImpl();
    +
    +    LogicalExpression mle = ExpressionTreeMaterializer.materialize(expr, incoming, collector,
    +        context.getFunctionRegistry());
    +
    +    MaterializedField outputField = MaterializedField.create(name, mle.getMajorType());
    +    ValueVector vector = TypeHelper.getNewVector(outputField, oContext.getAllocator());
    +    container.add(vector);
    +
    +    if (collector.hasErrors()) {
    +      throw new SchemaChangeException("Failure while materializing expression. "
    +          + collector.toErrorString());
    +    }
    +    parentMap.put(outputField, vector);
    +  }
    +
    +  private ValueVector addMapVector(String name, MapVector parent, LogicalExpression expr) throws SchemaChangeException {
    +    ErrorCollector collector = new ErrorCollectorImpl();
    +
    +    LogicalExpression mle = ExpressionTreeMaterializer.materialize(expr, incoming, collector,
    +        context.getFunctionRegistry());
    +
    +    Class<? extends ValueVector> vvc =
    +        TypeHelper.getValueVectorClass(mle.getMajorType().getMinorType(), mle.getMajorType().getMode());
    +    ValueVector vector = parent.addOrGet(name, mle.getMajorType(), vvc);
    +
    +    if (collector.hasErrors()) {
    +      throw new SchemaChangeException("Failure while materializing expression. "
    +          + collector.toErrorString());
    +    }
    +    return vector;
    +  }
    +
    +  /**
    +   * Identify the list of fields within a map which are unpivoted as columns in output
    +   */
    +  private void buildKeyList() {
    +    List<String> lastMapKeyList = null;
    +    for (VectorWrapper<?> vw : incoming) {
    +      if (vw.getField().getType().getMinorType() != TypeProtos.MinorType.MAP) {
    +        continue;
    +      }
    +
    +      keyList = Lists.newArrayList();
    +
    +      for (ValueVector vv : vw.getValueVector()) {
    +        keyList.add(vv.getField().getLastName());
    +      }
    +
    +      if (lastMapKeyList == null) {
    +        lastMapKeyList = keyList;
    +      } else {
    +        if (keyList.size() != lastMapKeyList.size() || !lastMapKeyList.containsAll(keyList)) {
    +          throw new UnsupportedOperationException("Maps have different fields");
    +        }
    +      }
    +    }
    +  }
    +
    +  private void buildOutputContainer() throws SchemaChangeException {
    +    dataSrcVecMap = Maps.newHashMap();
    +    copySrcVecMap = Maps.newHashMap();
    +    MajorType mt = null;
    +
    +    ErrorCollector collector = new ErrorCollectorImpl();
    +    GregorianCalendar calendar = new GregorianCalendar(TimeZone.getTimeZone("UTC"));
    +
    +    calendar.setTimeInMillis(System.currentTimeMillis());
    +    createKeyColumn("schema", ValueExpressions.getBigInt(schema++), copySrcVecMap);
    +    createKeyColumn("computed", ValueExpressions.getDate(calendar), copySrcVecMap);
    +
    +    for (VectorWrapper<?> vw : incoming) {
    +      addVectorToOutgoingContainer(vw.getField().getLastName(), vw, collector);
    +      /*MaterializedField ds = vw.getField();
    +      String field = vw.getField().getLastName();
    +      // Input map vector
    +      MapVector mapVector = (MapVector) vw.getValueVector();
    +      assert mapVector.getPrimitiveVectors().size() > 0;
    +      // Proceed to create output map vector with same name e.g. statcount etc.
    +      mt = mapVector.getField().getType();
    +      MaterializedField mf = MaterializedField.create(functions.get(field), mt);
    +      assert !dataSrcVecMap.containsKey(mf);
    +      ValueVector vector = TypeHelper.getNewVector(mf, oContext.getAllocator());
    +      container.add(vector);
    +      MapVector outputMapVector = (MapVector) vector;
    +
    +      for (ValueVector vv : mapVector) {
    +        String fieldName = vv.getField().getLastName();
    +        if (!keyList.contains(fieldName)) {
    +          throw new UnsupportedOperationException("Unpivot data vector " +
    +              ds + " contains key " + fieldName + " not contained in key source!");
    +        }
    +        if (vv.getField().getType().getMinorType() == TypeProtos.MinorType.MAP) {
    +          throw new UnsupportedOperationException("Unpivot of nested map is not supported!");
    +        }
    +        if (field.equals("column")) {
    +          outputMapVector.addOrGet(fieldName, vv.getField().getType(), vv.getClass());
    +        } else {
    +          List<LogicalExpression> args = Lists.newArrayList();
    +          //TODO: Something else to access value of col such as emp_id?
    +          args.add(SchemaPath.getSimplePath(vv.getField().getPath()));
    +          //TODO: Put in the mapVector
    +          LogicalExpression call = FunctionCallFactory.createExpression(functions.get(field), args);
    +          //TODO: Is this sufficient to add to new Map?
    +          ValueVector vector1 = addMapVector(fieldName, outputMapVector, call);
    +          if (collector.hasErrors()) {
    +            throw new SchemaChangeException("Failure while materializing expression. "
    +                + collector.toErrorString());
    +          }
    +        }
    +      }
    +      dataSrcVecMap.put(ds, outputMapVector);*/
    +    }
    +    //Now create NDV in the outgoing container which was not avaliable in the incoming
    +    for (VectorWrapper<?> vw : incoming) {
    +      if (vw.getField().getLastName().equalsIgnoreCase("sum_width")) {//NullableFloat8 type
    +        addVectorToOutgoingContainer("ndv", vw, collector);
    +        break;
    +      }
    +    }
    +    container.setRecordCount(0);
    +    recordCount = 0;
    +    container.buildSchema(incoming.getSchema().getSelectionVectorMode());
    +  }
    +
    +  private void addVectorToOutgoingContainer(String field, VectorWrapper vw, ErrorCollector collector)
    +     throws SchemaChangeException {
    +    // Input map vector
    +    MapVector mapVector = (MapVector) vw.getValueVector();
    +    MaterializedField mf;
    +    assert mapVector.getPrimitiveVectors().size() > 0;
    +    // Proceed to create output map vector with same name e.g. statcount etc.
    +    MajorType mt = mapVector.getField().getType();
    +    if (functions.get(field) != null) {
    +      mf = MaterializedField.create(functions.get(field), mt);
    +    } else {
    +      mf = MaterializedField.create(field, mt);
    +    }
    +    assert !dataSrcVecMap.containsKey(mf);
    +    ValueVector vector = TypeHelper.getNewVector(mf, oContext.getAllocator());
    +    container.add(vector);
    +    MapVector outputMapVector = (MapVector) vector;
    +
    +    for (ValueVector vv : mapVector) {
    +      String fieldName = vv.getField().getLastName();
    +      if (!keyList.contains(fieldName)) {
    +        throw new UnsupportedOperationException("Unpivot data vector " +
    +                field + " contains key " + fieldName + " not contained in key source!");
    +      }
    +      if (vv.getField().getType().getMinorType() == TypeProtos.MinorType.MAP) {
    +        throw new UnsupportedOperationException("Unpivot of nested map is not supported!");
    +      }
    +      if (field.equals("column")) {
    +        outputMapVector.addOrGet(fieldName, vv.getField().getType(), vv.getClass());
    +      } else {
    +        List<LogicalExpression> args = Lists.newArrayList();
    +        LogicalExpression call;
    +        //TODO: Something else to access value of col such as emp_id?
    +        args.add(SchemaPath.getSimplePath(vv.getField().getPath()));
    +        //TODO: Put in the mapVector
    +        if (functions.get(field) != null) {
    +          call = FunctionCallFactory.createExpression(functions.get(field), args);
    +        } else {
    +          call = FunctionCallFactory.createExpression(field, args);
    +        }
    +        //TODO: Is this sufficient to add to new Map?
    +        ValueVector vector1 = addMapVector(fieldName, outputMapVector, call);
    +        if (collector.hasErrors()) {
    +          throw new SchemaChangeException("Failure while materializing expression. "
    +                  + collector.toErrorString());
    +        }
    +      }
    +    }
    +  }
    +
    +  @Override
    +  protected boolean setupNewSchema() throws SchemaChangeException {
    +    container.clear();
    +    buildKeyList();
    +    buildOutputContainer();
    +    return true;
    +  }
    +
    +  @Override
    +  protected IterOutcome doWork() {
    +    int outRecordCount = incoming.getRecordCount();
    +    HashMap<String, Long> nonNullRows = new HashMap<>();
    +
    +    for (VectorWrapper<?> vw : incoming) {
    +      ValueVector vv = vw.getValueVector();
    +      if (vv.getField().getType().getMinorType() != TypeProtos.MinorType.MAP) {
    +        //We only expect Maps in the incoming. STOP, if this is not the case.
    +        return IterOutcome.STOP;
    +      }
    +      MapVector mapVec = (MapVector) vv;
    +      Map<String, ValueHolder> statMap = aggregationMap.get(vv.getField().getLastName());
    +
    +      for (ValueVector mapElt : mapVec) {
    +        if (vv.getField().getLastName().equalsIgnoreCase("column")) {
    --- End diff --
    
    Yes, we do not need `.equalsIgnoreCase`. Created `public static final` strings for the hard-coded strings.


---
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 #729: Drill 1328: Support table statistics for Parquet

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

    https://github.com/apache/drill/pull/729#discussion_r103618796
  
    --- Diff: exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/statistics/AvgWidthMergedStatistic.java ---
    @@ -0,0 +1,135 @@
    +/*
    + * 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.impl.statistics;
    +
    +import org.apache.drill.common.types.TypeProtos;
    +import org.apache.drill.exec.expr.holders.NullableFloat8Holder;
    +import org.apache.drill.exec.expr.holders.ValueHolder;
    +import org.apache.drill.exec.vector.NullableFloat8Vector;
    +import org.apache.drill.exec.vector.ValueVector;
    +import org.apache.drill.exec.vector.complex.MapVector;
    +
    +import java.util.HashMap;
    +import java.util.List;
    +import java.util.Map;
    +
    +public class AvgWidthMergedStatistic extends AbstractMergedStatistic {
    +
    +  private String name;
    +  private String inputName;
    +  private boolean configureComplete = false;
    +  private boolean mergeComplete = false;
    +  private Map<String, ValueHolder> sumHolder;
    +  MergedStatistic types, nonNullStatCounts, statCounts;
    +
    +  public AvgWidthMergedStatistic (String name, String inputName) {
    +    this.name = name;
    +    this.inputName = inputName;
    +    this.sumHolder = new HashMap<>();
    +    types = nonNullStatCounts = statCounts = null;
    +  }
    +
    +  @Override
    +  public String getName() {
    +    return name;
    +  }
    +
    +  @Override
    +  public String getInput() {
    +    return inputName;
    +  }
    +
    +  @Override
    +  public void merge(ValueVector input) {
    +    // Check the input is a Map Vector
    +    assert (input.getField().getType().getMinorType() == TypeProtos.MinorType.MAP);
    +    MapVector inputMap = (MapVector) input;
    +    for (ValueVector vv : inputMap) {
    +      String colName = vv.getField().getLastName();
    +      NullableFloat8Holder colSumHolder;
    +      if (sumHolder.get(colName) != null) {
    +        colSumHolder = (NullableFloat8Holder) sumHolder.get(colName);
    +      } else {
    +        colSumHolder = new NullableFloat8Holder();
    +        sumHolder.put(colName, colSumHolder);
    +      }
    +      Object val = vv.getAccessor().getObject(0);
    +      if (val != null) {
    +        colSumHolder.value += (double) val;
    +        colSumHolder.isSet = 1;
    +      }
    +    }
    +  }
    +
    +  @Override
    +  public Object getStat(String colName) {
    +      if (mergeComplete != true) {
    +        throw new IllegalStateException(
    +            String.format("Statistic `%s` has not completed merging statistics", name));
    +      }
    +      NullableFloat8Holder colSumHolder = (NullableFloat8Holder) sumHolder.get(colName);
    +      return (long) (colSumHolder.value/ getRowCount(colName));
    +    }
    +
    +  @Override
    +  public void setOutput(ValueVector output) {
    +    // Check the input is a Map Vector
    +    assert (output.getField().getType().getMinorType() == TypeProtos.MinorType.MAP);
    +    // Dependencies have been configured correctly
    +    assert (configureComplete == true);
    +    MapVector outputMap = (MapVector) output;
    +
    +    for (ValueVector outMapCol : outputMap) {
    +      String colName = outMapCol.getField().getLastName();
    +      NullableFloat8Holder colSumHolder = (NullableFloat8Holder) sumHolder.get(colName);
    +      NullableFloat8Vector vv = (NullableFloat8Vector) outMapCol;
    +      vv.allocateNewSafe();
    +      vv.getMutator().setSafe(0, (colSumHolder.value / getRowCount(colName)));
    +    }
    +    mergeComplete = true;
    +  }
    +
    +  @Override
    +  public void configure(Object configurations) {
    +    List<MergedStatistic> statistics = (List<MergedStatistic>) configurations;
    --- 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 #729: Drill 1328: Support table statistics for Parquet

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/729#discussion_r100680743
  
    --- Diff: exec/java-exec/src/main/java/org/apache/drill/exec/physical/config/UnpivotMaps.java ---
    @@ -0,0 +1,59 @@
    +/**
    + * 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.config;
    +
    +import java.util.List;
    +
    +import org.apache.drill.exec.physical.base.AbstractSingle;
    +import org.apache.drill.exec.physical.base.PhysicalOperator;
    +import org.apache.drill.exec.physical.base.PhysicalVisitor;
    +import org.apache.drill.exec.proto.UserBitShared.CoreOperatorType;
    +
    +import com.fasterxml.jackson.annotation.JsonCreator;
    +import com.fasterxml.jackson.annotation.JsonProperty;
    +import com.fasterxml.jackson.annotation.JsonTypeName;
    +
    +@JsonTypeName("unpivot-maps")
    +public class UnpivotMaps extends AbstractSingle {
    --- End diff --
    
    Is this a generic new feature in Drill to unpivot any arbitrary map? Or, is this really `PivotStatistics` that is used only in statistics? (Note that there is no "pivot" defined anywhere to unpivot.) Or, should the name be even more descriptive and less implementation specific: "ConsolidateStatistics"?


---
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 #729: Drill 1328: Support table statistics for Parquet

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/729#discussion_r102875280
  
    --- Diff: exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/statistics/StatisticsAggBatch.java ---
    @@ -0,0 +1,230 @@
    +/**
    --- End diff --
    
    Stopped here for now, will continue with the rest tomorrow.


---
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 #729: Drill 1328: Support table statistics for Parquet

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/729#discussion_r100677264
  
    --- Diff: exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/statistics/StatisticsAggBatch.java ---
    @@ -0,0 +1,256 @@
    +/**
    + * 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.impl.statistics;
    +
    +import com.google.common.collect.Lists;
    +import com.sun.codemodel.JExpr;
    +import org.apache.drill.common.expression.ErrorCollector;
    +import org.apache.drill.common.expression.ErrorCollectorImpl;
    +import org.apache.drill.common.expression.FunctionCallFactory;
    +import org.apache.drill.common.expression.LogicalExpression;
    +import org.apache.drill.common.expression.SchemaPath;
    +import org.apache.drill.common.expression.ValueExpressions;
    +import org.apache.drill.exec.exception.ClassTransformationException;
    +import org.apache.drill.exec.exception.OutOfMemoryException;
    +import org.apache.drill.exec.exception.SchemaChangeException;
    +import org.apache.drill.exec.expr.ClassGenerator;
    +import org.apache.drill.exec.expr.CodeGenerator;
    +import org.apache.drill.exec.expr.ExpressionTreeMaterializer;
    +import org.apache.drill.exec.expr.TypeHelper;
    +import org.apache.drill.exec.expr.ValueVectorWriteExpression;
    +import org.apache.drill.exec.ops.FragmentContext;
    +import org.apache.drill.exec.physical.config.StatisticsAggregate;
    +import org.apache.drill.exec.physical.impl.aggregate.StreamingAggBatch;
    +import org.apache.drill.exec.physical.impl.aggregate.StreamingAggTemplate;
    +import org.apache.drill.exec.physical.impl.aggregate.StreamingAggregator;
    +import org.apache.drill.exec.planner.physical.StatsAggPrel.OperatorPhase;
    +import org.apache.drill.exec.record.BatchSchema.SelectionVectorMode;
    +import org.apache.drill.exec.record.MaterializedField;
    +import org.apache.drill.exec.record.RecordBatch;
    +import org.apache.drill.exec.record.TypedFieldId;
    +import org.apache.drill.exec.store.ImplicitColumnExplorer;
    +import org.apache.drill.exec.vector.ValueVector;
    +import org.apache.drill.exec.vector.complex.FieldIdUtil;
    +import org.apache.drill.exec.vector.complex.MapVector;
    +
    +import java.io.IOException;
    +import java.util.GregorianCalendar;
    +import java.util.List;
    +import java.util.TimeZone;
    +
    +/**
    + * TODO: This needs cleanup. Currently the key values are constants and we compare the constants
    + * for every record. Seems unnecessary.
    + *
    + * Example input and output:
    + * Schema of incoming batch: region_id (VARCHAR), sales_city (VARCHAR), cnt (BIGINT)
    + * Schema of output:
    + *    "schema" : BIGINT - Schema number. For each schema change this number is incremented.
    + *    "computed" : BIGINT - What time is it computed?
    + *    "columns"       : MAP - Column names
    + *       "region_id"  : VARCHAR
    + *       "sales_city" : VARCHAR
    + *       "cnt"        : VARCHAR
    + *    "statscount" : MAP
    + *       "region_id"  : BIGINT - statscount(region_id) - aggregation over all values of region_id
    + *                      in incoming batch
    + *       "sales_city" : BIGINT - statscount(sales_city)
    + *       "cnt"        : BIGINT - statscount(cnt)
    + *    "nonnullstatcount" : MAP
    + *       "region_id"  : BIGINT - nonnullstatcount(region_id)
    + *       "sales_city" : BIGINT - nonnullstatcount(sales_city)
    + *       "cnt"        : BIGINT - nonnullstatcount(cnt)
    + *   .... another map for next stats function ....
    + */
    +public class StatisticsAggBatch extends StreamingAggBatch {
    +  private List<String> functions;
    +  private int schema = 0;
    +
    +  public StatisticsAggBatch(StatisticsAggregate popConfig, RecordBatch incoming,
    +      FragmentContext context) throws OutOfMemoryException {
    +    super(popConfig, incoming, context);
    +    this.functions = popConfig.getFunctions();
    +  }
    +
    +  private void createKeyColumn(String name, LogicalExpression expr, List<LogicalExpression> keyExprs,
    +      List<TypedFieldId> keyOutputIds) throws SchemaChangeException {
    +    ErrorCollector collector = new ErrorCollectorImpl();
    +
    +    LogicalExpression mle = ExpressionTreeMaterializer.materialize(expr, incoming, collector,
    +        context.getFunctionRegistry());
    +
    +    MaterializedField outputField = MaterializedField.create(name, mle.getMajorType());
    +    ValueVector vector = TypeHelper.getNewVector(outputField, oContext.getAllocator());
    +
    +    keyExprs.add(mle);
    +    keyOutputIds.add(container.add(vector));
    +
    +    if (collector.hasErrors()) {
    +      throw new SchemaChangeException("Failure while materializing expression. "
    +          + collector.toErrorString());
    +    }
    +  }
    +
    +  private void createNestedKeyColumn(MapVector parent, String name, LogicalExpression expr,
    +      List<LogicalExpression> keyExprs, List<TypedFieldId> keyOutputIds)
    +          throws SchemaChangeException {
    +    ErrorCollector collector = new ErrorCollectorImpl();
    +
    +    LogicalExpression mle = ExpressionTreeMaterializer.materialize(expr, incoming, collector,
    +        context.getFunctionRegistry());
    +
    +    Class<? extends ValueVector> vvc =
    +        TypeHelper.getValueVectorClass(mle.getMajorType().getMinorType(),
    +            mle.getMajorType().getMode());
    +
    +    ValueVector vv = parent.addOrGet(name, mle.getMajorType(), vvc);
    +
    +    TypedFieldId pfid = container.getValueVectorId(SchemaPath.getSimplePath(parent.getField().getPath()));
    +    assert pfid.getFieldIds().length == 1;
    +    TypedFieldId.Builder builder = TypedFieldId.newBuilder();
    +    builder.addId(pfid.getFieldIds()[0]);
    +    TypedFieldId id =
    +        FieldIdUtil.getFieldIdIfMatches(parent, builder, true,
    +            SchemaPath.getSimplePath(vv.getField().getPath()).getRootSegment());
    +
    +    keyExprs.add(mle);
    +    keyOutputIds.add(id);
    +
    +    if (collector.hasErrors()) {
    +      throw new SchemaChangeException("Failure while materializing expression. "
    +          + collector.toErrorString());
    +    }
    +  }
    +
    +  private void addMapVector(String name, MapVector parent, LogicalExpression expr,
    +      List<LogicalExpression> valueExprs) throws SchemaChangeException {
    +    ErrorCollector collector = new ErrorCollectorImpl();
    --- End diff --
    
    There is quite a bit of duplication across these three methods. While reuse by copy & paste has a long and storied tradition, it is actually easier to maintain code if we can move the repeated bits into functions that are called where needed. Also means us poor reviewers don't have to review the same code multiple times looking for subtle differences...


---
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 #729: Drill 1328: Support table statistics for Parquet

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

    https://github.com/apache/drill/pull/729#discussion_r100416219
  
    --- Diff: exec/java-exec/src/main/java/org/apache/drill/exec/planner/physical/NestedLoopJoinPrule.java ---
    @@ -84,8 +88,14 @@ public void onMatch(RelOptRuleCall call) {
         if (!settings.isNestedLoopJoinEnabled()) {
           return;
         }
    -
    -    final DrillJoinRel join = (DrillJoinRel) call.rel(0);
    +    int[] joinFields = new int[2];
    +    DrillJoinRel join = (DrillJoinRel) call.rel(0);
    +    // If right outer join on simply equi join convert it to left outer join. We only support left outer NLJ as of now
    +    if (join.getJoinType() == JoinRelType.RIGHT
    --- End diff --
    
    Created Drill-5214


---
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 #729: Drill 1328: Support table statistics for Parquet

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

    https://github.com/apache/drill/pull/729#discussion_r103611152
  
    --- Diff: exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/statistics/StatisticsAggBatch.java ---
    @@ -0,0 +1,256 @@
    +/**
    + * 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.impl.statistics;
    +
    +import com.google.common.collect.Lists;
    +import com.sun.codemodel.JExpr;
    +import org.apache.drill.common.expression.ErrorCollector;
    +import org.apache.drill.common.expression.ErrorCollectorImpl;
    +import org.apache.drill.common.expression.FunctionCallFactory;
    +import org.apache.drill.common.expression.LogicalExpression;
    +import org.apache.drill.common.expression.SchemaPath;
    +import org.apache.drill.common.expression.ValueExpressions;
    +import org.apache.drill.exec.exception.ClassTransformationException;
    +import org.apache.drill.exec.exception.OutOfMemoryException;
    +import org.apache.drill.exec.exception.SchemaChangeException;
    +import org.apache.drill.exec.expr.ClassGenerator;
    +import org.apache.drill.exec.expr.CodeGenerator;
    +import org.apache.drill.exec.expr.ExpressionTreeMaterializer;
    +import org.apache.drill.exec.expr.TypeHelper;
    +import org.apache.drill.exec.expr.ValueVectorWriteExpression;
    +import org.apache.drill.exec.ops.FragmentContext;
    +import org.apache.drill.exec.physical.config.StatisticsAggregate;
    +import org.apache.drill.exec.physical.impl.aggregate.StreamingAggBatch;
    +import org.apache.drill.exec.physical.impl.aggregate.StreamingAggTemplate;
    +import org.apache.drill.exec.physical.impl.aggregate.StreamingAggregator;
    +import org.apache.drill.exec.planner.physical.StatsAggPrel.OperatorPhase;
    +import org.apache.drill.exec.record.BatchSchema.SelectionVectorMode;
    +import org.apache.drill.exec.record.MaterializedField;
    +import org.apache.drill.exec.record.RecordBatch;
    +import org.apache.drill.exec.record.TypedFieldId;
    +import org.apache.drill.exec.store.ImplicitColumnExplorer;
    +import org.apache.drill.exec.vector.ValueVector;
    +import org.apache.drill.exec.vector.complex.FieldIdUtil;
    +import org.apache.drill.exec.vector.complex.MapVector;
    +
    +import java.io.IOException;
    +import java.util.GregorianCalendar;
    +import java.util.List;
    +import java.util.TimeZone;
    +
    +/**
    + * TODO: This needs cleanup. Currently the key values are constants and we compare the constants
    + * for every record. Seems unnecessary.
    --- End diff --
    
    Refactored out the Javadoc comment since it includes a TODO tag


---
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 #729: Drill 1328: Support table statistics for Parquet

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

    https://github.com/apache/drill/pull/729#discussion_r103403068
  
    --- Diff: exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/statistics/AvgWidthMergedStatistic.java ---
    @@ -0,0 +1,135 @@
    +/*
    + * 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.impl.statistics;
    +
    +import org.apache.drill.common.types.TypeProtos;
    +import org.apache.drill.exec.expr.holders.NullableFloat8Holder;
    +import org.apache.drill.exec.expr.holders.ValueHolder;
    +import org.apache.drill.exec.vector.NullableFloat8Vector;
    +import org.apache.drill.exec.vector.ValueVector;
    +import org.apache.drill.exec.vector.complex.MapVector;
    +
    +import java.util.HashMap;
    +import java.util.List;
    +import java.util.Map;
    +
    +public class AvgWidthMergedStatistic extends AbstractMergedStatistic {
    +
    +  private String name;
    +  private String inputName;
    +  private boolean configureComplete = false;
    +  private boolean mergeComplete = false;
    +  private Map<String, ValueHolder> sumHolder;
    +  MergedStatistic types, nonNullStatCounts, statCounts;
    +
    +  public AvgWidthMergedStatistic (String name, String inputName) {
    +    this.name = name;
    +    this.inputName = inputName;
    +    this.sumHolder = new HashMap<>();
    +    types = nonNullStatCounts = statCounts = null;
    +  }
    +
    +  @Override
    +  public String getName() {
    +    return name;
    +  }
    +
    +  @Override
    +  public String getInput() {
    +    return inputName;
    +  }
    +
    +  @Override
    +  public void merge(ValueVector input) {
    +    // Check the input is a Map Vector
    +    assert (input.getField().getType().getMinorType() == TypeProtos.MinorType.MAP);
    +    MapVector inputMap = (MapVector) input;
    +    for (ValueVector vv : inputMap) {
    +      String colName = vv.getField().getLastName();
    +      NullableFloat8Holder colSumHolder;
    +      if (sumHolder.get(colName) != null) {
    +        colSumHolder = (NullableFloat8Holder) sumHolder.get(colName);
    +      } else {
    +        colSumHolder = new NullableFloat8Holder();
    +        sumHolder.put(colName, colSumHolder);
    +      }
    +      Object val = vv.getAccessor().getObject(0);
    +      if (val != null) {
    +        colSumHolder.value += (double) val;
    +        colSumHolder.isSet = 1;
    +      }
    +    }
    +  }
    +
    +  @Override
    +  public Object getStat(String colName) {
    +      if (mergeComplete != true) {
    +        throw new IllegalStateException(
    +            String.format("Statistic `%s` has not completed merging statistics", name));
    +      }
    +      NullableFloat8Holder colSumHolder = (NullableFloat8Holder) sumHolder.get(colName);
    +      return (long) (colSumHolder.value/ getRowCount(colName));
    +    }
    +
    +  @Override
    +  public void setOutput(ValueVector output) {
    +    // Check the input is a Map Vector
    +    assert (output.getField().getType().getMinorType() == TypeProtos.MinorType.MAP);
    +    // Dependencies have been configured correctly
    +    assert (configureComplete == true);
    +    MapVector outputMap = (MapVector) output;
    +
    +    for (ValueVector outMapCol : outputMap) {
    +      String colName = outMapCol.getField().getLastName();
    +      NullableFloat8Holder colSumHolder = (NullableFloat8Holder) sumHolder.get(colName);
    +      NullableFloat8Vector vv = (NullableFloat8Vector) outMapCol;
    +      vv.allocateNewSafe();
    +      vv.getMutator().setSafe(0, (colSumHolder.value / getRowCount(colName)));
    +    }
    --- End diff --
    
    Since we create these VVs in the StatisticsMergeBatch, we will only create one per statistic per column. We can redesign this in a subsequent implementation - not sure how we can get by without generating a VV or maybe you mean generate less VVs?


---
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 #729: Drill 1328: Support table statistics for Parquet

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

    https://github.com/apache/drill/pull/729#discussion_r101173499
  
    --- Diff: exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/statistics/StatisticsAggBatch.java ---
    @@ -0,0 +1,256 @@
    +/**
    + * 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.impl.statistics;
    +
    +import com.google.common.collect.Lists;
    +import com.sun.codemodel.JExpr;
    +import org.apache.drill.common.expression.ErrorCollector;
    +import org.apache.drill.common.expression.ErrorCollectorImpl;
    +import org.apache.drill.common.expression.FunctionCallFactory;
    +import org.apache.drill.common.expression.LogicalExpression;
    +import org.apache.drill.common.expression.SchemaPath;
    +import org.apache.drill.common.expression.ValueExpressions;
    +import org.apache.drill.exec.exception.ClassTransformationException;
    +import org.apache.drill.exec.exception.OutOfMemoryException;
    +import org.apache.drill.exec.exception.SchemaChangeException;
    +import org.apache.drill.exec.expr.ClassGenerator;
    +import org.apache.drill.exec.expr.CodeGenerator;
    +import org.apache.drill.exec.expr.ExpressionTreeMaterializer;
    +import org.apache.drill.exec.expr.TypeHelper;
    +import org.apache.drill.exec.expr.ValueVectorWriteExpression;
    +import org.apache.drill.exec.ops.FragmentContext;
    +import org.apache.drill.exec.physical.config.StatisticsAggregate;
    +import org.apache.drill.exec.physical.impl.aggregate.StreamingAggBatch;
    +import org.apache.drill.exec.physical.impl.aggregate.StreamingAggTemplate;
    +import org.apache.drill.exec.physical.impl.aggregate.StreamingAggregator;
    +import org.apache.drill.exec.planner.physical.StatsAggPrel.OperatorPhase;
    +import org.apache.drill.exec.record.BatchSchema.SelectionVectorMode;
    +import org.apache.drill.exec.record.MaterializedField;
    +import org.apache.drill.exec.record.RecordBatch;
    +import org.apache.drill.exec.record.TypedFieldId;
    +import org.apache.drill.exec.store.ImplicitColumnExplorer;
    +import org.apache.drill.exec.vector.ValueVector;
    +import org.apache.drill.exec.vector.complex.FieldIdUtil;
    +import org.apache.drill.exec.vector.complex.MapVector;
    +
    +import java.io.IOException;
    +import java.util.GregorianCalendar;
    +import java.util.List;
    +import java.util.TimeZone;
    +
    +/**
    + * TODO: This needs cleanup. Currently the key values are constants and we compare the constants
    + * for every record. Seems unnecessary.
    + *
    + * Example input and output:
    + * Schema of incoming batch: region_id (VARCHAR), sales_city (VARCHAR), cnt (BIGINT)
    + * Schema of output:
    + *    "schema" : BIGINT - Schema number. For each schema change this number is incremented.
    + *    "computed" : BIGINT - What time is it computed?
    + *    "columns"       : MAP - Column names
    + *       "region_id"  : VARCHAR
    + *       "sales_city" : VARCHAR
    + *       "cnt"        : VARCHAR
    + *    "statscount" : MAP
    + *       "region_id"  : BIGINT - statscount(region_id) - aggregation over all values of region_id
    + *                      in incoming batch
    + *       "sales_city" : BIGINT - statscount(sales_city)
    + *       "cnt"        : BIGINT - statscount(cnt)
    + *    "nonnullstatcount" : MAP
    + *       "region_id"  : BIGINT - nonnullstatcount(region_id)
    + *       "sales_city" : BIGINT - nonnullstatcount(sales_city)
    + *       "cnt"        : BIGINT - nonnullstatcount(cnt)
    + *   .... another map for next stats function ....
    + */
    +public class StatisticsAggBatch extends StreamingAggBatch {
    +  private List<String> functions;
    +  private int schema = 0;
    +
    +  public StatisticsAggBatch(StatisticsAggregate popConfig, RecordBatch incoming,
    +      FragmentContext context) throws OutOfMemoryException {
    +    super(popConfig, incoming, context);
    +    this.functions = popConfig.getFunctions();
    +  }
    +
    +  private void createKeyColumn(String name, LogicalExpression expr, List<LogicalExpression> keyExprs,
    +      List<TypedFieldId> keyOutputIds) throws SchemaChangeException {
    +    ErrorCollector collector = new ErrorCollectorImpl();
    +
    +    LogicalExpression mle = ExpressionTreeMaterializer.materialize(expr, incoming, collector,
    +        context.getFunctionRegistry());
    +
    +    MaterializedField outputField = MaterializedField.create(name, mle.getMajorType());
    +    ValueVector vector = TypeHelper.getNewVector(outputField, oContext.getAllocator());
    +
    +    keyExprs.add(mle);
    +    keyOutputIds.add(container.add(vector));
    +
    +    if (collector.hasErrors()) {
    +      throw new SchemaChangeException("Failure while materializing expression. "
    +          + collector.toErrorString());
    +    }
    +  }
    +
    +  private void createNestedKeyColumn(MapVector parent, String name, LogicalExpression expr,
    +      List<LogicalExpression> keyExprs, List<TypedFieldId> keyOutputIds)
    +          throws SchemaChangeException {
    +    ErrorCollector collector = new ErrorCollectorImpl();
    +
    +    LogicalExpression mle = ExpressionTreeMaterializer.materialize(expr, incoming, collector,
    +        context.getFunctionRegistry());
    +
    +    Class<? extends ValueVector> vvc =
    +        TypeHelper.getValueVectorClass(mle.getMajorType().getMinorType(),
    +            mle.getMajorType().getMode());
    +
    +    ValueVector vv = parent.addOrGet(name, mle.getMajorType(), vvc);
    +
    +    TypedFieldId pfid = container.getValueVectorId(SchemaPath.getSimplePath(parent.getField().getPath()));
    +    assert pfid.getFieldIds().length == 1;
    +    TypedFieldId.Builder builder = TypedFieldId.newBuilder();
    +    builder.addId(pfid.getFieldIds()[0]);
    +    TypedFieldId id =
    +        FieldIdUtil.getFieldIdIfMatches(parent, builder, true,
    +            SchemaPath.getSimplePath(vv.getField().getPath()).getRootSegment());
    +
    +    keyExprs.add(mle);
    +    keyOutputIds.add(id);
    +
    +    if (collector.hasErrors()) {
    +      throw new SchemaChangeException("Failure while materializing expression. "
    +          + collector.toErrorString());
    +    }
    +  }
    +
    +  private void addMapVector(String name, MapVector parent, LogicalExpression expr,
    +      List<LogicalExpression> valueExprs) throws SchemaChangeException {
    +    ErrorCollector collector = new ErrorCollectorImpl();
    +
    +    LogicalExpression mle = ExpressionTreeMaterializer.materialize(expr, incoming, collector,
    +        context.getFunctionRegistry());
    +
    +    Class<? extends ValueVector> vvc =
    +        TypeHelper.getValueVectorClass(mle.getMajorType().getMinorType(), mle.getMajorType().getMode());
    +    ValueVector vv = parent.addOrGet(name, mle.getMajorType(), vvc);
    +
    +    TypedFieldId pfid = container.getValueVectorId(SchemaPath.getSimplePath(parent.getField().getPath()));
    +    assert pfid.getFieldIds().length == 1;
    +    TypedFieldId.Builder builder = TypedFieldId.newBuilder();
    +    builder.addId(pfid.getFieldIds()[0]);
    +    TypedFieldId id = FieldIdUtil.getFieldIdIfMatches(parent, builder, true,
    +        SchemaPath.getSimplePath(vv.getField().getPath()).getRootSegment());
    +
    +    valueExprs.add(new ValueVectorWriteExpression(id, mle, true));
    +
    +    if (collector.hasErrors()) {
    +      throw new SchemaChangeException("Failure while materializing expression. "
    +          + collector.toErrorString());
    +    }
    +  }
    +
    +  private StreamingAggregator codegenAggregator(List<LogicalExpression> keyExprs,
    +      List<LogicalExpression> valueExprs, List<TypedFieldId> keyOutputIds)
    +          throws SchemaChangeException, ClassTransformationException, IOException {
    +    ClassGenerator<StreamingAggregator> cg =
    +        CodeGenerator.getRoot(StreamingAggTemplate.TEMPLATE_DEFINITION, context.getFunctionRegistry(),
    +            context.getOptions());
    +
    +    LogicalExpression[] keyExprsArray = new LogicalExpression[keyExprs.size()];
    +    LogicalExpression[] valueExprsArray = new LogicalExpression[valueExprs.size()];
    +    TypedFieldId[] keyOutputIdsArray = new TypedFieldId[keyOutputIds.size()];
    +
    +    keyExprs.toArray(keyExprsArray);
    +    valueExprs.toArray(valueExprsArray);
    +    keyOutputIds.toArray(keyOutputIdsArray);
    +
    +    setupIsSame(cg, keyExprsArray);
    +    setupIsSameApart(cg, keyExprsArray);
    +    addRecordValues(cg, valueExprsArray);
    +    outputRecordKeys(cg, keyOutputIdsArray, keyExprsArray);
    +    outputRecordKeysPrev(cg, keyOutputIdsArray, keyExprsArray);
    +
    +    cg.getBlock("resetValues")._return(JExpr.TRUE);
    +    getIndex(cg);
    +
    +    container.buildSchema(SelectionVectorMode.NONE);
    +    StreamingAggregator agg = context.getImplementationClass(cg);
    +    agg.setup(oContext, incoming, this);
    +    return agg;
    +  }
    +
    +  private boolean isImplicitFileColumn(MaterializedField mf) {
    +    return ImplicitColumnExplorer.initImplicitFileColumns(context.getOptions()).get(mf.getName()) != null;
    +  }
    +
    +  protected StreamingAggregator createAggregatorInternal()
    +      throws SchemaChangeException, ClassTransformationException, IOException {
    +    container.clear();
    +
    +    List<LogicalExpression> keyExprs = Lists.newArrayList();
    +    List<LogicalExpression> valueExprs = Lists.newArrayList();
    +    List<TypedFieldId> keyOutputIds = Lists.newArrayList();
    +    GregorianCalendar calendar = new GregorianCalendar(TimeZone.getTimeZone("UTC"));
    +    calendar.setTimeInMillis(System.currentTimeMillis());
    +
    +    if (this.getPopConfig() instanceof StatisticsAggregate
    +        && (((StatisticsAggregate) this.getPopConfig()).getPhase() == OperatorPhase.PHASE_1of1
    +            || ((StatisticsAggregate) this.getPopConfig()).getPhase() == OperatorPhase.PHASE_2of2)) {
    +      createKeyColumn("schema",
    +          ValueExpressions.getBigInt(schema++),
    +          keyExprs,
    +          keyOutputIds
    +      );
    +      createKeyColumn("computed",
    +          ValueExpressions.getDate(calendar),
    +          keyExprs,
    +          keyOutputIds
    +      );
    +    }
    +
    --- End diff --
    
    Refactored out some unused code. Added comments for each block for clarity. I think the code is small (~20 lines excluding comments) and simple enough to not warrant refactoring into separate functions. The code generation is already in a separate function.


---
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 #729: Drill 1328: Support table statistics for Parquet

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

    https://github.com/apache/drill/pull/729#discussion_r103366521
  
    --- Diff: exec/java-exec/src/main/java/org/apache/drill/exec/ops/FragmentContext.java ---
    @@ -245,6 +247,18 @@ public SchemaPlus getRootSchema() {
       }
     
       /**
    +   * Returns the statement type (e.g. SELECT, CTAS, ANALYZE) from the query context.
    +   * @return query statement type {@link SqlStatementType}, if known.
    +   */
    +  public SqlStatementType getSQLStatementType() {
    +    if (queryContext == null) {
    +      fail(new UnsupportedOperationException("Statement type is only valid for root fragment. " +
    --- 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 #729: Drill 1328: Support table statistics for Parquet

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/729#discussion_r100676900
  
    --- Diff: exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/statistics/StatisticsAggBatch.java ---
    @@ -0,0 +1,256 @@
    +/**
    + * 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.impl.statistics;
    +
    +import com.google.common.collect.Lists;
    +import com.sun.codemodel.JExpr;
    +import org.apache.drill.common.expression.ErrorCollector;
    +import org.apache.drill.common.expression.ErrorCollectorImpl;
    +import org.apache.drill.common.expression.FunctionCallFactory;
    +import org.apache.drill.common.expression.LogicalExpression;
    +import org.apache.drill.common.expression.SchemaPath;
    +import org.apache.drill.common.expression.ValueExpressions;
    +import org.apache.drill.exec.exception.ClassTransformationException;
    +import org.apache.drill.exec.exception.OutOfMemoryException;
    +import org.apache.drill.exec.exception.SchemaChangeException;
    +import org.apache.drill.exec.expr.ClassGenerator;
    +import org.apache.drill.exec.expr.CodeGenerator;
    +import org.apache.drill.exec.expr.ExpressionTreeMaterializer;
    +import org.apache.drill.exec.expr.TypeHelper;
    +import org.apache.drill.exec.expr.ValueVectorWriteExpression;
    +import org.apache.drill.exec.ops.FragmentContext;
    +import org.apache.drill.exec.physical.config.StatisticsAggregate;
    +import org.apache.drill.exec.physical.impl.aggregate.StreamingAggBatch;
    +import org.apache.drill.exec.physical.impl.aggregate.StreamingAggTemplate;
    +import org.apache.drill.exec.physical.impl.aggregate.StreamingAggregator;
    +import org.apache.drill.exec.planner.physical.StatsAggPrel.OperatorPhase;
    +import org.apache.drill.exec.record.BatchSchema.SelectionVectorMode;
    +import org.apache.drill.exec.record.MaterializedField;
    +import org.apache.drill.exec.record.RecordBatch;
    +import org.apache.drill.exec.record.TypedFieldId;
    +import org.apache.drill.exec.store.ImplicitColumnExplorer;
    +import org.apache.drill.exec.vector.ValueVector;
    +import org.apache.drill.exec.vector.complex.FieldIdUtil;
    +import org.apache.drill.exec.vector.complex.MapVector;
    +
    +import java.io.IOException;
    +import java.util.GregorianCalendar;
    +import java.util.List;
    +import java.util.TimeZone;
    +
    +/**
    + * TODO: This needs cleanup. Currently the key values are constants and we compare the constants
    + * for every record. Seems unnecessary.
    --- End diff --
    
    Thanks for the explanation!
    
    This is a Javadoc comment. Please use HTML to format. \<p> to separate paragraphs, \<ul>\<li>\</li>\</ul> for bulleted lists, etc.


---
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 #729: Drill 1328: Support table statistics for Parquet

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

    https://github.com/apache/drill/pull/729#discussion_r100863450
  
    --- Diff: exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/statistics/StatisticsAggBatch.java ---
    @@ -0,0 +1,256 @@
    +/**
    + * 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.impl.statistics;
    +
    +import com.google.common.collect.Lists;
    +import com.sun.codemodel.JExpr;
    +import org.apache.drill.common.expression.ErrorCollector;
    +import org.apache.drill.common.expression.ErrorCollectorImpl;
    +import org.apache.drill.common.expression.FunctionCallFactory;
    +import org.apache.drill.common.expression.LogicalExpression;
    +import org.apache.drill.common.expression.SchemaPath;
    +import org.apache.drill.common.expression.ValueExpressions;
    +import org.apache.drill.exec.exception.ClassTransformationException;
    +import org.apache.drill.exec.exception.OutOfMemoryException;
    +import org.apache.drill.exec.exception.SchemaChangeException;
    +import org.apache.drill.exec.expr.ClassGenerator;
    +import org.apache.drill.exec.expr.CodeGenerator;
    +import org.apache.drill.exec.expr.ExpressionTreeMaterializer;
    +import org.apache.drill.exec.expr.TypeHelper;
    +import org.apache.drill.exec.expr.ValueVectorWriteExpression;
    +import org.apache.drill.exec.ops.FragmentContext;
    +import org.apache.drill.exec.physical.config.StatisticsAggregate;
    +import org.apache.drill.exec.physical.impl.aggregate.StreamingAggBatch;
    +import org.apache.drill.exec.physical.impl.aggregate.StreamingAggTemplate;
    +import org.apache.drill.exec.physical.impl.aggregate.StreamingAggregator;
    +import org.apache.drill.exec.planner.physical.StatsAggPrel.OperatorPhase;
    +import org.apache.drill.exec.record.BatchSchema.SelectionVectorMode;
    +import org.apache.drill.exec.record.MaterializedField;
    +import org.apache.drill.exec.record.RecordBatch;
    +import org.apache.drill.exec.record.TypedFieldId;
    +import org.apache.drill.exec.store.ImplicitColumnExplorer;
    +import org.apache.drill.exec.vector.ValueVector;
    +import org.apache.drill.exec.vector.complex.FieldIdUtil;
    +import org.apache.drill.exec.vector.complex.MapVector;
    +
    +import java.io.IOException;
    +import java.util.GregorianCalendar;
    +import java.util.List;
    +import java.util.TimeZone;
    +
    +/**
    + * TODO: This needs cleanup. Currently the key values are constants and we compare the constants
    + * for every record. Seems unnecessary.
    + *
    + * Example input and output:
    + * Schema of incoming batch: region_id (VARCHAR), sales_city (VARCHAR), cnt (BIGINT)
    + * Schema of output:
    + *    "schema" : BIGINT - Schema number. For each schema change this number is incremented.
    + *    "computed" : BIGINT - What time is it computed?
    + *    "columns"       : MAP - Column names
    + *       "region_id"  : VARCHAR
    + *       "sales_city" : VARCHAR
    + *       "cnt"        : VARCHAR
    + *    "statscount" : MAP
    + *       "region_id"  : BIGINT - statscount(region_id) - aggregation over all values of region_id
    + *                      in incoming batch
    + *       "sales_city" : BIGINT - statscount(sales_city)
    + *       "cnt"        : BIGINT - statscount(cnt)
    + *    "nonnullstatcount" : MAP
    + *       "region_id"  : BIGINT - nonnullstatcount(region_id)
    + *       "sales_city" : BIGINT - nonnullstatcount(sales_city)
    + *       "cnt"        : BIGINT - nonnullstatcount(cnt)
    + *   .... another map for next stats function ....
    + */
    +public class StatisticsAggBatch extends StreamingAggBatch {
    +  private List<String> functions;
    +  private int schema = 0;
    +
    +  public StatisticsAggBatch(StatisticsAggregate popConfig, RecordBatch incoming,
    +      FragmentContext context) throws OutOfMemoryException {
    +    super(popConfig, incoming, context);
    +    this.functions = popConfig.getFunctions();
    +  }
    +
    +  private void createKeyColumn(String name, LogicalExpression expr, List<LogicalExpression> keyExprs,
    +      List<TypedFieldId> keyOutputIds) throws SchemaChangeException {
    +    ErrorCollector collector = new ErrorCollectorImpl();
    +
    +    LogicalExpression mle = ExpressionTreeMaterializer.materialize(expr, incoming, collector,
    +        context.getFunctionRegistry());
    +
    +    MaterializedField outputField = MaterializedField.create(name, mle.getMajorType());
    +    ValueVector vector = TypeHelper.getNewVector(outputField, oContext.getAllocator());
    +
    +    keyExprs.add(mle);
    +    keyOutputIds.add(container.add(vector));
    +
    +    if (collector.hasErrors()) {
    +      throw new SchemaChangeException("Failure while materializing expression. "
    +          + collector.toErrorString());
    +    }
    +  }
    +
    +  private void createNestedKeyColumn(MapVector parent, String name, LogicalExpression expr,
    +      List<LogicalExpression> keyExprs, List<TypedFieldId> keyOutputIds)
    +          throws SchemaChangeException {
    +    ErrorCollector collector = new ErrorCollectorImpl();
    +
    +    LogicalExpression mle = ExpressionTreeMaterializer.materialize(expr, incoming, collector,
    +        context.getFunctionRegistry());
    +
    +    Class<? extends ValueVector> vvc =
    +        TypeHelper.getValueVectorClass(mle.getMajorType().getMinorType(),
    +            mle.getMajorType().getMode());
    +
    +    ValueVector vv = parent.addOrGet(name, mle.getMajorType(), vvc);
    +
    +    TypedFieldId pfid = container.getValueVectorId(SchemaPath.getSimplePath(parent.getField().getPath()));
    +    assert pfid.getFieldIds().length == 1;
    +    TypedFieldId.Builder builder = TypedFieldId.newBuilder();
    +    builder.addId(pfid.getFieldIds()[0]);
    +    TypedFieldId id =
    +        FieldIdUtil.getFieldIdIfMatches(parent, builder, true,
    +            SchemaPath.getSimplePath(vv.getField().getPath()).getRootSegment());
    +
    +    keyExprs.add(mle);
    +    keyOutputIds.add(id);
    +
    +    if (collector.hasErrors()) {
    +      throw new SchemaChangeException("Failure while materializing expression. "
    +          + collector.toErrorString());
    +    }
    +  }
    +
    +  private void addMapVector(String name, MapVector parent, LogicalExpression expr,
    +      List<LogicalExpression> valueExprs) throws SchemaChangeException {
    +    ErrorCollector collector = new ErrorCollectorImpl();
    +
    +    LogicalExpression mle = ExpressionTreeMaterializer.materialize(expr, incoming, collector,
    +        context.getFunctionRegistry());
    +
    +    Class<? extends ValueVector> vvc =
    +        TypeHelper.getValueVectorClass(mle.getMajorType().getMinorType(), mle.getMajorType().getMode());
    +    ValueVector vv = parent.addOrGet(name, mle.getMajorType(), vvc);
    +
    +    TypedFieldId pfid = container.getValueVectorId(SchemaPath.getSimplePath(parent.getField().getPath()));
    +    assert pfid.getFieldIds().length == 1;
    +    TypedFieldId.Builder builder = TypedFieldId.newBuilder();
    +    builder.addId(pfid.getFieldIds()[0]);
    +    TypedFieldId id = FieldIdUtil.getFieldIdIfMatches(parent, builder, true,
    +        SchemaPath.getSimplePath(vv.getField().getPath()).getRootSegment());
    +
    +    valueExprs.add(new ValueVectorWriteExpression(id, mle, true));
    +
    +    if (collector.hasErrors()) {
    +      throw new SchemaChangeException("Failure while materializing expression. "
    +          + collector.toErrorString());
    +    }
    +  }
    +
    +  private StreamingAggregator codegenAggregator(List<LogicalExpression> keyExprs,
    +      List<LogicalExpression> valueExprs, List<TypedFieldId> keyOutputIds)
    +          throws SchemaChangeException, ClassTransformationException, IOException {
    +    ClassGenerator<StreamingAggregator> cg =
    +        CodeGenerator.getRoot(StreamingAggTemplate.TEMPLATE_DEFINITION, context.getFunctionRegistry(),
    +            context.getOptions());
    +
    +    LogicalExpression[] keyExprsArray = new LogicalExpression[keyExprs.size()];
    +    LogicalExpression[] valueExprsArray = new LogicalExpression[valueExprs.size()];
    +    TypedFieldId[] keyOutputIdsArray = new TypedFieldId[keyOutputIds.size()];
    +
    +    keyExprs.toArray(keyExprsArray);
    +    valueExprs.toArray(valueExprsArray);
    +    keyOutputIds.toArray(keyOutputIdsArray);
    +
    +    setupIsSame(cg, keyExprsArray);
    +    setupIsSameApart(cg, keyExprsArray);
    +    addRecordValues(cg, valueExprsArray);
    +    outputRecordKeys(cg, keyOutputIdsArray, keyExprsArray);
    +    outputRecordKeysPrev(cg, keyOutputIdsArray, keyExprsArray);
    +
    +    cg.getBlock("resetValues")._return(JExpr.TRUE);
    +    getIndex(cg);
    +
    +    container.buildSchema(SelectionVectorMode.NONE);
    +    StreamingAggregator agg = context.getImplementationClass(cg);
    +    agg.setup(oContext, incoming, this);
    +    return agg;
    +  }
    +
    +  private boolean isImplicitFileColumn(MaterializedField mf) {
    +    return ImplicitColumnExplorer.initImplicitFileColumns(context.getOptions()).get(mf.getName()) != null;
    +  }
    +
    +  protected StreamingAggregator createAggregatorInternal()
    +      throws SchemaChangeException, ClassTransformationException, IOException {
    +    container.clear();
    +
    +    List<LogicalExpression> keyExprs = Lists.newArrayList();
    +    List<LogicalExpression> valueExprs = Lists.newArrayList();
    +    List<TypedFieldId> keyOutputIds = Lists.newArrayList();
    +    GregorianCalendar calendar = new GregorianCalendar(TimeZone.getTimeZone("UTC"));
    +    calendar.setTimeInMillis(System.currentTimeMillis());
    +
    +    if (this.getPopConfig() instanceof StatisticsAggregate
    +        && (((StatisticsAggregate) this.getPopConfig()).getPhase() == OperatorPhase.PHASE_1of1
    +            || ((StatisticsAggregate) this.getPopConfig()).getPhase() == OperatorPhase.PHASE_2of2)) {
    +      createKeyColumn("schema",
    +          ValueExpressions.getBigInt(schema++),
    +          keyExprs,
    +          keyOutputIds
    +      );
    +      createKeyColumn("computed",
    +          ValueExpressions.getDate(calendar),
    +          keyExprs,
    +          keyOutputIds
    +      );
    +    }
    +
    +    MapVector cparent = new MapVector("column", oContext.getAllocator(), null);
    +    container.add(cparent);
    +    for (MaterializedField mf : incoming.getSchema()) {
    +      // Ignore implicit columns
    +      if (!isImplicitFileColumn(mf)) {
    +        createNestedKeyColumn(
    +            cparent,
    +            mf.getLastName(),
    --- End diff --
    
    The schema changed here - we do not support schema changes in 1.10. Can we run into the same issue for Parquet? Before scanning, we check whether the input is a Parquet 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 #729: Drill 1328: Support table statistics for Parquet

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

    https://github.com/apache/drill/pull/729#discussion_r103406448
  
    --- Diff: exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/statistics/MergedStatisticFactory.java ---
    @@ -0,0 +1,46 @@
    +/*
    + * 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.impl.statistics;
    +
    +public class MergedStatisticFactory {
    --- 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 #729: Drill 1328: Support table statistics for Parquet

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

    https://github.com/apache/drill/pull/729#discussion_r103408160
  
    --- Diff: exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/statistics/Statistic.java ---
    @@ -0,0 +1,38 @@
    +/**
    --- 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.
---