You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@hive.apache.org by px...@apache.org on 2015/08/16 02:07:09 UTC
hive git commit: HIVE-10816: NPE in ExecDriver::handleSampling when
submitted via child JVM (Rui reviewed by Xuefu)
Repository: hive
Updated Branches:
refs/heads/branch-1.2 7ad22e164 -> f8b2735f8
HIVE-10816: NPE in ExecDriver::handleSampling when submitted via child JVM (Rui reviewed by Xuefu)
Project: http://git-wip-us.apache.org/repos/asf/hive/repo
Commit: http://git-wip-us.apache.org/repos/asf/hive/commit/f8b2735f
Tree: http://git-wip-us.apache.org/repos/asf/hive/tree/f8b2735f
Diff: http://git-wip-us.apache.org/repos/asf/hive/diff/f8b2735f
Branch: refs/heads/branch-1.2
Commit: f8b2735f89f725149921b13ad97215b7639bd039
Parents: 7ad22e1
Author: Rui Li <ru...@intel.com>
Authored: Tue Jun 9 15:30:14 2015 +0800
Committer: Pengcheng Xiong <px...@apache.org>
Committed: Sat Aug 15 16:59:30 2015 -0700
----------------------------------------------------------------------
.../apache/hadoop/hive/ql/exec/PartitionKeySampler.java | 9 +++++----
.../org/apache/hadoop/hive/ql/exec/mr/ExecDriver.java | 12 ++++++------
2 files changed, 11 insertions(+), 10 deletions(-)
----------------------------------------------------------------------
http://git-wip-us.apache.org/repos/asf/hive/blob/f8b2735f/ql/src/java/org/apache/hadoop/hive/ql/exec/PartitionKeySampler.java
----------------------------------------------------------------------
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/exec/PartitionKeySampler.java b/ql/src/java/org/apache/hadoop/hive/ql/exec/PartitionKeySampler.java
index 96f4530..dc1b601 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/exec/PartitionKeySampler.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/exec/PartitionKeySampler.java
@@ -112,7 +112,7 @@ public class PartitionKeySampler implements OutputCollector<HiveKey, Object> {
return partitionKeys;
}
- public void writePartitionKeys(Path path, HiveConf conf, JobConf job) throws IOException {
+ public void writePartitionKeys(Path path, JobConf job) throws IOException {
byte[][] partitionKeys = getPartitionKeys(job.getNumReduceTasks());
int numPartition = partitionKeys.length + 1;
if (numPartition != job.getNumReduceTasks()) {
@@ -133,10 +133,11 @@ public class PartitionKeySampler implements OutputCollector<HiveKey, Object> {
}
// random sampling
- public static FetchOperator createSampler(FetchWork work, HiveConf conf, JobConf job,
+ public static FetchOperator createSampler(FetchWork work, JobConf job,
Operator<?> operator) throws HiveException {
- int sampleNum = conf.getIntVar(HiveConf.ConfVars.HIVESAMPLINGNUMBERFORORDERBY);
- float samplePercent = conf.getFloatVar(HiveConf.ConfVars.HIVESAMPLINGPERCENTFORORDERBY);
+ int sampleNum = HiveConf.getIntVar(job, HiveConf.ConfVars.HIVESAMPLINGNUMBERFORORDERBY);
+ float samplePercent =
+ HiveConf.getFloatVar(job, HiveConf.ConfVars.HIVESAMPLINGPERCENTFORORDERBY);
if (samplePercent < 0.0 || samplePercent > 1.0) {
throw new IllegalArgumentException("Percentile value must be within the range of 0 to 1.");
}
http://git-wip-us.apache.org/repos/asf/hive/blob/f8b2735f/ql/src/java/org/apache/hadoop/hive/ql/exec/mr/ExecDriver.java
----------------------------------------------------------------------
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/exec/mr/ExecDriver.java b/ql/src/java/org/apache/hadoop/hive/ql/exec/mr/ExecDriver.java
index d7a08ec..41d0493 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/exec/mr/ExecDriver.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/exec/mr/ExecDriver.java
@@ -376,7 +376,7 @@ public class ExecDriver extends Task<MapredWork> implements Serializable, Hadoop
if (mWork.getSamplingType() > 0 && rWork != null && job.getNumReduceTasks() > 1) {
try {
- handleSampling(driverContext, mWork, job, conf);
+ handleSampling(ctx, mWork, job);
job.setPartitionerClass(HiveTotalOrderPartitioner.class);
} catch (IllegalStateException e) {
console.printInfo("Not enough sampling data.. Rolling back to single reducer task");
@@ -494,7 +494,7 @@ public class ExecDriver extends Task<MapredWork> implements Serializable, Hadoop
return (returnVal);
}
- private void handleSampling(DriverContext context, MapWork mWork, JobConf job, HiveConf conf)
+ private void handleSampling(Context context, MapWork mWork, JobConf job)
throws Exception {
assert mWork.getAliasToWork().keySet().size() == 1;
@@ -510,7 +510,7 @@ public class ExecDriver extends Task<MapredWork> implements Serializable, Hadoop
inputPaths.add(new Path(path));
}
- Path tmpPath = context.getCtx().getExternalTmpPath(inputPaths.get(0));
+ Path tmpPath = context.getExternalTmpPath(inputPaths.get(0));
Path partitionFile = new Path(tmpPath, ".partitions");
ShimLoader.getHadoopShims().setTotalOrderPartitionFile(job, partitionFile);
PartitionKeySampler sampler = new PartitionKeySampler();
@@ -539,9 +539,9 @@ public class ExecDriver extends Task<MapredWork> implements Serializable, Hadoop
fetchWork.setSource(ts);
// random sampling
- FetchOperator fetcher = PartitionKeySampler.createSampler(fetchWork, conf, job, ts);
+ FetchOperator fetcher = PartitionKeySampler.createSampler(fetchWork, job, ts);
try {
- ts.initialize(conf, new ObjectInspector[]{fetcher.getOutputObjectInspector()});
+ ts.initialize(job, new ObjectInspector[]{fetcher.getOutputObjectInspector()});
OperatorUtils.setChildrenCollector(ts.getChildOperators(), sampler);
while (fetcher.pushRow()) { }
} finally {
@@ -550,7 +550,7 @@ public class ExecDriver extends Task<MapredWork> implements Serializable, Hadoop
} else {
throw new IllegalArgumentException("Invalid sampling type " + mWork.getSamplingType());
}
- sampler.writePartitionKeys(partitionFile, conf, job);
+ sampler.writePartitionKeys(partitionFile, job);
}
/**