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) {