You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@hive.apache.org by xu...@apache.org on 2014/10/21 16:00:02 UTC

svn commit: r1633382 - in /hive/branches/spark/ql/src/java/org/apache/hadoop/hive/ql/exec/spark: ./ counter/

Author: xuefu
Date: Tue Oct 21 14:00:01 2014
New Revision: 1633382

URL: http://svn.apache.org/r1633382
Log:
HIVE-7709: Create SparkReporter [Spark Branch] (chengxiang via Xuefu)

Added:
    hive/branches/spark/ql/src/java/org/apache/hadoop/hive/ql/exec/spark/SparkReporter.java
Modified:
    hive/branches/spark/ql/src/java/org/apache/hadoop/hive/ql/exec/spark/HiveMapFunction.java
    hive/branches/spark/ql/src/java/org/apache/hadoop/hive/ql/exec/spark/HivePairFlatMapFunction.java
    hive/branches/spark/ql/src/java/org/apache/hadoop/hive/ql/exec/spark/HiveReduceFunction.java
    hive/branches/spark/ql/src/java/org/apache/hadoop/hive/ql/exec/spark/SparkClient.java
    hive/branches/spark/ql/src/java/org/apache/hadoop/hive/ql/exec/spark/SparkPlanGenerator.java
    hive/branches/spark/ql/src/java/org/apache/hadoop/hive/ql/exec/spark/SparkRecordHandler.java
    hive/branches/spark/ql/src/java/org/apache/hadoop/hive/ql/exec/spark/counter/SparkCounters.java

Modified: hive/branches/spark/ql/src/java/org/apache/hadoop/hive/ql/exec/spark/HiveMapFunction.java
URL: http://svn.apache.org/viewvc/hive/branches/spark/ql/src/java/org/apache/hadoop/hive/ql/exec/spark/HiveMapFunction.java?rev=1633382&r1=1633381&r2=1633382&view=diff
==============================================================================
--- hive/branches/spark/ql/src/java/org/apache/hadoop/hive/ql/exec/spark/HiveMapFunction.java (original)
+++ hive/branches/spark/ql/src/java/org/apache/hadoop/hive/ql/exec/spark/HiveMapFunction.java Tue Oct 21 14:00:01 2014
@@ -18,26 +18,21 @@
 
 package org.apache.hadoop.hive.ql.exec.spark;
 
-import java.util.Iterator;
-
 import org.apache.hadoop.hive.ql.exec.Utilities;
 import org.apache.hadoop.hive.ql.io.HiveKey;
 import org.apache.hadoop.hive.ql.io.merge.MergeFileMapper;
 import org.apache.hadoop.io.BytesWritable;
-import org.apache.hadoop.mapred.JobConf;
-import org.apache.hadoop.mapred.Reporter;
-import org.apache.spark.TaskContext;
-import org.apache.spark.api.java.function.PairFlatMapFunction;
-
 import scala.Tuple2;
 
