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 infrastruct...@apache.org or file a JIRA ticket with INFRA. ---