maheshk114 commented on a change in pull request #1736:
URL: https://github.com/apache/hive/pull/1736#discussion_r539842254



##########
File path: 
ql/src/java/org/apache/hadoop/hive/ql/exec/vector/VectorGroupByCombiner.java
##########
@@ -0,0 +1,377 @@
+/*
+ * 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.hadoop.hive.ql.exec.vector;
+
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.hive.conf.HiveConf;
+import org.apache.hadoop.hive.ql.exec.Utilities;
+import org.apache.hadoop.hive.ql.exec.mr.ExecReducer;
+import 
org.apache.hadoop.hive.ql.exec.vector.expressions.aggregates.VectorAggregateExpression;
+import org.apache.hadoop.hive.ql.metadata.HiveException;
+import org.apache.hadoop.hive.ql.plan.ReduceWork;
+import org.apache.hadoop.hive.ql.plan.TableDesc;
+import org.apache.hadoop.hive.serde2.AbstractSerDe;
+import org.apache.hadoop.hive.serde2.ByteStream;
+import org.apache.hadoop.hive.serde2.Deserializer;
+import org.apache.hadoop.hive.serde2.SerDeException;
+import org.apache.hadoop.hive.serde2.SerDeUtils;
+import org.apache.hadoop.hive.serde2.lazybinary.fast.LazyBinaryDeserializeRead;
+import org.apache.hadoop.hive.serde2.lazybinary.fast.LazyBinarySerializeWrite;
+import org.apache.hadoop.hive.serde2.objectinspector.ObjectInspector;
+import org.apache.hadoop.hive.serde2.objectinspector.StructObjectInspector;
+import org.apache.hadoop.io.DataInputBuffer;
+import org.apache.hadoop.mapreduce.TaskCounter;
+import org.apache.hadoop.util.ReflectionUtils;
+import org.apache.hadoop.util.StringUtils;
+import org.apache.tez.common.TezUtils;
+import org.apache.tez.common.counters.TezCounter;
+import org.apache.tez.mapreduce.combine.MRCombiner;
+import org.apache.tez.runtime.api.TaskContext;
+import org.apache.tez.runtime.library.common.sort.impl.IFile;
+import org.apache.tez.runtime.library.common.sort.impl.TezRawKeyValueIterator;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+import java.io.IOException;
+
+import static org.apache.hadoop.hive.ql.exec.Utilities.HAS_REDUCE_WORK;
+import static org.apache.hadoop.hive.ql.exec.Utilities.MAPRED_REDUCER_CLASS;
+import static org.apache.hadoop.hive.ql.exec.Utilities.REDUCE_PLAN_NAME;
+import static 
org.apache.hadoop.hive.serde2.lazy.fast.LazySimpleDeserializeRead.byteArrayCompareRanges;
+
+// Combiner for vectorized group by operator. In case of map side aggregate, 
the partially
+// aggregated records are sorted based on group by key. If because of some 
reasons, like hash
+// table memory exceeded the limit or the first few batches of records have 
less ndvs, the
+// aggregation is not done, then here the aggregation can be done cheaply as 
the records
+// are sorted based on group by key.
+public class VectorGroupByCombiner extends MRCombiner {
+  private static final Logger LOG = LoggerFactory.getLogger(
+      VectorGroupByCombiner.class.getName());
+  protected final Configuration conf;
+  protected final TezCounter combineInputRecordsCounter;
+  protected final TezCounter combineOutputRecordsCounter;
+  VectorAggregateExpression[] aggregators;
+  VectorAggregationBufferRow aggregationBufferRow;
+  protected transient LazyBinarySerializeWrite valueLazyBinarySerializeWrite;
+
+  // This helper object serializes LazyBinary format reducer values from 
columns of a row
+  // in a vectorized row batch.
+  protected transient VectorSerializeRow<LazyBinarySerializeWrite> 
valueVectorSerializeRow;
+
+  // The output buffer used to serialize a value into.
+  protected transient ByteStream.Output valueOutput;
+  DataInputBuffer valueBytesWritable;
+
+  // Only required minimal configs are copied to the worker nodes. This hack 
(file.) is
+  // done to include these configs to be copied to the worker node.
+  protected static String confPrefixForWorker = "file.";
+
+  VectorDeserializeRow<LazyBinaryDeserializeRead> batchValueDeserializer;
+  int firstValueColumnOffset;
+  VectorizedRowBatchCtx batchContext = null;
+  int numValueCol = 0;
+  protected ReduceWork rw;
+  VectorizedRowBatch outputBatch = null;
+  VectorizedRowBatch inputBatch = null;
+  protected Deserializer inputKeyDeserializer = null;
+  protected ObjectInspector keyObjectInspector = null;
+  protected ObjectInspector valueObjectInspector = null;
+  protected StructObjectInspector valueStructInspectors = null;
+  protected StructObjectInspector keyStructInspector = null;
+
+  public VectorGroupByCombiner(TaskContext taskContext) throws HiveException, 
IOException {
+    super(taskContext);
+
+    combineInputRecordsCounter =
+            
taskContext.getCounters().findCounter(TaskCounter.COMBINE_INPUT_RECORDS);
+    combineOutputRecordsCounter =
+            
taskContext.getCounters().findCounter(TaskCounter.COMBINE_OUTPUT_RECORDS);
+
+    conf = TezUtils.createConfFromUserPayload(taskContext.getUserPayload());
+    rw = getReduceWork();
+    if (rw == null) {
+      return;
+    }
+
+    if (rw.getReducer() instanceof VectorGroupByOperator) {
+      VectorGroupByOperator vectorGroupByOperator = (VectorGroupByOperator) 
rw.getReducer();
+      vectorGroupByOperator.initializeOp(this.conf);
+      this.aggregators = vectorGroupByOperator.getAggregators();
+      this.aggregationBufferRow = allocateAggregationBuffer();
+      batchContext = rw.getVectorizedRowBatchCtx();
+    }
+
+    try {
+      initObjectInspectors(rw.getTagToValueDesc().get(0), rw.getKeyDesc());
+      if (batchContext != null && numValueCol > 0) {
+        initVectorBatches();
+      }
+    } catch (SerDeException e) {
+      LOG.error("Fail to initialize VectorGroupByCombiner.", e);
+      throw new RuntimeException(e.getCause());
+    }
+  }
+
+  // Get the reduce work from the config. Here some hack is used to prefix the 
config name with
+  // "file." to avoid the config being filtered out.
+  private ReduceWork getReduceWork() {
+    String plan =  conf.get(confPrefixForWorker + 
HiveConf.ConfVars.PLAN.varname);
+    this.conf.set(HiveConf.ConfVars.PLAN.varname, plan);
+    if (conf.getBoolean(confPrefixForWorker + 
HiveConf.ConfVars.HIVE_RPC_QUERY_PLAN.varname,
+            true)) {
+      Path planPath = new Path(plan);
+      planPath = new Path(planPath, REDUCE_PLAN_NAME);
+      String planString = conf.get(confPrefixForWorker + 
planPath.toUri().getPath());
+      this.conf.set(planPath.toUri().getPath(), planString);
+      this.conf.set(HiveConf.ConfVars.HIVE_RPC_QUERY_PLAN.varname, "true");
+    } else {
+      this.conf.set(HiveConf.ConfVars.HIVE_RPC_QUERY_PLAN.varname, "false");
+    }
+    this.conf.set(HAS_REDUCE_WORK, "true");
+    this.conf.set(MAPRED_REDUCER_CLASS, ExecReducer.class.getName());
+
+    return Utilities.getReduceWork(conf);
+  }
+
+  private void initObjectInspectors(TableDesc valueTableDesc,TableDesc 
keyTableDesc)
+          throws SerDeException {
+    inputKeyDeserializer =
+            ReflectionUtils.newInstance(keyTableDesc.getDeserializerClass(), 
null);
+    SerDeUtils.initializeSerDe(inputKeyDeserializer, null,
+            keyTableDesc.getProperties(), null);
+    keyObjectInspector = inputKeyDeserializer.getObjectInspector();
+
+    keyStructInspector = (StructObjectInspector) keyObjectInspector;
+    firstValueColumnOffset = keyStructInspector.getAllStructFieldRefs().size();
+
+    Deserializer inputValueDeserializer = (AbstractSerDe) 
ReflectionUtils.newInstance(
+            valueTableDesc.getDeserializerClass(), null);
+    SerDeUtils.initializeSerDe(inputValueDeserializer, null,
+            valueTableDesc.getProperties(), null);
+    valueObjectInspector = inputValueDeserializer.getObjectInspector();
+    valueStructInspectors = (StructObjectInspector) valueObjectInspector;
+    numValueCol = valueStructInspectors.getAllStructFieldRefs().size();
+  }
+
+  void initVectorBatches() throws HiveException {
+    inputBatch = batchContext.createVectorizedRowBatch();
+
+    // Create data buffers for value bytes column vectors.
+    for (int i = firstValueColumnOffset; i < inputBatch.numCols; i++) {
+      ColumnVector colVector = inputBatch.cols[i];
+      if (colVector instanceof BytesColumnVector) {
+        BytesColumnVector bytesColumnVector = (BytesColumnVector) colVector;
+        bytesColumnVector.initBuffer();
+      }
+    }
+
+    batchValueDeserializer =
+            new VectorDeserializeRow<>(
+                    new LazyBinaryDeserializeRead(
+                            
VectorizedBatchUtil.typeInfosFromStructObjectInspector(
+                                    valueStructInspectors),
+                            true));
+    batchValueDeserializer.init(firstValueColumnOffset);
+
+    int[] valueColumnMap = new int[numValueCol];
+    for (int i = 0; i < numValueCol; i++) {
+      valueColumnMap[i] = i + firstValueColumnOffset;
+    }
+
+    valueLazyBinarySerializeWrite = new LazyBinarySerializeWrite(numValueCol);
+    valueVectorSerializeRow = new 
VectorSerializeRow<>(valueLazyBinarySerializeWrite);
+    
valueVectorSerializeRow.init(VectorizedBatchUtil.typeInfosFromStructObjectInspector(
+            valueStructInspectors), valueColumnMap);
+    valueOutput = new ByteStream.Output();
+    valueVectorSerializeRow.setOutput(valueOutput);
+    outputBatch = batchContext.createVectorizedRowBatch();
+    valueBytesWritable = new DataInputBuffer();
+  }
+
+  private VectorAggregationBufferRow allocateAggregationBuffer() throws 
HiveException {
+    VectorAggregateExpression.AggregationBuffer[] aggregationBuffers =
+            new 
VectorAggregateExpression.AggregationBuffer[aggregators.length];
+    for (int i=0; i < aggregators.length; ++i) {
+      aggregationBuffers[i] = aggregators[i].getNewAggregationBuffer();
+      aggregators[i].reset(aggregationBuffers[i]);
+    }
+    return new VectorAggregationBufferRow(aggregationBuffers);
+  }
+
+  private void finishAggregation(DataInputBuffer key, IFile.Writer writer, 
boolean needFlush)
+          throws HiveException, IOException {
+    for (int i = 0; i < aggregators.length; ++i) {
+      try {
+        
aggregators[i].aggregateInput(aggregationBufferRow.getAggregationBuffer(i), 
inputBatch);
+      } catch (HiveException e) {
+        throw new RuntimeException(e.getCause());
+      }
+    }
+
+    // In case the input batch is full but the keys are still same we need not 
flush.
+    // Only evaluate the aggregates and store it in the aggregationBufferRow. 
The aggregate
+    // functions are incremental and will take care of correctness when next 
batch comes for
+    // aggregation.
+    if (!needFlush) {
+      return;
+    }
+
+    int colNum = firstValueColumnOffset;
+    for (int i = 0; i < aggregators.length; ++i) {
+      aggregators[i].assignRowColumn(outputBatch, 0, colNum++,
+              aggregationBufferRow.getAggregationBuffer(i));
+    }
+
+    valueLazyBinarySerializeWrite.reset();
+    valueVectorSerializeRow.serializeWrite(outputBatch, 0);
+    valueBytesWritable.reset(valueOutput.getData(), 0, 
valueOutput.getLength());
+    writer.append(key, valueBytesWritable);
+    combineOutputRecordsCounter.increment(1);
+    aggregationBufferRow.reset();
+    outputBatch.reset();
+  }
+
+  private void addValueToBatch(DataInputBuffer val, DataInputBuffer key,
+                      IFile.Writer writer, boolean needFLush) throws 
IOException, HiveException {
+    batchValueDeserializer.setBytes(val.getData(), val.getPosition(),
+            val.getLength() - val.getPosition());
+    batchValueDeserializer.deserialize(inputBatch, inputBatch.size);
+    inputBatch.size++;
+    if (needFLush || (inputBatch.size >= VectorizedRowBatch.DEFAULT_SIZE)) {
+      processVectorGroup(key, writer, needFLush);
+    }
+  }
+
+  private void processVectorGroup(DataInputBuffer key, IFile.Writer writer, 
boolean needFlush)
+          throws HiveException {
+    try {
+      finishAggregation(key, writer, needFlush);
+      inputBatch.reset();
+    } catch (Exception e) {
+      String rowString;
+      try {
+        rowString = inputBatch.toString();
+      } catch (Exception e2) {
+        rowString = "[Error getting row data with exception "
+                + StringUtils.stringifyException(e2) + " ]";
+      }
+      LOG.error("Hive Runtime Error while processing vector batch" + 
rowString, e);
+      throw new HiveException("Hive Runtime Error while processing vector 
batch", e);
+    }
+  }
+
+  protected void appendDirectlyToWriter(TezRawKeyValueIterator rawIter, 
IFile.Writer writer) {
+    long numRows = 0;
+    try {
+      do {
+        numRows++;
+        writer.append(rawIter.getKey(), rawIter.getValue());
+      } while (rawIter.next());
+      combineInputRecordsCounter.increment(numRows);
+      combineOutputRecordsCounter.increment(numRows);
+    } catch(IOException e) {
+      LOG.error("Append to writer failed", e);
+      throw new RuntimeException(e.getMessage());
+    }
+  }
+
+  private void appendToWriter(DataInputBuffer val, DataInputBuffer key, 
IFile.Writer writer) {
+    try {
+      writer.append(key, val);
+      combineOutputRecordsCounter.increment(1);
+    } catch(IOException e) {
+      LOG.error("Append value list to writer failed", e);
+      throw new RuntimeException(e.getMessage());
+    }
+  }
+
+  public static int compare(DataInputBuffer buf1, DataInputBuffer buf2) {

Review comment:
       make sense. I will check for the equal.




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
[email protected]



---------------------------------------------------------------------
To unsubscribe, e-mail: [email protected]
For additional commands, e-mail: [email protected]

Reply via email to