+import java.util.Iterator;
+
 public class HiveMapFunction extends HivePairFlatMapFunction<
   Iterator<Tuple2<BytesWritable, BytesWritable>>, HiveKey, BytesWritable> {
 
   private static final long serialVersionUID = 1L;
 
-  public HiveMapFunction(byte[] buffer) {
-    super(buffer);
+  public HiveMapFunction(byte[] jobConfBuffer, SparkReporter sparkReporter) {
+    super(jobConfBuffer, sparkReporter);
   }
 
   @Override
@@ -56,7 +51,7 @@ public class HiveMapFunction extends Hiv
 
     HiveMapFunctionResultList result = new HiveMapFunctionResultList(jobConf, it, mapRecordHandler);
     //TODO we need to implement a Spark specified Reporter to collect stats, refer to HIVE-7709.
-    mapRecordHandler.init(jobConf, result, Reporter.NULL);
+    mapRecordHandler.init(jobConf, result, sparkReporter);
 
     return result;
   }

Modified: hive/branches/spark/ql/src/java/org/apache/hadoop/hive/ql/exec/spark/HivePairFlatMapFunction.java
URL: http://svn.apache.org/viewvc/hive/branches/spark/ql/src/java/org/apache/hadoop/hive/ql/exec/spark/HivePairFlatMapFunction.java?rev=1633382&r1=1633381&r2=1633382&view=diff
==============================================================================
--- hive/branches/spark/ql/src/java/org/apache/hadoop/hive/ql/exec/spark/HivePairFlatMapFunction.java (original)
+++ hive/branches/spark/ql/src/java/org/apache/hadoop/hive/ql/exec/spark/HivePairFlatMapFunction.java Tue Oct 21 14:00:01 2014
@@ -9,13 +9,8 @@ import org.apache.spark.api.java.functio
 
 public abstract class HivePairFlatMapFunction<T, K, V> implements PairFlatMapFunction<T, K, V> {
 
-  protected transient JobConf jobConf;
-
-  private byte[] buffer;
-
   protected static final NumberFormat taskIdFormat = NumberFormat.getInstance();
   protected static final NumberFormat stageIdFormat = NumberFormat.getInstance();
-
   static {
     taskIdFormat.setGroupingUsed(false);
     taskIdFormat.setMinimumIntegerDigits(6);
@@ -23,8 +18,14 @@ public abstract class HivePairFlatMapFun
     stageIdFormat.setMinimumIntegerDigits(4);
   }
 
-  public HivePairFlatMapFunction(byte[] buffer) {
+  protected transient JobConf jobConf;
+  protected SparkReporter sparkReporter;
+
+  private byte[] buffer;
+
+  public HivePairFlatMapFunction(byte[] buffer, SparkReporter sparkReporter) {
     this.buffer = buffer;
+    this.sparkReporter = sparkReporter;
   }
 
   protected void initJobConf() {

Modified: hive/branches/spark/ql/src/java/org/apache/hadoop/hive/ql/exec/spark/HiveReduceFunction.java
URL: http://svn.apache.org/viewvc/hive/branches/spark/ql/src/java/org/apache/hadoop/hive/ql/exec/spark/HiveReduceFunction.java?rev=1633382&r1=1633381&r2=1633382&view=diff
==============================================================================
--- hive/branches/spark/ql/src/java/org/apache/hadoop/hive/ql/exec/spark/HiveReduceFunction.java (original)
+++ hive/branches/spark/ql/src/java/org/apache/hadoop/hive/ql/exec/spark/HiveReduceFunction.java Tue Oct 21 14:00:01 2014
@@ -18,21 +18,19 @@
 
 package org.apache.hadoop.hive.ql.exec.spark;
 
-import java.util.Iterator;
-
 import org.apache.hadoop.hive.ql.io.HiveKey;
 import org.apache.hadoop.io.BytesWritable;
-import org.apache.hadoop.mapred.Reporter;
-
 import scala.Tuple2;
 
+import java.util.Iterator;
+
 public class HiveReduceFunction extends HivePairFlatMapFunction<
   Iterator<Tuple2<HiveKey, Iterable<BytesWritable>>>, HiveKey, BytesWritable> {
 
   private static final long serialVersionUID = 1L;
 
-  public HiveReduceFunction(byte[] buffer) {
-    super(buffer);
+  public HiveReduceFunction(byte[] buffer, SparkReporter sparkReporter) {
+    super(buffer, sparkReporter);
   }
 
   @Override
@@ -43,7 +41,7 @@ public class HiveReduceFunction extends 
     SparkReduceRecordHandler reducerRecordhandler = new SparkReduceRecordHandler();
     HiveReduceFunctionResultList result =
         new HiveReduceFunctionResultList(jobConf, it, reducerRecordhandler);
-    reducerRecordhandler.init(jobConf, result, Reporter.NULL);
+    reducerRecordhandler.init(jobConf, result, sparkReporter);
 
     return result;
   }

Modified: hive/branches/spark/ql/src/java/org/apache/hadoop/hive/ql/exec/spark/SparkClient.java
URL: http://svn.apache.org/viewvc/hive/branches/spark/ql/src/java/org/apache/hadoop/hive/ql/exec/spark/SparkClient.java?rev=1633382&r1=1633381&r2=1633382&view=diff
==============================================================================
--- hive/branches/spark/ql/src/java/org/apache/hadoop/hive/ql/exec/spark/SparkClient.java (original)
+++ hive/branches/spark/ql/src/java/org/apache/hadoop/hive/ql/exec/spark/SparkClient.java Tue Oct 21 14:00:01 2014
@@ -29,6 +29,9 @@ import org.apache.hadoop.hive.conf.HiveC
 import org.apache.hadoop.hive.ql.Context;
 import org.apache.hadoop.hive.ql.DriverContext;
 import org.apache.hadoop.hive.ql.exec.Utilities;
+import org.apache.hadoop.hive.ql.exec.spark.counter.SparkCounter;
+import org.apache.hadoop.hive.ql.exec.spark.counter.SparkCounterGroup;
+import org.apache.hadoop.hive.ql.exec.spark.counter.SparkCounters;
 import org.apache.hadoop.hive.ql.exec.spark.status.SparkJobMonitor;
 import org.apache.hadoop.hive.ql.exec.spark.status.impl.JobStateListener;
 import org.apache.hadoop.hive.ql.exec.spark.status.impl.SimpleSparkJobStatus;
@@ -166,9 +169,12 @@ public class SparkClient implements Seri
       return 5;
     }
 
+    SparkCounters sparkCounters = new SparkCounters(sc, hiveConf);
+    SparkReporter sparkReporter = new SparkReporter(sparkCounters);
+
     // Generate Spark plan
-    SparkPlanGenerator gen = new SparkPlanGenerator(sc, ctx, jobConf,
-        emptyScratchDir);
+    SparkPlanGenerator gen =
+      new SparkPlanGenerator(sc, ctx, jobConf, emptyScratchDir, sparkReporter);
     SparkPlan plan;
     try {
       plan = gen.generate(sparkWork);
@@ -192,6 +198,7 @@ public class SparkClient implements Seri
       LOG.error("Error executing Spark Plan", e);
       return 1;
     }
+
     return 0;
   }
 

Modified: hive/branches/spark/ql/src/java/org/apache/hadoop/hive/ql/exec/spark/SparkPlanGenerator.java
URL: http://svn.apache.org/viewvc/hive/branches/spark/ql/src/java/org/apache/hadoop/hive/ql/exec/spark/SparkPlanGenerator.java?rev=1633382&r1=1633381&r2=1633382&view=diff
==============================================================================
--- hive/branches/spark/ql/src/java/org/apache/hadoop/hive/ql/exec/spark/SparkPlanGenerator.java (original)
+++ hive/branches/spark/ql/src/java/org/apache/hadoop/hive/ql/exec/spark/SparkPlanGenerator.java Tue Oct 21 14:00:01 2014
@@ -71,12 +71,18 @@ public class SparkPlanGenerator {
   private final JobConf jobConf;
   private Context context;
   private Path scratchDir;
+  private SparkReporter sparkReporter;
   private final Map<BaseWork, BaseWork> cloneToWork;
   private final Map<BaseWork, SparkTran> workToTranMap;
   private final Map<BaseWork, SparkTran> workToParentWorkTranMap;
 
-  public SparkPlanGenerator(JavaSparkContext sc, Context context,
-      JobConf jobConf, Path scratchDir) {
+  public SparkPlanGenerator(
+    JavaSparkContext sc,
+    Context context,
+    JobConf jobConf,
+    Path scratchDir,
+    SparkReporter sparkReporter) {
+
     this.sc = sc;
     this.context = context;
     this.jobConf = jobConf;
@@ -84,6 +90,7 @@ public class SparkPlanGenerator {
     this.cloneToWork = new HashMap<BaseWork, BaseWork>();
     this.workToTranMap = new HashMap<BaseWork, SparkTran>();
     this.workToParentWorkTranMap = new HashMap<BaseWork, SparkTran>();
+    this.sparkReporter = sparkReporter;
   }
 
   public SparkPlan generate(SparkWork sparkWork) throws Exception {
@@ -311,7 +318,7 @@ public class SparkPlanGenerator {
     MapTran result = new MapTran();
     JobConf newJobConf = cloneJobConf(mw);
     byte[] confBytes = KryoSerializer.serializeJobConf(newJobConf);
-    HiveMapFunction mapFunc = new HiveMapFunction(confBytes);
+    HiveMapFunction mapFunc = new HiveMapFunction(confBytes, sparkReporter);
     result.setMapFunction(mapFunc);
     return result;
   }
@@ -320,7 +327,7 @@ public class SparkPlanGenerator {
     ReduceTran result = new ReduceTran();
     JobConf newJobConf = cloneJobConf(rw);
     byte[] confBytes = KryoSerializer.serializeJobConf(newJobConf);
-    HiveReduceFunction redFunc = new HiveReduceFunction(confBytes);
+    HiveReduceFunction redFunc = new HiveReduceFunction(confBytes, sparkReporter);
     result.setReduceFunction(redFunc);
     return result;
   }

Modified: hive/branches/spark/ql/src/java/org/apache/hadoop/hive/ql/exec/spark/SparkRecordHandler.java
URL: http://svn.apache.org/viewvc/hive/branches/spark/ql/src/java/org/apache/hadoop/hive/ql/exec/spark/SparkRecordHandler.java?rev=1633382&r1=1633381&r2=1633382&view=diff
==============================================================================
--- hive/branches/spark/ql/src/java/org/apache/hadoop/hive/ql/exec/spark/SparkRecordHandler.java (original)
+++ hive/branches/spark/ql/src/java/org/apache/hadoop/hive/ql/exec/spark/SparkRecordHandler.java Tue Oct 21 14:00:01 2014
@@ -51,7 +51,7 @@ public abstract class SparkRecordHandler
 
     oc = output;
     rp = reporter;
-    MapredContext.get().setReporter(reporter);
+//    MapredContext.get().setReporter(reporter);
 
     LOG.info("maximum memory = " + memoryMXBean.getHeapMemoryUsage().getMax());
 

Added: hive/branches/spark/ql/src/java/org/apache/hadoop/hive/ql/exec/spark/SparkReporter.java
URL: http://svn.apache.org/viewvc/hive/branches/spark/ql/src/java/org/apache/hadoop/hive/ql/exec/spark/SparkReporter.java?rev=1633382&view=auto
==============================================================================
--- hive/branches/spark/ql/src/java/org/apache/hadoop/hive/ql/exec/spark/SparkReporter.java (added)
+++ hive/branches/spark/ql/src/java/org/apache/hadoop/hive/ql/exec/spark/SparkReporter.java Tue Oct 21 14:00:01 2014
@@ -0,0 +1,83 @@
+/**
+ * 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.hadoop.hive.ql.exec.spark;
+
+import org.apache.hadoop.hive.ql.exec.spark.counter.SparkCounters;
+import org.apache.hadoop.mapred.Counters.Counter;
+import org.apache.hadoop.mapred.InputSplit;
+import org.apache.hadoop.mapred.Reporter;
+
+import java.io.Serializable;
+
+/**
+ * Implement SparkReporter for Hive operator level statistics collection, and throw
+ * UnsupportedOperationException for other unrelated methods, so if any Hive feature
+ * depends on these unimplemented methods, we could go back here quickly and enable it.
+ */
+public class SparkReporter implements Reporter, Serializable {
+
+  private SparkCounters sparkCounters;
+  private String status;
+  public SparkReporter(SparkCounters sparkCounters) {
+    this.sparkCounters = sparkCounters;
+  }
+
+  @Override
+  public void setStatus(String status) {
+    this.status = status;
+  }
+
+  public String getStatus() {
+    return this.status;
+  }
+
+  @Override
+  public Counter getCounter(Enum<?> name) {
+    throw new UnsupportedOperationException("do not support this method now.");
+  }
+
+  @Override
+  public Counter getCounter(String group, String name) {
+    throw new UnsupportedOperationException("do not support this method now.");
+  }
+
+  @Override
+  public void incrCounter(Enum<?> key, long amount) {
+    sparkCounters.increment(key.getDeclaringClass().getName(), key.name(), amount);
+  }
+
+  @Override
+  public void incrCounter(String group, String counter, long amount) {
+    sparkCounters.increment(group, counter, amount);
+  }
+
+  @Override
+  public InputSplit getInputSplit() throws UnsupportedOperationException {
+    throw new UnsupportedOperationException("do not support this method now.");
+  }
+
+  @Override
+  public float getProgress() {
+    throw new UnsupportedOperationException("do not support this method now.");
+  }
+
+  @Override
+  public void progress() {
+    //do not support task level progress, do nothing here.
+  }
+}

Modified: hive/branches/spark/ql/src/java/org/apache/hadoop/hive/ql/exec/spark/counter/SparkCounters.java
URL: http://svn.apache.org/viewvc/hive/branches/spark/ql/src/java/org/apache/hadoop/hive/ql/exec/spark/counter/SparkCounters.java?rev=1633382&r1=1633381&r2=1633382&view=diff
==============================================================================
--- hive/branches/spark/ql/src/java/org/apache/hadoop/hive/ql/exec/spark/counter/SparkCounters.java (original)
+++ hive/branches/spark/ql/src/java/org/apache/hadoop/hive/ql/exec/spark/counter/SparkCounters.java Tue Oct 21 14:00:01 2014
@@ -23,8 +23,11 @@ import java.util.Map;
 
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.hive.conf.HiveConf;
+import org.apache.hadoop.hive.ql.exec.FilterOperator;
+import org.apache.hadoop.hive.ql.exec.JoinOperator;
 import org.apache.hadoop.hive.ql.exec.MapOperator;
 import org.apache.hadoop.hive.ql.exec.Operator;
+import org.apache.hadoop.hive.ql.exec.ScriptOperator;
 import org.apache.hadoop.mapreduce.util.ResourceBundles;
 import org.apache.spark.api.java.JavaSparkContext;
 
@@ -61,6 +64,11 @@ public class SparkCounters implements Se
     createCounter(HiveConf.getVar(hiveConf, HiveConf.ConfVars.HIVECOUNTERGROUP),
       Operator.HIVECOUNTERCREATEDFILES);
     createCounter(MapOperator.Counter.DESERIALIZE_ERRORS);
+    createCounter(FilterOperator.Counter.FILTERED);
+    createCounter(FilterOperator.Counter.PASSED);
+    createCounter(ScriptOperator.Counter.DESERIALIZE_ERRORS);
+    createCounter(ScriptOperator.Counter.SERIALIZE_ERRORS);
+    createCounter(JoinOperator.SkewkeyTableCounter.SKEWJOINFOLLOWUPJOBS);
   }
 
   public void createCounter(Enum<?> key) {