[ 
https://issues.apache.org/jira/browse/HIVE-24471?focusedWorklogId=530507&page=com.atlassian.jira.plugin.system.issuetabpanels:worklog-tabpanel#worklog-530507
 ]

ASF GitHub Bot logged work on HIVE-24471:
-----------------------------------------

                Author: ASF GitHub Bot
            Created on: 04/Jan/21 04:07
            Start Date: 04/Jan/21 04:07
    Worklog Time Spent: 10m 
      Work Description: maheshk114 commented on a change in pull request #1736:
URL: https://github.com/apache/hive/pull/1736#discussion_r551109482



##########
File path: ql/src/java/org/apache/hadoop/hive/ql/exec/GroupByCombiner.java
##########
@@ -0,0 +1,282 @@
+/*
+ * 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;
+
+import org.apache.hadoop.hive.conf.HiveConf;
+import org.apache.hadoop.hive.ql.exec.vector.VectorGroupByCombiner;
+import org.apache.hadoop.hive.ql.exec.vector.VectorGroupByOperator;
+import org.apache.hadoop.mapred.JobConf;
+import org.apache.hadoop.hive.ql.metadata.HiveException;
+import org.apache.hadoop.hive.ql.plan.BaseWork;
+import org.apache.hadoop.hive.ql.plan.GroupByDesc;
+import org.apache.hadoop.hive.ql.plan.ReduceWork;
+import org.apache.hadoop.hive.ql.plan.TableDesc;
+import org.apache.hadoop.hive.ql.udf.generic.GenericUDAFEvaluator;
+import org.apache.hadoop.hive.serde2.AbstractSerDe;
+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.Serializer;
+import org.apache.hadoop.hive.serde2.objectinspector.ObjectInspector;
+import org.apache.hadoop.hive.serde2.objectinspector.ObjectInspectorFactory;
+import org.apache.hadoop.io.BytesWritable;
+import org.apache.hadoop.io.DataInputBuffer;
+import org.apache.hadoop.util.ReflectionUtils;
+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 org.apache.hadoop.fs.Path;
+
+import java.io.IOException;
+import java.util.ArrayList;
+
+import static org.apache.hadoop.hive.ql.exec.Utilities.HAS_REDUCE_WORK;
+import static org.apache.hadoop.hive.ql.exec.Utilities.REDUCE_PLAN_NAME;
+
+// Combiner for normal 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 GroupByCombiner extends VectorGroupByCombiner {
+
+  private static final Logger LOG = LoggerFactory.getLogger(
+          org.apache.hadoop.hive.ql.exec.GroupByCombiner.class.getName());
+
+  private transient GenericUDAFEvaluator[] aggregationEvaluators;
+  Deserializer valueDeserializer;
+  GenericUDAFEvaluator.AggregationBuffer[] aggregationBuffers;
+  GroupByOperator groupByOperator;
+  Serializer valueSerializer;
+  ObjectInspector aggrObjectInspector;
+  DataInputBuffer valueBuffer;
+  Object[] cachedValues;
+
+  public GroupByCombiner(TaskContext taskContext) throws HiveException, 
IOException {
+    super(taskContext);
+    if (rw != null) {
+      try {
+        groupByOperator = (GroupByOperator) rw.getReducer();
+
+        ArrayList<ObjectInspector> ois = new ArrayList<ObjectInspector>();
+        ois.add(keyObjectInspector);
+        ois.add(valueObjectInspector);
+        ObjectInspector[] rowObjectInspector = new ObjectInspector[1];
+        rowObjectInspector[0] =
+            
ObjectInspectorFactory.getStandardStructObjectInspector(Utilities.reduceFieldNameList,
+                        ois);
+        groupByOperator.setInputObjInspectors(rowObjectInspector);
+        groupByOperator.initializeOp(conf);
+        aggregationBuffers = groupByOperator.getAggregationBuffers();
+        aggregationEvaluators = groupByOperator.getAggregationEvaluator();
+
+        TableDesc valueTableDesc = rw.getTagToValueDesc().get(0);
+        if ((aggregationEvaluators == null) || (aggregationEvaluators.length 
!= numValueCol)) {
+          //TODO : Need to support distinct. The logic has to be changed to 
extract only
+          // those aggregates which are not part of distinct.
+          LOG.info(" Combiner is disabled as the number of value columns does" 
+
+                  " not match with number of aggregators");
+          numValueCol = 0;
+          rw = null;
+          return;
+        }
+        valueSerializer = (Serializer) valueTableDesc.getDeserializerClass()
+                .newInstance();
+        valueSerializer.initialize(null, valueTableDesc.getProperties());
+
+        valueDeserializer = (AbstractSerDe) ReflectionUtils.newInstance(
+                valueTableDesc.getDeserializerClass(), null);
+        SerDeUtils.initializeSerDe(valueDeserializer, null,
+                valueTableDesc.getProperties(), null);
+
+        aggrObjectInspector = groupByOperator.getAggrObjInspector();
+        valueBuffer = new DataInputBuffer();
+        cachedValues = new Object[aggregationEvaluators.length];
+      } catch (Exception e) {
+        LOG.error(" GroupByCombiner failed", e);
+        throw new RuntimeException(e.getMessage());
+      }
+    }
+  }
+
+  private void processAggregation(IFile.Writer writer, DataInputBuffer key)
+          throws Exception {
+    for (int i = 0; i < aggregationEvaluators.length; i++) {
+      cachedValues[i] = 
aggregationEvaluators[i].evaluate(aggregationBuffers[i]);
+    }
+    BytesWritable result = (BytesWritable) 
valueSerializer.serialize(cachedValues,
+            aggrObjectInspector);
+    valueBuffer.reset(result.getBytes(), result.getLength());
+    writer.append(key, valueBuffer);
+    combineOutputRecordsCounter.increment(1);
+    for (int i = 0; i < aggregationEvaluators.length; i++) {
+      aggregationEvaluators[i].reset(aggregationBuffers[i]);
+    }
+  }
+
+  private void updateAggregation(BytesWritable valWritable, DataInputBuffer 
value)
+          throws HiveException, SerDeException {
+    valWritable.set(value.getData(), value.getPosition(),
+            value.getLength() - value.getPosition());
+    Object row = valueDeserializer.deserialize(valWritable);
+    groupByOperator.updateAggregation(row);
+  }
+
+  private void processRows(TezRawKeyValueIterator rawIter, IFile.Writer 
writer) {
+    long numRows = 0;
+    try {
+      DataInputBuffer key = rawIter.getKey();
+      DataInputBuffer prevKey = new DataInputBuffer();

Review comment:
       done

##########
File path: ql/src/java/org/apache/hadoop/hive/ql/exec/tez/TezTask.java
##########
@@ -515,6 +516,10 @@ DAG build(JobConf conf, TezWork tezWork, Path scratchDir, 
Context ctx,
           Edge e = null;
 
           TezEdgeProperty edgeProp = tezWork.getEdgeProperty(workUnit, v);
+
+          //Add the reducer plan to config to create the combiner object in 
case of group by.
+          wxConf = GroupByCombiner.setCombinerInConf(v, wxConf, 
workToConf.get(v));

Review comment:
       During create vertex we may not know the child or parent. The node may 
get reordered after vertex is created. So to be on safer side it is done here 
once all the optimizations are done.




----------------------------------------------------------------
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:
us...@infra.apache.org


Issue Time Tracking
-------------------

    Worklog Id:     (was: 530507)
    Time Spent: 1h 40m  (was: 1.5h)

> Add support for combiner in hash mode group aggregation 
> --------------------------------------------------------
>
>                 Key: HIVE-24471
>                 URL: https://issues.apache.org/jira/browse/HIVE-24471
>             Project: Hive
>          Issue Type: Bug
>          Components: Hive
>            Reporter: mahesh kumar behera
>            Assignee: mahesh kumar behera
>            Priority: Major
>              Labels: pull-request-available
>          Time Spent: 1h 40m
>  Remaining Estimate: 0h
>
> In map side group aggregation, partial grouped aggregation is calculated to 
> reduce the data written to disk by map task. In case of hash aggregation, 
> where the input data is not sorted, hash table is used (with sorting also 
> being performed before flushing). If the hash table size increases beyond 
> configurable limit, data is flushed to disk and new hash table is generated. 
> If the reduction by hash table is less than min hash aggregation reduction 
> calculated during compile time, the map side aggregation is converted to 
> streaming mode. So if the first few batch of records does not result into 
> significant reduction, then the mode is switched to streaming mode. This may 
> have impact on performance, if the subsequent batch of records have less 
> number of distinct values. 
> To improve performance both in Hash and Streaming mode, a combiner can be 
> added to the map task after the keys are sorted. This will make sure that the 
> aggregation is done if possible and reduce the data written to disk.



--
This message was sent by Atlassian Jira
(v8.3.4#803005)

Reply via email to