Return-Path: X-Original-To: archive-asf-public-internal@cust-asf2.ponee.io Delivered-To: archive-asf-public-internal@cust-asf2.ponee.io Received: from cust-asf.ponee.io (cust-asf.ponee.io [163.172.22.183]) by cust-asf2.ponee.io (Postfix) with ESMTP id B28C0200C18 for ; Sat, 11 Feb 2017 22:57:25 +0100 (CET) Received: by cust-asf.ponee.io (Postfix) id B1173160B4C; Sat, 11 Feb 2017 21:57:25 +0000 (UTC) Delivered-To: archive-asf-public@cust-asf.ponee.io Received: from mail.apache.org (hermes.apache.org [140.211.11.3]) by cust-asf.ponee.io (Postfix) with SMTP id AC584160B6A for ; Sat, 11 Feb 2017 22:57:24 +0100 (CET) Received: (qmail 35359 invoked by uid 500); 11 Feb 2017 21:57:16 -0000 Mailing-List: contact dev-help@drill.apache.org; run by ezmlm Precedence: bulk List-Help: List-Unsubscribe: List-Post: List-Id: Reply-To: dev@drill.apache.org Delivered-To: mailing list dev@drill.apache.org Received: (qmail 33997 invoked by uid 99); 11 Feb 2017 21:57:15 -0000 Received: from git1-us-west.apache.org (HELO git1-us-west.apache.org) (140.211.11.23) by apache.org (qpsmtpd/0.29) with ESMTP; Sat, 11 Feb 2017 21:57:15 +0000 Received: by git1-us-west.apache.org (ASF Mail Server at git1-us-west.apache.org, from userid 33) id BCC70DF9AF; Sat, 11 Feb 2017 21:57:15 +0000 (UTC) From: paul-rogers To: dev@drill.apache.org Reply-To: dev@drill.apache.org References: In-Reply-To: Subject: [GitHub] drill pull request #729: Drill 1328: Support table statistics for Parquet Content-Type: text/plain Message-Id: <20170211215715.BCC70DF9AF@git1-us-west.apache.org> Date: Sat, 11 Feb 2017 21:57:15 +0000 (UTC) archived-at: Sat, 11 Feb 2017 21:57:25 -0000 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 + *

+ * 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.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 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 keyExprs, + List 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 keyExprs, List keyOutputIds) + throws SchemaChangeException { + ErrorCollector collector = new ErrorCollectorImpl(); + + LogicalExpression mle = ExpressionTreeMaterializer.materialize(expr, incoming, collector, + context.getFunctionRegistry()); + + Class 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 valueExprs) throws SchemaChangeException { + ErrorCollector collector = new ErrorCollectorImpl(); + + LogicalExpression mle = ExpressionTreeMaterializer.materialize(expr, incoming, collector, + context.getFunctionRegistry()); + + Class 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 keyExprs, + List valueExprs, List keyOutputIds) + throws SchemaChangeException, ClassTransformationException, IOException { + ClassGenerator 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 keyExprs = Lists.newArrayList(); + List valueExprs = Lists.newArrayList(); + List 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. ---