[SPARK-19660][CORE][SQL] Replace the configuration property names that are deprecated in the version of Hadoop 2.6
## What changes were proposed in this pull request? Replace all the Hadoop deprecated configuration property names according to [DeprecatedProperties](https://hadoop.apache.org/docs/r2.6.0/hadoop-project-dist/hadoop-common/DeprecatedProperties.html). except: https://github.com/apache/spark/blob/v2.1.0/python/pyspark/sql/tests.py#L1533 https://github.com/apache/spark/blob/v2.1.0/sql/core/src/test/scala/org/apache/spark/sql/SQLQuerySuite.scala#L987 https://github.com/apache/spark/blob/v2.1.0/sql/core/src/main/scala/org/apache/spark/sql/execution/command/SetCommand.scala#L45 https://github.com/apache/spark/blob/v2.1.0/sql/core/src/main/scala/org/apache/spark/sql/internal/SQLConf.scala#L614 ## How was this patch tested? Existing tests Author: Yuming Wang <wgyumg@gmail.com> Closes #16990 from wangyum/HadoopDeprecatedProperties.
This commit is contained in:
parent
a350bc16d3
commit
9b8eca65dc
|
@ -38,6 +38,6 @@ To run the SparkR unit tests on Windows, the following steps are required —ass
|
|||
|
||||
```
|
||||
R -e "install.packages('testthat', repos='http://cran.us.r-project.org')"
|
||||
.\bin\spark-submit2.cmd --conf spark.hadoop.fs.default.name="file:///" R\pkg\tests\run-all.R
|
||||
.\bin\spark-submit2.cmd --conf spark.hadoop.fs.defaultFS="file:///" R\pkg\tests\run-all.R
|
||||
```
|
||||
|
||||
|
|
|
@ -23,7 +23,7 @@ FAILED=0
|
|||
LOGFILE=$FWDIR/unit-tests.out
|
||||
rm -f $LOGFILE
|
||||
|
||||
SPARK_TESTING=1 $FWDIR/../bin/spark-submit --driver-java-options "-Dlog4j.configuration=file:$FWDIR/log4j.properties" --conf spark.hadoop.fs.default.name="file:///" $FWDIR/pkg/tests/run-all.R 2>&1 | tee -a $LOGFILE
|
||||
SPARK_TESTING=1 $FWDIR/../bin/spark-submit --driver-java-options "-Dlog4j.configuration=file:$FWDIR/log4j.properties" --conf spark.hadoop.fs.defaultFS="file:///" $FWDIR/pkg/tests/run-all.R 2>&1 | tee -a $LOGFILE
|
||||
FAILED=$((PIPESTATUS[0]||$FAILED))
|
||||
|
||||
NUM_TEST_WARNING="$(grep -c -e 'Warnings ----------------' $LOGFILE)"
|
||||
|
|
|
@ -46,7 +46,7 @@ build_script:
|
|||
- cmd: mvn -DskipTests -Psparkr -Phive -Phive-thriftserver package
|
||||
|
||||
test_script:
|
||||
- cmd: .\bin\spark-submit2.cmd --conf spark.hadoop.fs.default.name="file:///" R\pkg\tests\run-all.R
|
||||
- cmd: .\bin\spark-submit2.cmd --conf spark.hadoop.fs.defaultFS="file:///" R\pkg\tests\run-all.R
|
||||
|
||||
notifications:
|
||||
- provider: Email
|
||||
|
|
|
@ -113,11 +113,11 @@ class HadoopMapReduceCommitProtocol(jobId: String, path: String)
|
|||
val taskAttemptId = new TaskAttemptID(taskId, 0)
|
||||
|
||||
// Set up the configuration object
|
||||
jobContext.getConfiguration.set("mapred.job.id", jobId.toString)
|
||||
jobContext.getConfiguration.set("mapred.tip.id", taskAttemptId.getTaskID.toString)
|
||||
jobContext.getConfiguration.set("mapred.task.id", taskAttemptId.toString)
|
||||
jobContext.getConfiguration.setBoolean("mapred.task.is.map", true)
|
||||
jobContext.getConfiguration.setInt("mapred.task.partition", 0)
|
||||
jobContext.getConfiguration.set("mapreduce.job.id", jobId.toString)
|
||||
jobContext.getConfiguration.set("mapreduce.task.id", taskAttemptId.getTaskID.toString)
|
||||
jobContext.getConfiguration.set("mapreduce.task.attempt.id", taskAttemptId.toString)
|
||||
jobContext.getConfiguration.setBoolean("mapreduce.task.ismap", true)
|
||||
jobContext.getConfiguration.setInt("mapreduce.task.partition", 0)
|
||||
|
||||
val taskAttemptContext = new TaskAttemptContextImpl(jobContext.getConfiguration, taskAttemptId)
|
||||
committer = setupCommitter(taskAttemptContext)
|
||||
|
|
|
@ -79,7 +79,7 @@ object SparkHadoopMapReduceWriter extends Logging {
|
|||
val committer = FileCommitProtocol.instantiate(
|
||||
className = classOf[HadoopMapReduceCommitProtocol].getName,
|
||||
jobId = stageId.toString,
|
||||
outputPath = conf.value.get("mapred.output.dir"),
|
||||
outputPath = conf.value.get("mapreduce.output.fileoutputformat.outputdir"),
|
||||
isAppend = false).asInstanceOf[HadoopMapReduceCommitProtocol]
|
||||
committer.setupJob(jobContext)
|
||||
|
||||
|
|
|
@ -365,11 +365,11 @@ private[spark] object HadoopRDD extends Logging {
|
|||
val jobID = new JobID(jobTrackerId, jobId)
|
||||
val taId = new TaskAttemptID(new TaskID(jobID, TaskType.MAP, splitId), attemptId)
|
||||
|
||||
conf.set("mapred.tip.id", taId.getTaskID.toString)
|
||||
conf.set("mapred.task.id", taId.toString)
|
||||
conf.setBoolean("mapred.task.is.map", true)
|
||||
conf.setInt("mapred.task.partition", splitId)
|
||||
conf.set("mapred.job.id", jobID.toString)
|
||||
conf.set("mapreduce.task.id", taId.getTaskID.toString)
|
||||
conf.set("mapreduce.task.attempt.id", taId.toString)
|
||||
conf.setBoolean("mapreduce.task.ismap", true)
|
||||
conf.setInt("mapreduce.task.partition", splitId)
|
||||
conf.set("mapreduce.job.id", jobID.toString)
|
||||
}
|
||||
|
||||
/**
|
||||
|
|
|
@ -998,7 +998,7 @@ class PairRDDFunctions[K, V](self: RDD[(K, V)])
|
|||
job.setOutputValueClass(valueClass)
|
||||
job.setOutputFormatClass(outputFormatClass)
|
||||
val jobConfiguration = job.getConfiguration
|
||||
jobConfiguration.set("mapred.output.dir", path)
|
||||
jobConfiguration.set("mapreduce.output.fileoutputformat.outputdir", path)
|
||||
saveAsNewAPIHadoopDataset(jobConfiguration)
|
||||
}
|
||||
|
||||
|
@ -1039,10 +1039,11 @@ class PairRDDFunctions[K, V](self: RDD[(K, V)])
|
|||
conf.setOutputFormat(outputFormatClass)
|
||||
for (c <- codec) {
|
||||
hadoopConf.setCompressMapOutput(true)
|
||||
hadoopConf.set("mapred.output.compress", "true")
|
||||
hadoopConf.set("mapreduce.output.fileoutputformat.compress", "true")
|
||||
hadoopConf.setMapOutputCompressorClass(c)
|
||||
hadoopConf.set("mapred.output.compression.codec", c.getCanonicalName)
|
||||
hadoopConf.set("mapred.output.compression.type", CompressionType.BLOCK.toString)
|
||||
hadoopConf.set("mapreduce.output.fileoutputformat.compress.codec", c.getCanonicalName)
|
||||
hadoopConf.set("mapreduce.output.fileoutputformat.compress.type",
|
||||
CompressionType.BLOCK.toString)
|
||||
}
|
||||
|
||||
// Use configured output committer if already set
|
||||
|
|
|
@ -401,7 +401,7 @@ class FileSuite extends SparkFunSuite with LocalSparkContext {
|
|||
job.setOutputKeyClass(classOf[String])
|
||||
job.setOutputValueClass(classOf[String])
|
||||
job.set("mapred.output.format.class", classOf[TextOutputFormat[String, String]].getName)
|
||||
job.set("mapred.output.dir", tempDir.getPath + "/outputDataset_old")
|
||||
job.set("mapreduce.output.fileoutputformat.outputdir", tempDir.getPath + "/outputDataset_old")
|
||||
randomRDD.saveAsHadoopDataset(job)
|
||||
assert(new File(tempDir.getPath + "/outputDataset_old/part-00000").exists() === true)
|
||||
}
|
||||
|
@ -415,7 +415,8 @@ class FileSuite extends SparkFunSuite with LocalSparkContext {
|
|||
job.setOutputValueClass(classOf[String])
|
||||
job.setOutputFormatClass(classOf[NewTextOutputFormat[String, String]])
|
||||
val jobConfig = job.getConfiguration
|
||||
jobConfig.set("mapred.output.dir", tempDir.getPath + "/outputDataset_new")
|
||||
jobConfig.set("mapreduce.output.fileoutputformat.outputdir",
|
||||
tempDir.getPath + "/outputDataset_new")
|
||||
randomRDD.saveAsNewAPIHadoopDataset(jobConfig)
|
||||
assert(new File(tempDir.getPath + "/outputDataset_new/part-r-00000").exists() === true)
|
||||
}
|
||||
|
|
|
@ -15,8 +15,8 @@ possible**. We recommend the following:
|
|||
* If at all possible, run Spark on the same nodes as HDFS. The simplest way is to set up a Spark
|
||||
[standalone mode cluster](spark-standalone.html) on the same nodes, and configure Spark and
|
||||
Hadoop's memory and CPU usage to avoid interference (for Hadoop, the relevant options are
|
||||
`mapred.child.java.opts` for the per-task memory and `mapred.tasktracker.map.tasks.maximum`
|
||||
and `mapred.tasktracker.reduce.tasks.maximum` for number of tasks). Alternatively, you can run
|
||||
`mapred.child.java.opts` for the per-task memory and `mapreduce.tasktracker.map.tasks.maximum`
|
||||
and `mapreduce.tasktracker.reduce.tasks.maximum` for number of tasks). Alternatively, you can run
|
||||
Hadoop and Spark on a common cluster manager like [Mesos](running-on-mesos.html) or
|
||||
[Hadoop YARN](running-on-yarn.html).
|
||||
|
||||
|
|
|
@ -1347,7 +1347,7 @@ class InputFormatTests(ReusedPySparkTestCase):
|
|||
self.assertEqual(ints, ei)
|
||||
|
||||
hellopath = os.path.join(SPARK_HOME, "python/test_support/hello/hello.txt")
|
||||
oldconf = {"mapred.input.dir": hellopath}
|
||||
oldconf = {"mapreduce.input.fileinputformat.inputdir": hellopath}
|
||||
hello = self.sc.hadoopRDD("org.apache.hadoop.mapred.TextInputFormat",
|
||||
"org.apache.hadoop.io.LongWritable",
|
||||
"org.apache.hadoop.io.Text",
|
||||
|
@ -1366,7 +1366,7 @@ class InputFormatTests(ReusedPySparkTestCase):
|
|||
self.assertEqual(ints, ei)
|
||||
|
||||
hellopath = os.path.join(SPARK_HOME, "python/test_support/hello/hello.txt")
|
||||
newconf = {"mapred.input.dir": hellopath}
|
||||
newconf = {"mapreduce.input.fileinputformat.inputdir": hellopath}
|
||||
hello = self.sc.newAPIHadoopRDD("org.apache.hadoop.mapreduce.lib.input.TextInputFormat",
|
||||
"org.apache.hadoop.io.LongWritable",
|
||||
"org.apache.hadoop.io.Text",
|
||||
|
@ -1515,12 +1515,12 @@ class OutputFormatTests(ReusedPySparkTestCase):
|
|||
|
||||
conf = {
|
||||
"mapred.output.format.class": "org.apache.hadoop.mapred.SequenceFileOutputFormat",
|
||||
"mapred.output.key.class": "org.apache.hadoop.io.IntWritable",
|
||||
"mapred.output.value.class": "org.apache.hadoop.io.MapWritable",
|
||||
"mapred.output.dir": basepath + "/olddataset/"
|
||||
"mapreduce.job.output.key.class": "org.apache.hadoop.io.IntWritable",
|
||||
"mapreduce.job.output.value.class": "org.apache.hadoop.io.MapWritable",
|
||||
"mapreduce.output.fileoutputformat.outputdir": basepath + "/olddataset/"
|
||||
}
|
||||
self.sc.parallelize(dict_data).saveAsHadoopDataset(conf)
|
||||
input_conf = {"mapred.input.dir": basepath + "/olddataset/"}
|
||||
input_conf = {"mapreduce.input.fileinputformat.inputdir": basepath + "/olddataset/"}
|
||||
result = self.sc.hadoopRDD(
|
||||
"org.apache.hadoop.mapred.SequenceFileInputFormat",
|
||||
"org.apache.hadoop.io.IntWritable",
|
||||
|
@ -1547,14 +1547,14 @@ class OutputFormatTests(ReusedPySparkTestCase):
|
|||
self.assertEqual(result, data)
|
||||
|
||||
conf = {
|
||||
"mapreduce.outputformat.class":
|
||||
"mapreduce.job.outputformat.class":
|
||||
"org.apache.hadoop.mapreduce.lib.output.SequenceFileOutputFormat",
|
||||
"mapred.output.key.class": "org.apache.hadoop.io.IntWritable",
|
||||
"mapred.output.value.class": "org.apache.hadoop.io.Text",
|
||||
"mapred.output.dir": basepath + "/newdataset/"
|
||||
"mapreduce.job.output.key.class": "org.apache.hadoop.io.IntWritable",
|
||||
"mapreduce.job.output.value.class": "org.apache.hadoop.io.Text",
|
||||
"mapreduce.output.fileoutputformat.outputdir": basepath + "/newdataset/"
|
||||
}
|
||||
self.sc.parallelize(data).saveAsNewAPIHadoopDataset(conf)
|
||||
input_conf = {"mapred.input.dir": basepath + "/newdataset/"}
|
||||
input_conf = {"mapreduce.input.fileinputformat.inputdir": basepath + "/newdataset/"}
|
||||
new_dataset = sorted(self.sc.newAPIHadoopRDD(
|
||||
"org.apache.hadoop.mapreduce.lib.input.SequenceFileInputFormat",
|
||||
"org.apache.hadoop.io.IntWritable",
|
||||
|
@ -1584,16 +1584,16 @@ class OutputFormatTests(ReusedPySparkTestCase):
|
|||
self.assertEqual(result, array_data)
|
||||
|
||||
conf = {
|
||||
"mapreduce.outputformat.class":
|
||||
"mapreduce.job.outputformat.class":
|
||||
"org.apache.hadoop.mapreduce.lib.output.SequenceFileOutputFormat",
|
||||
"mapred.output.key.class": "org.apache.hadoop.io.IntWritable",
|
||||
"mapred.output.value.class": "org.apache.spark.api.python.DoubleArrayWritable",
|
||||
"mapred.output.dir": basepath + "/newdataset/"
|
||||
"mapreduce.job.output.key.class": "org.apache.hadoop.io.IntWritable",
|
||||
"mapreduce.job.output.value.class": "org.apache.spark.api.python.DoubleArrayWritable",
|
||||
"mapreduce.output.fileoutputformat.outputdir": basepath + "/newdataset/"
|
||||
}
|
||||
self.sc.parallelize(array_data).saveAsNewAPIHadoopDataset(
|
||||
conf,
|
||||
valueConverter="org.apache.spark.api.python.DoubleArrayToWritableConverter")
|
||||
input_conf = {"mapred.input.dir": basepath + "/newdataset/"}
|
||||
input_conf = {"mapreduce.input.fileinputformat.inputdir": basepath + "/newdataset/"}
|
||||
new_dataset = sorted(self.sc.newAPIHadoopRDD(
|
||||
"org.apache.hadoop.mapreduce.lib.input.SequenceFileInputFormat",
|
||||
"org.apache.hadoop.io.IntWritable",
|
||||
|
@ -1663,18 +1663,19 @@ class OutputFormatTests(ReusedPySparkTestCase):
|
|||
|
||||
conf4 = {
|
||||
"mapred.output.format.class": "org.apache.hadoop.mapred.SequenceFileOutputFormat",
|
||||
"mapred.output.key.class": "org.apache.hadoop.io.IntWritable",
|
||||
"mapred.output.value.class": "org.apache.hadoop.io.IntWritable",
|
||||
"mapred.output.dir": basepath + "/reserialize/dataset"}
|
||||
"mapreduce.job.output.key.class": "org.apache.hadoop.io.IntWritable",
|
||||
"mapreduce.job.output.value.class": "org.apache.hadoop.io.IntWritable",
|
||||
"mapreduce.output.fileoutputformat.outputdir": basepath + "/reserialize/dataset"}
|
||||
rdd.saveAsHadoopDataset(conf4)
|
||||
result4 = sorted(self.sc.sequenceFile(basepath + "/reserialize/dataset").collect())
|
||||
self.assertEqual(result4, data)
|
||||
|
||||
conf5 = {"mapreduce.outputformat.class":
|
||||
conf5 = {"mapreduce.job.outputformat.class":
|
||||
"org.apache.hadoop.mapreduce.lib.output.SequenceFileOutputFormat",
|
||||
"mapred.output.key.class": "org.apache.hadoop.io.IntWritable",
|
||||
"mapred.output.value.class": "org.apache.hadoop.io.IntWritable",
|
||||
"mapred.output.dir": basepath + "/reserialize/newdataset"}
|
||||
"mapreduce.job.output.key.class": "org.apache.hadoop.io.IntWritable",
|
||||
"mapreduce.job.output.value.class": "org.apache.hadoop.io.IntWritable",
|
||||
"mapreduce.output.fileoutputformat.outputdir": basepath + "/reserialize/newdataset"
|
||||
}
|
||||
rdd.saveAsNewAPIHadoopDataset(conf5)
|
||||
result5 = sorted(self.sc.sequenceFile(basepath + "/reserialize/newdataset").collect())
|
||||
self.assertEqual(result5, data)
|
||||
|
|
|
@ -269,8 +269,8 @@ case class LoadDataCommand(
|
|||
} else {
|
||||
// Follow Hive's behavior:
|
||||
// If no schema or authority is provided with non-local inpath,
|
||||
// we will use hadoop configuration "fs.default.name".
|
||||
val defaultFSConf = sparkSession.sessionState.newHadoopConf().get("fs.default.name")
|
||||
// we will use hadoop configuration "fs.defaultFS".
|
||||
val defaultFSConf = sparkSession.sessionState.newHadoopConf().get("fs.defaultFS")
|
||||
val defaultFS = if (defaultFSConf == null) {
|
||||
new URI("")
|
||||
} else {
|
||||
|
|
|
@ -210,11 +210,11 @@ object FileFormatWriter extends Logging {
|
|||
val taskAttemptContext: TaskAttemptContext = {
|
||||
// Set up the configuration object
|
||||
val hadoopConf = description.serializableHadoopConf.value
|
||||
hadoopConf.set("mapred.job.id", jobId.toString)
|
||||
hadoopConf.set("mapred.tip.id", taskAttemptId.getTaskID.toString)
|
||||
hadoopConf.set("mapred.task.id", taskAttemptId.toString)
|
||||
hadoopConf.setBoolean("mapred.task.is.map", true)
|
||||
hadoopConf.setInt("mapred.task.partition", 0)
|
||||
hadoopConf.set("mapreduce.job.id", jobId.toString)
|
||||
hadoopConf.set("mapreduce.task.id", taskAttemptId.getTaskID.toString)
|
||||
hadoopConf.set("mapreduce.task.attempt.id", taskAttemptId.toString)
|
||||
hadoopConf.setBoolean("mapreduce.task.ismap", true)
|
||||
hadoopConf.setInt("mapreduce.task.partition", 0)
|
||||
|
||||
new TaskAttemptContextImpl(hadoopConf, taskAttemptId)
|
||||
}
|
||||
|
|
|
@ -95,9 +95,9 @@ class HiveWindowFunctionQuerySuite extends HiveComparisonTest with BeforeAndAfte
|
|||
// This is used to generate golden files.
|
||||
sql("set hive.plan.serialization.format=kryo")
|
||||
// Explicitly set fs to local fs.
|
||||
sql(s"set fs.default.name=file://$testTempDir/")
|
||||
sql(s"set fs.defaultFS=file://$testTempDir/")
|
||||
// Ask Hive to run jobs in-process as a single map and reduce task.
|
||||
sql("set mapred.job.tracker=local")
|
||||
sql("set mapreduce.jobtracker.address=local")
|
||||
}
|
||||
|
||||
override def afterAll() {
|
||||
|
@ -764,9 +764,9 @@ class HiveWindowFunctionQueryFileSuite
|
|||
// This is used to generate golden files.
|
||||
// sql("set hive.plan.serialization.format=kryo")
|
||||
// Explicitly set fs to local fs.
|
||||
// sql(s"set fs.default.name=file://$testTempDir/")
|
||||
// sql(s"set fs.defaultFS=file://$testTempDir/")
|
||||
// Ask Hive to run jobs in-process as a single map and reduce task.
|
||||
// sql("set mapred.job.tracker=local")
|
||||
// sql("set mapreduce.jobtracker.address=local")
|
||||
}
|
||||
|
||||
override def afterAll() {
|
||||
|
|
|
@ -66,14 +66,16 @@ class HadoopTableReader(
|
|||
hadoopConf: Configuration)
|
||||
extends TableReader with Logging {
|
||||
|
||||
// Hadoop honors "mapred.map.tasks" as hint, but will ignore when mapred.job.tracker is "local".
|
||||
// https://hadoop.apache.org/docs/r1.0.4/mapred-default.html
|
||||
// Hadoop honors "mapreduce.job.maps" as hint,
|
||||
// but will ignore when mapreduce.jobtracker.address is "local".
|
||||
// https://hadoop.apache.org/docs/r2.6.5/hadoop-mapreduce-client/hadoop-mapreduce-client-core/
|
||||
// mapred-default.xml
|
||||
//
|
||||
// In order keep consistency with Hive, we will let it be 0 in local mode also.
|
||||
private val _minSplitsPerRDD = if (sparkSession.sparkContext.isLocal) {
|
||||
0 // will splitted based on block by default.
|
||||
} else {
|
||||
math.max(hadoopConf.getInt("mapred.map.tasks", 1),
|
||||
math.max(hadoopConf.getInt("mapreduce.job.maps", 1),
|
||||
sparkSession.sparkContext.defaultMinPartitions)
|
||||
}
|
||||
|
||||
|
|
|
@ -228,13 +228,16 @@ case class InsertIntoHiveTable(
|
|||
val isCompressed = hadoopConf.get("hive.exec.compress.output", "false").toBoolean
|
||||
|
||||
if (isCompressed) {
|
||||
// Please note that isCompressed, "mapred.output.compress", "mapred.output.compression.codec",
|
||||
// and "mapred.output.compression.type" have no impact on ORC because it uses table properties
|
||||
// to store compression information.
|
||||
hadoopConf.set("mapred.output.compress", "true")
|
||||
// Please note that isCompressed, "mapreduce.output.fileoutputformat.compress",
|
||||
// "mapreduce.output.fileoutputformat.compress.codec", and
|
||||
// "mapreduce.output.fileoutputformat.compress.type"
|
||||
// have no impact on ORC because it uses table properties to store compression information.
|
||||
hadoopConf.set("mapreduce.output.fileoutputformat.compress", "true")
|
||||
fileSinkConf.setCompressed(true)
|
||||
fileSinkConf.setCompressCodec(hadoopConf.get("mapred.output.compression.codec"))
|
||||
fileSinkConf.setCompressType(hadoopConf.get("mapred.output.compression.type"))
|
||||
fileSinkConf.setCompressCodec(hadoopConf
|
||||
.get("mapreduce.output.fileoutputformat.compress.codec"))
|
||||
fileSinkConf.setCompressType(hadoopConf
|
||||
.get("mapreduce.output.fileoutputformat.compress.type"))
|
||||
}
|
||||
|
||||
val numDynamicPartitions = partition.values.count(_.isEmpty)
|
||||
|
|
|
@ -439,7 +439,7 @@ private[hive] class TestHiveSparkSession(
|
|||
foreach { udfName => FunctionRegistry.unregisterTemporaryUDF(udfName) }
|
||||
|
||||
// Some tests corrupt this value on purpose, which breaks the RESET call below.
|
||||
sessionState.conf.setConfString("fs.default.name", new File(".").toURI.toString)
|
||||
sessionState.conf.setConfString("fs.defaultFS", new File(".").toURI.toString)
|
||||
// It is important that we RESET first as broken hooks that might have been set could break
|
||||
// other sql exec here.
|
||||
sessionState.metadataHive.runSqlHive("RESET")
|
||||
|
|
|
@ -5,7 +5,7 @@ set hive.auto.convert.join = true;
|
|||
|
||||
CREATE TABLE dest1(c1 INT, c2 STRING) STORED AS TEXTFILE;
|
||||
|
||||
set mapred.job.tracker=localhost:58;
|
||||
set mapreduce.jobtracker.address=localhost:58;
|
||||
set hive.exec.mode.local.auto=true;
|
||||
|
||||
explain
|
||||
|
|
|
@ -4,7 +4,7 @@ set hive.enforce.sorting = true;
|
|||
set hive.exec.reducers.max = 1;
|
||||
set hive.merge.mapfiles = true;
|
||||
set hive.merge.mapredfiles = true;
|
||||
set mapred.reduce.tasks = 2;
|
||||
set mapreduce.job.reduces = 2;
|
||||
|
||||
-- Tests that when a multi insert inserts into a bucketed table and a table which is not bucketed
|
||||
-- the bucketed table is not merged and the table which is not bucketed is
|
||||
|
|
|
@ -1,6 +1,6 @@
|
|||
set hive.enforce.bucketing = true;
|
||||
set hive.exec.mode.local.auto=false;
|
||||
set mapred.reduce.tasks = 10;
|
||||
set mapreduce.job.reduces = 10;
|
||||
|
||||
-- This test sets number of mapred tasks to 10 for a database with 50 buckets,
|
||||
-- and uses a post-hook to confirm that 10 tasks were created
|
||||
|
|
|
@ -1,5 +1,5 @@
|
|||
set hive.input.format=org.apache.hadoop.hive.ql.io.BucketizedHiveInputFormat;
|
||||
set mapred.min.split.size = 64;
|
||||
set mapreduce.input.fileinputformat.split.minsize = 64;
|
||||
|
||||
CREATE TABLE T1(name STRING) STORED AS TEXTFILE;
|
||||
|
||||
|
|
|
@ -1,11 +1,11 @@
|
|||
set hive.exec.compress.output = true;
|
||||
set hive.input.format=org.apache.hadoop.hive.ql.io.CombineHiveInputFormat;
|
||||
set mapred.min.split.size=256;
|
||||
set mapred.min.split.size.per.node=256;
|
||||
set mapred.min.split.size.per.rack=256;
|
||||
set mapred.max.split.size=256;
|
||||
set mapreduce.input.fileinputformat.split.minsize=256;
|
||||
set mapreduce.input.fileinputformat.split.minsize.per.node=256;
|
||||
set mapreduce.input.fileinputformat.split.minsize.per.rack=256;
|
||||
set mapreduce.input.fileinputformat.split.maxsize=256;
|
||||
|
||||
set mapred.output.compression.codec=org.apache.hadoop.io.compress.GzipCodec;
|
||||
set mapreduce.output.fileoutputformat.compress.codec=org.apache.hadoop.io.compress.GzipCodec;
|
||||
|
||||
create table combine1_1(key string, value string) stored as textfile;
|
||||
|
||||
|
|
|
@ -1,10 +1,10 @@
|
|||
USE default;
|
||||
|
||||
set hive.input.format=org.apache.hadoop.hive.ql.io.CombineHiveInputFormat;
|
||||
set mapred.min.split.size=256;
|
||||
set mapred.min.split.size.per.node=256;
|
||||
set mapred.min.split.size.per.rack=256;
|
||||
set mapred.max.split.size=256;
|
||||
set mapreduce.input.fileinputformat.split.minsize=256;
|
||||
set mapreduce.input.fileinputformat.split.minsize.per.node=256;
|
||||
set mapreduce.input.fileinputformat.split.minsize.per.rack=256;
|
||||
set mapreduce.input.fileinputformat.split.maxsize=256;
|
||||
set hive.exec.dynamic.partition=true;
|
||||
set hive.exec.dynamic.partition.mode=nonstrict;
|
||||
set mapred.cache.shared.enabled=false;
|
||||
|
@ -18,7 +18,7 @@ set hive.merge.smallfiles.avgsize=0;
|
|||
create table combine2(key string) partitioned by (value string);
|
||||
|
||||
-- EXCLUDE_HADOOP_MAJOR_VERSIONS(0.20, 0.20S)
|
||||
-- This test sets mapred.max.split.size=256 and hive.merge.smallfiles.avgsize=0
|
||||
-- This test sets mapreduce.input.fileinputformat.split.maxsize=256 and hive.merge.smallfiles.avgsize=0
|
||||
-- in an attempt to force the generation of multiple splits and multiple output files.
|
||||
-- However, Hadoop 0.20 is incapable of generating splits smaller than the block size
|
||||
-- when using CombineFileInputFormat, so only one split is generated. This has a
|
||||
|
|
|
@ -1,10 +1,10 @@
|
|||
USE default;
|
||||
|
||||
set hive.input.format=org.apache.hadoop.hive.ql.io.CombineHiveInputFormat;
|
||||
set mapred.min.split.size=256;
|
||||
set mapred.min.split.size.per.node=256;
|
||||
set mapred.min.split.size.per.rack=256;
|
||||
set mapred.max.split.size=256;
|
||||
set mapreduce.input.fileinputformat.split.minsize=256;
|
||||
set mapreduce.input.fileinputformat.split.minsize.per.node=256;
|
||||
set mapreduce.input.fileinputformat.split.minsize.per.rack=256;
|
||||
set mapreduce.input.fileinputformat.split.maxsize=256;
|
||||
set hive.exec.dynamic.partition=true;
|
||||
set hive.exec.dynamic.partition.mode=nonstrict;
|
||||
set mapred.cache.shared.enabled=false;
|
||||
|
@ -17,7 +17,7 @@ set hive.merge.smallfiles.avgsize=0;
|
|||
create table combine2(key string) partitioned by (value string);
|
||||
|
||||
-- INCLUDE_HADOOP_MAJOR_VERSIONS(0.20, 0.20S)
|
||||
-- This test sets mapred.max.split.size=256 and hive.merge.smallfiles.avgsize=0
|
||||
-- This test sets mapreduce.input.fileinputformat.split.maxsize=256 and hive.merge.smallfiles.avgsize=0
|
||||
-- in an attempt to force the generation of multiple splits and multiple output files.
|
||||
-- However, Hadoop 0.20 is incapable of generating splits smaller than the block size
|
||||
-- when using CombineFileInputFormat, so only one split is generated. This has a
|
||||
|
|
|
@ -1,8 +1,8 @@
|
|||
set hive.input.format=org.apache.hadoop.hive.ql.io.CombineHiveInputFormat;
|
||||
set mapred.min.split.size=256;
|
||||
set mapred.min.split.size.per.node=256;
|
||||
set mapred.min.split.size.per.rack=256;
|
||||
set mapred.max.split.size=256;
|
||||
set mapreduce.input.fileinputformat.split.minsize=256;
|
||||
set mapreduce.input.fileinputformat.split.minsize.per.node=256;
|
||||
set mapreduce.input.fileinputformat.split.minsize.per.rack=256;
|
||||
set mapreduce.input.fileinputformat.split.maxsize=256;
|
||||
set hive.exec.dynamic.partition=true;
|
||||
set hive.exec.dynamic.partition.mode=nonstrict;
|
||||
set mapred.cache.shared.enabled=false;
|
||||
|
|
|
@ -1,9 +1,9 @@
|
|||
set hive.exec.compress.output = true;
|
||||
set hive.input.format=org.apache.hadoop.hive.ql.io.CombineHiveInputFormat;
|
||||
set mapred.min.split.size=256;
|
||||
set mapred.min.split.size.per.node=256;
|
||||
set mapred.min.split.size.per.rack=256;
|
||||
set mapred.max.split.size=256;
|
||||
set mapreduce.input.fileinputformat.split.minsize=256;
|
||||
set mapreduce.input.fileinputformat.split.minsize.per.node=256;
|
||||
set mapreduce.input.fileinputformat.split.minsize.per.rack=256;
|
||||
set mapreduce.input.fileinputformat.split.maxsize=256;
|
||||
|
||||
|
||||
drop table combine_3_srcpart_seq_rc;
|
||||
|
|
|
@ -1,4 +1,4 @@
|
|||
set fs.default.name=invalidscheme:///;
|
||||
set fs.defaultFS=invalidscheme:///;
|
||||
|
||||
CREATE TABLE table1 (a STRING, b STRING) STORED AS TEXTFILE;
|
||||
DESCRIBE table1;
|
||||
|
|
|
@ -49,7 +49,7 @@ describe formatted nzhang_CTAS4;
|
|||
|
||||
explain extended create table nzhang_ctas5 row format delimited fields terminated by ',' lines terminated by '\012' stored as textfile as select key, value from src sort by key, value limit 10;
|
||||
|
||||
set mapred.job.tracker=localhost:58;
|
||||
set mapreduce.jobtracker.address=localhost:58;
|
||||
set hive.exec.mode.local.auto=true;
|
||||
|
||||
create table nzhang_ctas5 row format delimited fields terminated by ',' lines terminated by '\012' stored as textfile as select key, value from src sort by key, value limit 10;
|
||||
|
|
|
@ -3,12 +3,12 @@ set hive.groupby.skewindata=true;
|
|||
|
||||
CREATE TABLE dest_g1(key INT, value DOUBLE) STORED AS TEXTFILE;
|
||||
|
||||
set fs.default.name=invalidscheme:///;
|
||||
set fs.defaultFS=invalidscheme:///;
|
||||
|
||||
EXPLAIN
|
||||
FROM src INSERT OVERWRITE TABLE dest_g1 SELECT src.key, sum(substr(src.value,5)) GROUP BY src.key;
|
||||
|
||||
set fs.default.name=file:///;
|
||||
set fs.defaultFS=file:///;
|
||||
|
||||
FROM src INSERT OVERWRITE TABLE dest_g1 SELECT src.key, sum(substr(src.value,5)) GROUP BY src.key;
|
||||
|
||||
|
|
|
@ -1,4 +1,4 @@
|
|||
set mapred.reduce.tasks=31;
|
||||
set mapreduce.job.reduces=31;
|
||||
|
||||
CREATE TABLE dest1(key INT, value DOUBLE) STORED AS TEXTFILE;
|
||||
|
||||
|
|
|
@ -1,6 +1,6 @@
|
|||
set hive.map.aggr=true;
|
||||
set hive.groupby.skewindata=false;
|
||||
set mapred.reduce.tasks=31;
|
||||
set mapreduce.job.reduces=31;
|
||||
|
||||
CREATE TABLE dest1(key INT, value DOUBLE) STORED AS TEXTFILE;
|
||||
|
||||
|
|
|
@ -1,6 +1,6 @@
|
|||
set hive.map.aggr=true;
|
||||
set hive.groupby.skewindata=true;
|
||||
set mapred.reduce.tasks=31;
|
||||
set mapreduce.job.reduces=31;
|
||||
|
||||
CREATE TABLE dest1(key INT, value DOUBLE) STORED AS TEXTFILE;
|
||||
|
||||
|
|
|
@ -1,6 +1,6 @@
|
|||
set hive.map.aggr=false;
|
||||
set hive.groupby.skewindata=false;
|
||||
set mapred.reduce.tasks=31;
|
||||
set mapreduce.job.reduces=31;
|
||||
|
||||
CREATE TABLE dest_g1(key INT, value DOUBLE) STORED AS TEXTFILE;
|
||||
|
||||
|
|
|
@ -1,4 +1,4 @@
|
|||
set mapred.reduce.tasks=31;
|
||||
set mapreduce.job.reduces=31;
|
||||
|
||||
EXPLAIN
|
||||
SELECT src.key, sum(substr(src.value,5)) FROM src GROUP BY src.key ORDER BY src.key LIMIT 5;
|
||||
|
|
|
@ -1,6 +1,6 @@
|
|||
set hive.map.aggr=true;
|
||||
set hive.groupby.skewindata=false;
|
||||
set mapred.reduce.tasks=31;
|
||||
set mapreduce.job.reduces=31;
|
||||
|
||||
CREATE TABLE dest1(key STRING, c1 INT, c2 STRING) STORED AS TEXTFILE;
|
||||
|
||||
|
|
|
@ -1,6 +1,6 @@
|
|||
set hive.map.aggr=true;
|
||||
set hive.groupby.skewindata=false;
|
||||
set mapred.reduce.tasks=31;
|
||||
set mapreduce.job.reduces=31;
|
||||
|
||||
CREATE TABLE dest1(key STRING, c1 INT, c2 STRING, c3 INT, c4 INT) STORED AS TEXTFILE;
|
||||
|
||||
|
|
|
@ -1,6 +1,6 @@
|
|||
set hive.map.aggr=true;
|
||||
set hive.groupby.skewindata=true;
|
||||
set mapred.reduce.tasks=31;
|
||||
set mapreduce.job.reduces=31;
|
||||
|
||||
CREATE TABLE dest1(key STRING, c1 INT, c2 STRING) STORED AS TEXTFILE;
|
||||
|
||||
|
|
|
@ -1,6 +1,6 @@
|
|||
set hive.map.aggr=false;
|
||||
set hive.groupby.skewindata=false;
|
||||
set mapred.reduce.tasks=31;
|
||||
set mapreduce.job.reduces=31;
|
||||
|
||||
CREATE TABLE dest_g2(key STRING, c1 INT, c2 STRING) STORED AS TEXTFILE;
|
||||
|
||||
|
|
|
@ -1,6 +1,6 @@
|
|||
set hive.map.aggr=false;
|
||||
set hive.groupby.skewindata=false;
|
||||
set mapred.reduce.tasks=31;
|
||||
set mapreduce.job.reduces=31;
|
||||
|
||||
CREATE TABLE dest_g2(key STRING, c1 INT, c2 STRING, c3 INT, c4 INT) STORED AS TEXTFILE;
|
||||
|
||||
|
|
|
@ -1,6 +1,6 @@
|
|||
set hive.map.aggr=true;
|
||||
set hive.groupby.skewindata=false;
|
||||
set mapred.reduce.tasks=31;
|
||||
set mapreduce.job.reduces=31;
|
||||
|
||||
CREATE TABLE dest1(c1 DOUBLE, c2 DOUBLE, c3 DOUBLE, c4 DOUBLE, c5 DOUBLE, c6 DOUBLE, c7 DOUBLE, c8 DOUBLE, c9 DOUBLE) STORED AS TEXTFILE;
|
||||
|
||||
|
|
|
@ -1,6 +1,6 @@
|
|||
set hive.map.aggr=true;
|
||||
set hive.groupby.skewindata=false;
|
||||
set mapred.reduce.tasks=31;
|
||||
set mapreduce.job.reduces=31;
|
||||
|
||||
CREATE TABLE dest1(c1 DOUBLE, c2 DOUBLE, c3 DOUBLE, c4 DOUBLE, c5 DOUBLE, c6 DOUBLE, c7 DOUBLE, c8 DOUBLE, c9 DOUBLE, c10 DOUBLE, c11 DOUBLE) STORED AS TEXTFILE;
|
||||
|
||||
|
|
|
@ -1,6 +1,6 @@
|
|||
set hive.map.aggr=true;
|
||||
set hive.groupby.skewindata=true;
|
||||
set mapred.reduce.tasks=31;
|
||||
set mapreduce.job.reduces=31;
|
||||
|
||||
CREATE TABLE dest1(c1 DOUBLE, c2 DOUBLE, c3 DOUBLE, c4 DOUBLE, c5 DOUBLE, c6 DOUBLE, c7 DOUBLE, c8 DOUBLE, c9 DOUBLE) STORED AS TEXTFILE;
|
||||
|
||||
|
|
|
@ -1,7 +1,7 @@
|
|||
set hive.map.aggr=false;
|
||||
|
||||
set hive.groupby.skewindata=false;
|
||||
set mapred.reduce.tasks=31;
|
||||
set mapreduce.job.reduces=31;
|
||||
|
||||
CREATE TABLE dest1(c1 DOUBLE, c2 DOUBLE, c3 DOUBLE, c4 DOUBLE, c5 DOUBLE, c6 DOUBLE, c7 DOUBLE, c8 DOUBLE, c9 DOUBLE) STORED AS TEXTFILE;
|
||||
|
||||
|
|
|
@ -1,7 +1,7 @@
|
|||
set hive.map.aggr=false;
|
||||
|
||||
set hive.groupby.skewindata=false;
|
||||
set mapred.reduce.tasks=31;
|
||||
set mapreduce.job.reduces=31;
|
||||
|
||||
CREATE TABLE dest1(c1 DOUBLE, c2 DOUBLE, c3 DOUBLE, c4 DOUBLE, c5 DOUBLE, c6 DOUBLE, c7 DOUBLE, c8 DOUBLE, c9 DOUBLE, c10 DOUBLE, c11 DOUBLE) STORED AS TEXTFILE;
|
||||
|
||||
|
|
|
@ -1,6 +1,6 @@
|
|||
set hive.map.aggr=true;
|
||||
set hive.groupby.skewindata=false;
|
||||
set mapred.reduce.tasks=31;
|
||||
set mapreduce.job.reduces=31;
|
||||
|
||||
CREATE TABLE dest1(key INT) STORED AS TEXTFILE;
|
||||
|
||||
|
|
|
@ -1,6 +1,6 @@
|
|||
set hive.map.aggr=true;
|
||||
set hive.groupby.skewindata=true;
|
||||
set mapred.reduce.tasks=31;
|
||||
set mapreduce.job.reduces=31;
|
||||
|
||||
CREATE TABLE dest1(key INT) STORED AS TEXTFILE;
|
||||
|
||||
|
|
|
@ -1,7 +1,7 @@
|
|||
set hive.map.aggr=false;
|
||||
|
||||
set hive.groupby.skewindata=false;
|
||||
set mapred.reduce.tasks=31;
|
||||
set mapreduce.job.reduces=31;
|
||||
|
||||
CREATE TABLE dest1(c1 STRING) STORED AS TEXTFILE;
|
||||
|
||||
|
|
|
@ -1,6 +1,6 @@
|
|||
set hive.map.aggr=true;
|
||||
set hive.groupby.skewindata=false;
|
||||
set mapred.reduce.tasks=31;
|
||||
set mapreduce.job.reduces=31;
|
||||
|
||||
CREATE TABLE dest1(key INT) STORED AS TEXTFILE;
|
||||
|
||||
|
|
|
@ -1,6 +1,6 @@
|
|||
set hive.map.aggr=true;
|
||||
set hive.groupby.skewindata=true;
|
||||
set mapred.reduce.tasks=31;
|
||||
set mapreduce.job.reduces=31;
|
||||
|
||||
CREATE TABLE dest1(key INT) STORED AS TEXTFILE;
|
||||
|
||||
|
|
|
@ -1,7 +1,7 @@
|
|||
set hive.map.aggr=false;
|
||||
|
||||
set hive.groupby.skewindata=false;
|
||||
set mapred.reduce.tasks=31;
|
||||
set mapreduce.job.reduces=31;
|
||||
|
||||
CREATE TABLE dest1(key INT, value STRING) STORED AS TEXTFILE;
|
||||
|
||||
|
|
|
@ -1,6 +1,6 @@
|
|||
set hive.map.aggr=true;
|
||||
set hive.groupby.skewindata=false;
|
||||
set mapred.reduce.tasks=31;
|
||||
set mapreduce.job.reduces=31;
|
||||
|
||||
CREATE TABLE dest1(c1 STRING) STORED AS TEXTFILE;
|
||||
|
||||
|
|
Some files were not shown because too many files have changed in this diff Show more
Loading…
Reference in a new issue