Skip to content
New issue

Have a question about this project? Sign up for a free GitHub account to open an issue and contact its maintainers and the community.

By clicking “Sign up for GitHub”, you agree to our terms of service and privacy statement. We’ll occasionally send you account related emails.

Already on GitHub? Sign in to your account

Spark dfsioe and repartition tuning #724

Open
wants to merge 16 commits into
base: master
Choose a base branch
from
Open
Show file tree
Hide file tree
Changes from all commits
Commits
File filter

Filter by extension

Filter by extension

Conversations
Failed to load comments.
Loading
Jump to
Jump to file
Failed to load files.
Loading
Diff view
Diff view
Original file line number Diff line number Diff line change
@@ -0,0 +1,111 @@
/*
* 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 com.intel.sparkbench.micro

import org.apache.hadoop.fs.{FileSystem, Path}
import org.apache.spark.SparkConf
import org.apache.spark.SparkContext
import org.apache.spark.internal.Logging
import org.apache.spark.rdd.MetricsUtils

import scala.collection.mutable.ArrayBuffer

object ScalaDFSIOReadOnly extends Logging {

def main(args: Array[String]) {
if (args.length != 7) {
System.err.println(
s"Usage: $ScalaDFSIOE <INPUT_HDFS> <OUTPUT_HDFS> <RD_NUM_OF_FILES> <RD_FILE_SIZE> <LINE_SIZE> <ROUND> <URI>"
)
System.exit(1)
}
logInfo("===========arguments[<INPUT_HDFS> <OUTPUT_HDFS> <RD_NUM_OF_FILES> <RD_FILE_SIZE> <LINE_SIZE> " +
"<ROUND> <URI>] ============")
args.foreach(logInfo(_))
val lineSize = toLong(args(4), "LINE_SIZE")
val round = toLong(args(5), "ROUND")
val nbrOfFiles = toLong(args(2), "RD_NUM_OF_FILES")
val fileSize = toLong(args(3), "RD_FILE_SIZE")
val totalSizeM = nbrOfFiles * fileSize
val uri = args(6)

val sparkConf = new SparkConf().setAppName("ScalaDFSIOReadOnly")
val sc = new SparkContext(sparkConf)
val fs = FileSystem.get(ScalaDFSIOWriteOnly.config(uri))
val folder = args(0)
val files = fs.listFiles(new Path(folder), false)
val paths = ArrayBuffer[String]()
while (files.hasNext()) {
paths += files.next().getPath().toString
}
try {
val fileSizeByte = fileSize * 1024 * 1024
for (elem <- 1 to round.toInt) {
val data = sc.parallelize(paths, paths.size)
val readStart = System.currentTimeMillis()
data.foreach(path => {
val fos = FileSystem.get(ScalaDFSIOWriteOnly.config(uri)).open(new Path(path))
try {
var count = 0L
var size = 1
val bytes = new Array[Byte](lineSize.toInt)
while (size > 0 & count < fileSizeByte) {
size = fos.read(bytes)
count += size
}
} finally {
fos.close()
}
MetricsUtils.setTaskRead(fileSizeByte, fileSizeByte / lineSize)
})
val readEnd = System.currentTimeMillis()
val dur = readEnd - readStart
val durSec = dur/1000

logInfo(s"#$elem===read [$totalSizeM(MB), $dur(ms)] perf: ${totalSizeM.toFloat/durSec}(MB/s) ")
}
} finally {
if (fs != null ) {
fs.close()
}
sc.stop()
}
}

// More hints on Exceptions
private def toBoolean(str: String, parameterName: String): Boolean = {
try {
str.toBoolean
} catch {
case e: IllegalArgumentException =>
throw new IllegalArgumentException(
s"Unrecognizable parameter ${parameterName}: ${str}, should be true or false")
}
}

private def toLong(str: String, parameterName: String): Long = {
try {
str.toLong
} catch {
case e: IllegalArgumentException =>
throw new IllegalArgumentException(
s"Unrecognizable parameter ${parameterName}: ${str}, should be integer")
}
}

}
Original file line number Diff line number Diff line change
@@ -0,0 +1,117 @@
/*
* 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 com.intel.sparkbench.micro

import org.apache.hadoop.conf.Configuration
import org.apache.hadoop.fs.{FileSystem, Path}
import org.apache.spark.{SparkConf, SparkContext}
import org.apache.spark.internal.Logging
import org.apache.spark.rdd.MetricsUtils

object ScalaDFSIOWriteOnly extends Logging {

def config(uriStr: String): Configuration = {
val conf = new Configuration(false)
conf.set("fs.defaultFS", uriStr)
conf
}

def main(args: Array[String]) {
if (args.length != 7) {
System.err.println(
s"Usage: $ScalaDFSIOWriteOnly <INPUT_HDFS> <OUTPUT_HDFS> <RD_NUM_OF_FILES> <RD_FILE_SIZE> <LINE_SIZE> <ROUND>" +
s" <URI>"
)
System.exit(1)
}
logInfo("===========arguments[<INPUT_HDFS> <OUTPUT_HDFS> <RD_NUM_OF_FILES> <RD_FILE_SIZE> <LINE_SIZE> " +
"<ROUND> <URI> ]============")
args.foreach(logInfo(_))
val nbrOfFiles = toLong(args(2), "RD_NUM_OF_FILES")
val fileSize = toLong(args(3), "RD_FILE_SIZE")
val round = toLong(args(5), "ROUND")
val totalSizeM = nbrOfFiles * fileSize
val lineSize = toLong(args(4), "LINE_SIZE")
val uri = args(6)
val fileSizeByte = fileSize * 1024 * 1024
if ((fileSizeByte % lineSize) != 0) {
throw new IllegalArgumentException("line size, " + lineSize + ", should divide file size, " + fileSizeByte +
", exactly")
}

val sparkConf = new SparkConf().setAppName("ScalaDFSIOWriteOnly")
val sc = new SparkContext(sparkConf)
val fs = FileSystem.get(config(uri))
fs.mkdirs(new Path(args(1)))
try {
val bytesGlobal = new Array[Byte](lineSize.toInt)
Range(0, lineSize.toInt).foreach(i => bytesGlobal(i) = (i % 255).toByte)
val lineBd = sc.broadcast(bytesGlobal)
for (elem <- 1 to round.toInt) {
fs.mkdirs(new Path(args(1) + "/" + elem))
}
for (elem <- 1 to round.toInt) {
val data = sc.parallelize(Range(0, nbrOfFiles.toInt), nbrOfFiles.toInt)
val readStart = System.currentTimeMillis()
data.foreach(i => {
val bytes = lineBd.value
val fos = FileSystem.get(config(uri)).create(new Path(args(1) + "/" + elem + "/data_" + i), true)
try {
var count = 0L
while (count < fileSizeByte) {
fos.write(bytes)
count += bytes.length
}
} finally {
fos.close()
}
MetricsUtils.setTaskWrite(fileSizeByte, fileSizeByte / lineSize)
})
val readEnd = System.currentTimeMillis()
val dur = readEnd - readStart
val durSec = dur / 1000

logInfo(s"$elem===write [$totalSizeM(MB), $dur(ms)] perf: ${totalSizeM.toFloat / durSec}(MB/s) ")
}
} finally {
sc.stop()
}
}

// More hints on Exceptions
private def toBoolean(str: String, parameterName: String): Boolean = {
try {
str.toBoolean
} catch {
case e: IllegalArgumentException =>
throw new IllegalArgumentException(
s"Unrecognizable parameter ${parameterName}: ${str}, should be true or false")
}
}

private def toLong(str: String, parameterName: String): Long = {
try {
str.toLong
} catch {
case e: IllegalArgumentException =>
throw new IllegalArgumentException(
s"Unrecognizable parameter ${parameterName}: ${str}, should be integer")
}
}

}
Original file line number Diff line number Diff line change
Expand Up @@ -36,17 +36,19 @@ object ScalaInMemRepartition {
)

def main(args: Array[String]) {
if (args.length != 4) {
if (args.length != 6) {
System.err.println(
s"Usage: $ScalaInMemRepartition <NBR_OF_RECORD> <OUTPUT_HDFS> <CACHE_IN_MEMORY> <DISABLE_OUTPUT>"
s"Usage: $ScalaInMemRepartition <NBR_OF_RECORD> <OUTPUT_HDFS> <CACHE_IN_MEMORY> <DISABLE_OUTPUT> <RECORD_SIZE>" +
s" <ROUND>"
)
System.exit(1)
}
val cache = toBoolean(args(2), ("CACHE_IN_MEMORY"))
val nbrOfRecords = toInt(args(0), ("NBR_OF_RECORD"))
val outputDir = args(1)
val disableOutput = toBoolean(args(3), ("DISABLE_OUTPUT"))
val localData = Range(0, 200).map(i => i.toByte).toArray
val recordSize = toInt(args(4), ("RECORD_SIZE"))
val round = toInt(args(5), ("ROUND"))

val sparkConf = new SparkConf().setAppName("ScalaInMemRepartition")
val sc = new SparkContext(sparkConf)
Expand All @@ -55,14 +57,12 @@ object ScalaInMemRepartition {
val reduceParallelism = IOCommon.getProperty("hibench.default.shuffle.parallelism")
.getOrElse((mapParallelism / 2).toString).toInt

val sleepDur = if (cache) 3000 else 0
val data = new MemoryDataRDD(sc, mapParallelism, nbrOfRecords, 200, sleepDur)
if (cache) {
data.persist(StorageLevel.MEMORY_ONLY)
data.foreach(_ => {})
}
// val sleepDur = if (cache) 3000 else 0
val data = new MemoryDataRDD(sc, mapParallelism, nbrOfRecords, recordSize)
// data.persist(StorageLevel.MEMORY_ONLY)
//data.foreach(_ => {})

val paired: PairRDDFunctions[Int, Array[Byte]] = data.mapPartitionsWithIndex {
val paired = data.mapPartitionsWithIndex {
val nbrOfReduces = reduceParallelism
(index, part) => {
var position = new Random(hashing.byteswap32(index)).nextInt(nbrOfReduces)
Expand All @@ -74,12 +74,19 @@ object ScalaInMemRepartition {
}
}

val shuffled = paired.partitionBy(new HashPartitioner(reduceParallelism))
if (disableOutput) {
shuffled.foreach(_ => {})
} else {
shuffled.map { case (_, v) => (NullWritable.get(), new Text(v)) }
.saveAsNewAPIHadoopFile[TextOutputFormat[NullWritable, Text]](outputDir)
if (cache) {
paired.persist(StorageLevel.MEMORY_ONLY)
paired.foreach(_ => {})
}

for (_ <- 1 to round) {
val shuffled = paired.partitionBy(new HashPartitioner(reduceParallelism))
if (disableOutput) {
shuffled.foreach(_ => {})
} else {
shuffled.map { case (_, v) => (NullWritable.get(), new Text(v)) }
.saveAsNewAPIHadoopFile[TextOutputFormat[NullWritable, Text]](outputDir)
}
}

sc.stop()
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -33,26 +33,27 @@ object ScalaRepartition {
def main(args: Array[String]) {
if (args.length != 4) {
System.err.println(
s"Usage: $ScalaRepartition <INPUT_HDFS> <OUTPUT_HDFS> <CACHE_IN_MEMORY> <DISABLE_OUTPUT>"
s"Usage: $ScalaRepartition <MAP_SIZE> <OUTPUT_HDFS> <CACHE_IN_MEMORY> <DISABLE_OUTPUT>"
)
System.exit(1)
}
val cache = toBoolean(args(2), ("CACHE_IN_MEMORY"))
if (!cache) {
throw new IllegalArgumentException("CACHE_IN_MEMORY should be set to true")
}

val mapSize = toLong(args(0), ("MAP_SIZE"))
MockedData.dataSize = mapSize

val disableOutput = toBoolean(args(3), ("DISABLE_OUTPUT"))

val sparkConf = new SparkConf().setAppName("ScalaRepartition")
val sc = new SparkContext(sparkConf)

val data = sc.newAPIHadoopFile[Text, Text, TeraInputFormat](args(0)).map {
case (k,v) => k.copyBytes ++ v.copyBytes
}
val mapParallelism = sc.getConf.getInt("spark.default.parallelism", sc.defaultParallelism)

if (cache) {
data.persist(StorageLevel.MEMORY_ONLY)
data.count()
}
val data = sc.parallelize(Range(0, mapParallelism - 1))

val mapParallelism = sc.getConf.getInt("spark.default.parallelism", sc.defaultParallelism)
val reduceParallelism = IOCommon.getProperty("hibench.default.shuffle.parallelism")
.getOrElse((mapParallelism / 2).toString).toInt

Expand All @@ -79,16 +80,27 @@ object ScalaRepartition {
}
}

private def toLong(str: String, parameterName: String): Long = {
try {
str.toLong
} catch {
case e: IllegalArgumentException =>
throw new IllegalArgumentException(
s"Unrecognizable parameter ${parameterName}: ${str}, should be integer")
}
}

// Save a CoalescedRDD than RDD.repartition API
private def repartition(previous: RDD[Array[Byte]], numReducers: Int): ShuffledRDD[Int, Array[Byte], Array[Byte]] = {
private def repartition(previous: RDD[Int], numReducers: Int):
ShuffledRDD[Int, Array[Byte], Array[Byte]] = {
/** Distributes elements evenly across output partitions, starting from a random partition. */
val distributePartition = (index: Int, items: Iterator[Array[Byte]]) => {
val distributePartition = (index: Int, items: Iterator[Int]) => {
var position = new Random(hashing.byteswap32(index)).nextInt(numReducers)
items.map { t =>
items.map { i =>
// Note that the hash code of the key will just be the key itself. The HashPartitioner
// will mod it with the number of total partitions.
position = position + 1
(position, t)
(position, MockedData.data)
}
} : Iterator[(Int, Array[Byte])]

Expand All @@ -97,4 +109,9 @@ object ScalaRepartition {
new HashPartitioner(numReducers))
}

object MockedData {
var dataSize: Long = _
lazy val data: Array[Byte] = Seq(0, dataSize).map(i => i.toByte).toArray
}

}
Loading