├── src └── main │ ├── java │ └── eastcircle │ │ └── terasort │ │ ├── OptimizedText.java │ │ ├── TeraOutputFormat.java │ │ ├── TeraScheduler.java │ │ ├── TotalOrderPartitioner.java │ │ └── TeraInputFormat.java │ └── scala │ └── eastcircle │ └── terasort │ ├── SparkTeraSort.scala │ └── FlinkTeraSort.scala └── README.md /src/main/java/eastcircle/terasort/OptimizedText.java: -------------------------------------------------------------------------------- 1 | package eastcircle.terasort; 2 | 3 | import java.io.IOException; 4 | 5 | import org.apache.flink.core.memory.DataInputView; 6 | import org.apache.flink.core.memory.DataOutputView; 7 | import org.apache.flink.core.memory.MemorySegment; 8 | import org.apache.flink.types.NormalizableKey; 9 | import org.apache.hadoop.io.Text; 10 | 11 | public final class OptimizedText implements NormalizableKey { 12 | 13 | private final Text text; 14 | 15 | public OptimizedText () { 16 | this.text = new Text(); 17 | } 18 | 19 | public OptimizedText (Text from) { 20 | this.text = from; 21 | } 22 | 23 | public Text getText() { 24 | return text; 25 | } 26 | 27 | @Override 28 | public int getMaxNormalizedKeyLen() { 29 | return 10; 30 | } 31 | 32 | @Override 33 | public void copyNormalizedKey(MemorySegment memory, int offset, int len) { 34 | memory.put(offset, text.getBytes(), 0, Math.min(text.getLength(), Math.min(10, len))); 35 | } 36 | 37 | @Override 38 | public void write(DataOutputView out) throws IOException { 39 | text.write(out); 40 | } 41 | 42 | @Override 43 | public void read(DataInputView in) throws IOException { 44 | text.readFields(in); 45 | } 46 | 47 | @Override 48 | public int compareTo(OptimizedText o) { 49 | return this.text.compareTo(o.text); 50 | } 51 | } 52 | -------------------------------------------------------------------------------- /README.md: -------------------------------------------------------------------------------- 1 | # terasort 2 | 3 | TeraSort for Apache Spark and Flink. 4 | 5 | # Build 6 | 7 | `sbt package` 8 | 9 | # Run TeraSort using Spark on Yarn 10 | 11 | The below shows an example of a bash script to execute TeraSort on Spark on top of Yarn. 12 | 13 | ``` 14 | #!/usr/bin/env bash 15 | 16 | EXECUTORS=42 17 | EXECUTOR_CORES=12 18 | EXECUTOR_MEMORY=10G 19 | SCALA_VERSION=2.10 20 | HDFS=hdfs://master:54321 21 | INDIR=/nodedata_80gb/blk_256mb 22 | OUTDIR=/spark_out 23 | PARTITIONS=$[42*12] 24 | 25 | spark-submit --master yarn\ 26 | --num-executors ${EXECUTORS}\ 27 | --executor-cores ${EXECUTOR_CORES}\ 28 | --executor-memory ${EXECUTOR_MEMORY}\ 29 | --class eastcircle.terasort.SparkTeraSort\ 30 | target/${SCALA_VERSION}/terasort_${SCALA_VERSION}-0.0.1.jar\ 31 | ${HDFS} ${INDIR} ${OUTDIR} ${PARTITIONS} 32 | ``` 33 | 34 | 35 | # Run TeraSort using Flink on Yarn 36 | 37 | ## execute job manager and task managers 38 | 39 | The below shows an example of a command to execute job manager and 42 task managers. 40 | 41 | ``` 42 | yarn-session.sh -n 42 -tm 10240 -s 12 -tmc 12 43 | ``` 44 | 45 | yarn-session.sh will show you the address of job manager via stdout as follows: 46 | ``` 47 | ... 48 | Flink JobManager is now running on slave1:33970 49 | ... 50 | ``` 51 | 52 | ## Submit a TeraSort job to job manager 53 | 54 | The below shows an example of a bash script to execute TeraSort on Flink. 55 | 56 | ``` 57 | #!/usr/bin/env bash 58 | 59 | JOBMANAGER=slave1:33970 60 | PARTITIONS=$[42*12] 61 | SCALA_VERSION=2.10 62 | HDFS=hdfs://master:54321 63 | INDIR=/nodedata_80gb/blk_256mb 64 | OUTDIR=/flink_out 65 | 66 | flink run\ 67 | -m ${JOBMANAGER}\ 68 | -p ${PARTITIONS}\ 69 | -c eastcircle.terasort.FlinkTeraSort\ 70 | target/${SCALA_VERSION}/terasort_${SCALA_VERSION}-0.0.1.jar\ 71 | ${HDFS} ${INDIR} ${OUTDIR} ${PARTITIONS} 72 | 73 | ``` 74 | -------------------------------------------------------------------------------- /src/main/scala/eastcircle/terasort/SparkTeraSort.scala: -------------------------------------------------------------------------------- 1 | package eastcircle.terasort 2 | 3 | import org.apache.spark.SparkContext._ 4 | 5 | import org.apache.hadoop.mapred.{FileInputFormat, JobConf} 6 | import org.apache.hadoop.mapreduce.Job 7 | import org.apache.hadoop.fs.Path 8 | import org.apache.hadoop.io.{NullWritable, SequenceFile, Text} 9 | import org.apache.spark.{SparkConf, SparkContext, Partitioner} 10 | import org.apache.spark.RangePartitioner 11 | import org.apache.spark.HashPartitioner 12 | import org.apache.spark.rdd.RDD 13 | 14 | class SparkTeraRangePartitioner(underlying:TotalOrderPartitioner, 15 | partitions:Int) extends Partitioner { 16 | def numPartitions: Int = partitions 17 | def getPartition(key: Any): Int = { 18 | val textKey = key.asInstanceOf[Text] 19 | underlying.getPartition(textKey) 20 | } 21 | } 22 | 23 | object SparkTeraSort { 24 | 25 | implicit val textOrdering = new Ordering[Text] { 26 | override def compare(a: Text, b: Text) = a.compareTo(b) 27 | } 28 | 29 | def main(args: Array[String]){ 30 | val conf = new SparkConf().setAppName("TeraSort") 31 | val sc = new SparkContext() 32 | 33 | val hdfs = args(0) 34 | val inputPath = hdfs+args(1) 35 | val outputPath = hdfs+args(2) 36 | val partitions = args(3).toInt 37 | 38 | val hadoopConf = new JobConf() 39 | hadoopConf.set("fs.defaultFS", hdfs) 40 | hadoopConf.set("mapreduce.input.fileinputformat.inputdir", inputPath) 41 | hadoopConf.setInt("mapreduce.job.reduces", partitions) 42 | 43 | val partitionFile = new Path(outputPath, 44 | TeraInputFormat.PARTITION_FILENAME) 45 | val jobContext = Job.getInstance(hadoopConf) 46 | TeraInputFormat.writePartitionFile(jobContext, partitionFile) 47 | 48 | val inputFile = sc.newAPIHadoopFile[Text, Text, TeraInputFormat](inputPath) 49 | val partitioner = 50 | new SparkTeraRangePartitioner( 51 | new TotalOrderPartitioner(hadoopConf, partitionFile), 52 | partitions 53 | ) 54 | val repartitioned = inputFile.repartitionAndSortWithinPartitions( partitioner ) 55 | repartitioned.saveAsNewAPIHadoopFile[TeraOutputFormat](outputPath) 56 | } 57 | } 58 | -------------------------------------------------------------------------------- /src/main/scala/eastcircle/terasort/FlinkTeraSort.scala: -------------------------------------------------------------------------------- 1 | package eastcircle.terasort 2 | 3 | import org.apache.flink.api.scala._ 4 | 5 | import org.apache.flink.api.common.functions.Partitioner 6 | import org.apache.flink.api.common.operators.Order 7 | 8 | import org.apache.flink.api.scala.hadoop.mapreduce.HadoopOutputFormat 9 | 10 | import org.apache.hadoop.fs.Path 11 | import org.apache.hadoop.io.Text 12 | import org.apache.hadoop.mapred.JobConf 13 | import org.apache.hadoop.mapreduce.Job 14 | 15 | class OptimizedFlinkTeraPartitioner(underlying:TotalOrderPartitioner) extends Partitioner[OptimizedText] { 16 | def partition(key:OptimizedText, numPartitions:Int):Int = { 17 | underlying.getPartition(key.getText()) 18 | } 19 | } 20 | 21 | 22 | object FlinkTeraSort { 23 | 24 | implicit val textOrdering = new Ordering[Text] { 25 | override def compare(a:Text, b:Text) = a.compareTo(b) 26 | } 27 | 28 | def main(args: Array[String]){ 29 | if(args.size != 4){ 30 | println("Usage: FlinkTeraSort hdfs inputPath outputPath #partitions ") 31 | return 32 | } 33 | 34 | val env = ExecutionEnvironment.getExecutionEnvironment 35 | env.getConfig.enableObjectReuse() 36 | 37 | val hdfs = args(0) 38 | val inputPath= hdfs+args(1) 39 | val outputPath = hdfs+args(2) 40 | val partitions = args(3).toInt 41 | 42 | val mapredConf = new JobConf() 43 | mapredConf.set("fs.defaultFS", hdfs) 44 | mapredConf.set("mapreduce.input.fileinputformat.inputdir", inputPath) 45 | mapredConf.set("mapreduce.output.fileoutputformat.outputdir", outputPath) 46 | mapredConf.setInt("mapreduce.job.reduces", partitions) 47 | 48 | val partitionFile = new Path(outputPath, TeraInputFormat.PARTITION_FILENAME) 49 | val jobContext = Job.getInstance(mapredConf) 50 | TeraInputFormat.writePartitionFile(jobContext, partitionFile) 51 | val partitioner = new OptimizedFlinkTeraPartitioner(new TotalOrderPartitioner(mapredConf, partitionFile)) 52 | 53 | env 54 | .readHadoopFile(new TeraInputFormat(), classOf[Text], classOf[Text], inputPath) 55 | .map(tp => (new OptimizedText(tp._1), tp._2)) 56 | .partitionCustom(partitioner, 0).sortPartition(0, Order.ASCENDING) 57 | .map(tp => (tp._1.getText, tp._2)) 58 | .output(new HadoopOutputFormat[Text, Text](new TeraOutputFormat(), jobContext)) 59 | env.execute("TeraSort") 60 | } 61 | } 62 | -------------------------------------------------------------------------------- /src/main/java/eastcircle/terasort/TeraOutputFormat.java: -------------------------------------------------------------------------------- 1 | /** 2 | * Licensed to the Apache Software Foundation (ASF) under one 3 | * or more contributor license agreements. See the NOTICE file 4 | * distributed with this work for additional information 5 | * regarding copyright ownership. The ASF licenses this file 6 | * to you under the Apache License, Version 2.0 (the 7 | * "License"); you may not use this file except in compliance 8 | * with the License. You may obtain a copy of the License at 9 | * 10 | * http://www.apache.org/licenses/LICENSE-2.0 11 | * 12 | * Unless required by applicable law or agreed to in writing, software 13 | * distributed under the License is distributed on an "AS IS" BASIS, 14 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 15 | * See the License for the specific language governing permissions and 16 | * limitations under the License. 17 | */ 18 | 19 | package eastcircle.terasort; 20 | 21 | import java.io.IOException; 22 | 23 | import org.apache.hadoop.fs.FSDataOutputStream; 24 | import org.apache.hadoop.fs.FileSystem; 25 | import org.apache.hadoop.fs.Path; 26 | import org.apache.hadoop.io.Text; 27 | import org.apache.hadoop.mapred.InvalidJobConfException; 28 | import org.apache.hadoop.mapreduce.JobContext; 29 | import org.apache.hadoop.mapreduce.OutputCommitter; 30 | import org.apache.hadoop.mapreduce.RecordWriter; 31 | import org.apache.hadoop.mapreduce.TaskAttemptContext; 32 | import org.apache.hadoop.mapreduce.lib.output.FileOutputCommitter; 33 | import org.apache.hadoop.mapreduce.lib.output.FileOutputFormat; 34 | import org.apache.hadoop.mapreduce.security.TokenCache; 35 | 36 | /** 37 | * An output format that writes the key and value appended together. 38 | */ 39 | public class TeraOutputFormat extends FileOutputFormat { 40 | static final String FINAL_SYNC_ATTRIBUTE = "mapreduce.terasort.final.sync"; 41 | private OutputCommitter committer = null; 42 | 43 | /** 44 | * Set the requirement for a final sync before the stream is closed. 45 | */ 46 | static void setFinalSync(JobContext job, boolean newValue) { 47 | job.getConfiguration().setBoolean(FINAL_SYNC_ATTRIBUTE, newValue); 48 | } 49 | 50 | /** 51 | * Does the user want a final sync at close? 52 | */ 53 | public static boolean getFinalSync(JobContext job) { 54 | return job.getConfiguration().getBoolean(FINAL_SYNC_ATTRIBUTE, false); 55 | } 56 | 57 | static class TeraRecordWriter extends RecordWriter { 58 | private boolean finalSync = false; 59 | private FSDataOutputStream out; 60 | 61 | public TeraRecordWriter(FSDataOutputStream out, 62 | JobContext job) { 63 | finalSync = getFinalSync(job); 64 | this.out = out; 65 | } 66 | 67 | public synchronized void write(Text key, 68 | Text value) throws IOException { 69 | out.write(key.getBytes(), 0, key.getLength()); 70 | out.write(value.getBytes(), 0, value.getLength()); 71 | } 72 | 73 | public void close(TaskAttemptContext context) throws IOException { 74 | if (finalSync) { 75 | out.sync(); 76 | } 77 | out.close(); 78 | } 79 | } 80 | 81 | @Override 82 | public void checkOutputSpecs(JobContext job 83 | ) throws InvalidJobConfException, IOException { 84 | // Ensure that the output directory is set 85 | Path outDir = getOutputPath(job); 86 | if (outDir == null) { 87 | throw new InvalidJobConfException("Output directory not set in JobConf."); 88 | } 89 | 90 | // get delegation token for outDir's file system 91 | TokenCache.obtainTokensForNamenodes(job.getCredentials(), 92 | new Path[] { outDir }, job.getConfiguration()); 93 | } 94 | 95 | public RecordWriter getRecordWriter(TaskAttemptContext job 96 | ) throws IOException { 97 | Path file = getDefaultWorkFile(job, ""); 98 | FileSystem fs = file.getFileSystem(job.getConfiguration()); 99 | FSDataOutputStream fileOut = fs.create(file); 100 | return new TeraRecordWriter(fileOut, job); 101 | } 102 | 103 | public OutputCommitter getOutputCommitter(TaskAttemptContext context) 104 | throws IOException { 105 | if (committer == null) { 106 | Path output = getOutputPath(context); 107 | committer = new FileOutputCommitter(output, context); 108 | } 109 | return committer; 110 | } 111 | 112 | } 113 | -------------------------------------------------------------------------------- /src/main/java/eastcircle/terasort/TeraScheduler.java: -------------------------------------------------------------------------------- 1 | /** 2 | * Licensed to the Apache Software Foundation (ASF) under one 3 | * or more contributor license agreements. See the NOTICE file 4 | * distributed with this work for additional information 5 | * regarding copyright ownership. The ASF licenses this file 6 | * to you under the Apache License, Version 2.0 (the 7 | * "License"); you may not use this file except in compliance 8 | * with the License. You may obtain a copy of the License at 9 | * 10 | * http://www.apache.org/licenses/LICENSE-2.0 11 | * 12 | * Unless required by applicable law or agreed to in writing, software 13 | * distributed under the License is distributed on an "AS IS" BASIS, 14 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 15 | * See the License for the specific language governing permissions and 16 | * limitations under the License. 17 | */ 18 | 19 | package eastcircle.terasort; 20 | 21 | import java.io.*; 22 | import java.util.*; 23 | 24 | import org.apache.commons.logging.Log; 25 | import org.apache.commons.logging.LogFactory; 26 | import org.apache.hadoop.conf.Configuration; 27 | import org.apache.hadoop.mapreduce.InputSplit; 28 | import org.apache.hadoop.mapreduce.lib.input.FileSplit; 29 | import org.apache.hadoop.mapreduce.server.tasktracker.TTConfig; 30 | 31 | import com.google.common.base.Charsets; 32 | 33 | class TeraScheduler { 34 | static String USE = "mapreduce.terasort.use.terascheduler"; 35 | private static final Log LOG = LogFactory.getLog(TeraScheduler.class); 36 | private Split[] splits; 37 | private List hosts = new ArrayList(); 38 | private int slotsPerHost; 39 | private int remainingSplits = 0; 40 | private FileSplit[] realSplits = null; 41 | 42 | static class Split { 43 | String filename; 44 | boolean isAssigned = false; 45 | List locations = new ArrayList(); 46 | Split(String filename) { 47 | this.filename = filename; 48 | } 49 | public String toString() { 50 | StringBuffer result = new StringBuffer(); 51 | result.append(filename); 52 | result.append(" on "); 53 | for(Host host: locations) { 54 | result.append(host.hostname); 55 | result.append(", "); 56 | } 57 | return result.toString(); 58 | } 59 | } 60 | static class Host { 61 | String hostname; 62 | List splits = new ArrayList(); 63 | Host(String hostname) { 64 | this.hostname = hostname; 65 | } 66 | public String toString() { 67 | StringBuffer result = new StringBuffer(); 68 | result.append(splits.size()); 69 | result.append(" "); 70 | result.append(hostname); 71 | return result.toString(); 72 | } 73 | } 74 | 75 | List readFile(String filename) throws IOException { 76 | List result = new ArrayList(10000); 77 | BufferedReader in = new BufferedReader( 78 | new InputStreamReader(new FileInputStream(filename), Charsets.UTF_8)); 79 | String line = in.readLine(); 80 | while (line != null) { 81 | result.add(line); 82 | line = in.readLine(); 83 | } 84 | in.close(); 85 | return result; 86 | } 87 | 88 | public TeraScheduler(String splitFilename, 89 | String nodeFilename) throws IOException { 90 | slotsPerHost = 4; 91 | // get the hosts 92 | Map hostIds = new HashMap(); 93 | for(String hostName: readFile(nodeFilename)) { 94 | Host host = new Host(hostName); 95 | hosts.add(host); 96 | hostIds.put(hostName, host); 97 | } 98 | // read the blocks 99 | List splitLines = readFile(splitFilename); 100 | splits = new Split[splitLines.size()]; 101 | remainingSplits = 0; 102 | for(String line: splitLines) { 103 | StringTokenizer itr = new StringTokenizer(line); 104 | Split newSplit = new Split(itr.nextToken()); 105 | splits[remainingSplits++] = newSplit; 106 | while (itr.hasMoreTokens()) { 107 | Host host = hostIds.get(itr.nextToken()); 108 | newSplit.locations.add(host); 109 | host.splits.add(newSplit); 110 | } 111 | } 112 | } 113 | 114 | public TeraScheduler(FileSplit[] realSplits, 115 | Configuration conf) throws IOException { 116 | this.realSplits = realSplits; 117 | this.slotsPerHost = conf.getInt(TTConfig.TT_MAP_SLOTS, 4); 118 | Map hostTable = new HashMap(); 119 | splits = new Split[realSplits.length]; 120 | for(FileSplit realSplit: realSplits) { 121 | Split split = new Split(realSplit.getPath().toString()); 122 | splits[remainingSplits++] = split; 123 | for(String hostname: realSplit.getLocations()) { 124 | Host host = hostTable.get(hostname); 125 | if (host == null) { 126 | host = new Host(hostname); 127 | hostTable.put(hostname, host); 128 | hosts.add(host); 129 | } 130 | host.splits.add(split); 131 | split.locations.add(host); 132 | } 133 | } 134 | } 135 | 136 | Host pickBestHost() { 137 | Host result = null; 138 | int splits = Integer.MAX_VALUE; 139 | for(Host host: hosts) { 140 | if (host.splits.size() < splits) { 141 | result = host; 142 | splits = host.splits.size(); 143 | } 144 | } 145 | if (result != null) { 146 | hosts.remove(result); 147 | LOG.debug("picking " + result); 148 | } 149 | return result; 150 | } 151 | 152 | void pickBestSplits(Host host) { 153 | int tasksToPick = Math.min(slotsPerHost, 154 | (int) Math.ceil((double) remainingSplits / 155 | hosts.size())); 156 | Split[] best = new Split[tasksToPick]; 157 | for(Split cur: host.splits) { 158 | LOG.debug(" examine: " + cur.filename + " " + cur.locations.size()); 159 | int i = 0; 160 | while (i < tasksToPick && best[i] != null && 161 | best[i].locations.size() <= cur.locations.size()) { 162 | i += 1; 163 | } 164 | if (i < tasksToPick) { 165 | for(int j = tasksToPick - 1; j > i; --j) { 166 | best[j] = best[j-1]; 167 | } 168 | best[i] = cur; 169 | } 170 | } 171 | // for the chosen blocks, remove them from the other locations 172 | for(int i=0; i < tasksToPick; ++i) { 173 | if (best[i] != null) { 174 | LOG.debug(" best: " + best[i].filename); 175 | for (Host other: best[i].locations) { 176 | other.splits.remove(best[i]); 177 | } 178 | best[i].locations.clear(); 179 | best[i].locations.add(host); 180 | best[i].isAssigned = true; 181 | remainingSplits -= 1; 182 | } 183 | } 184 | // for the non-chosen blocks, remove this host 185 | for(Split cur: host.splits) { 186 | if (!cur.isAssigned) { 187 | cur.locations.remove(host); 188 | } 189 | } 190 | } 191 | 192 | void solve() throws IOException { 193 | Host host = pickBestHost(); 194 | while (host != null) { 195 | pickBestSplits(host); 196 | host = pickBestHost(); 197 | } 198 | } 199 | 200 | /** 201 | * Solve the schedule and modify the FileSplit array to reflect the new 202 | * schedule. It will move placed splits to front and unplacable splits 203 | * to the end. 204 | * @return a new list of FileSplits that are modified to have the 205 | * best host as the only host. 206 | * @throws IOException 207 | */ 208 | public List getNewFileSplits() throws IOException { 209 | solve(); 210 | FileSplit[] result = new FileSplit[realSplits.length]; 211 | int left = 0; 212 | int right = realSplits.length - 1; 213 | for(int i=0; i < splits.length; ++i) { 214 | if (splits[i].isAssigned) { 215 | // copy the split and fix up the locations 216 | String[] newLocations = {splits[i].locations.get(0).hostname}; 217 | realSplits[i] = new FileSplit(realSplits[i].getPath(), 218 | realSplits[i].getStart(), realSplits[i].getLength(), newLocations); 219 | result[left++] = realSplits[i]; 220 | } else { 221 | result[right--] = realSplits[i]; 222 | } 223 | } 224 | List ret = new ArrayList(); 225 | for (FileSplit fs : result) { 226 | ret.add(fs); 227 | } 228 | return ret; 229 | } 230 | 231 | public static void main(String[] args) throws IOException { 232 | TeraScheduler problem = new TeraScheduler("block-loc.txt", "nodes"); 233 | for(Host host: problem.hosts) { 234 | System.out.println(host); 235 | } 236 | LOG.info("starting solve"); 237 | problem.solve(); 238 | List leftOvers = new ArrayList(); 239 | for(int i=0; i < problem.splits.length; ++i) { 240 | if (problem.splits[i].isAssigned) { 241 | System.out.println("sched: " + problem.splits[i]); 242 | } else { 243 | leftOvers.add(problem.splits[i]); 244 | } 245 | } 246 | for(Split cur: leftOvers) { 247 | System.out.println("left: " + cur); 248 | } 249 | System.out.println("left over: " + leftOvers.size()); 250 | LOG.info("done"); 251 | } 252 | 253 | } 254 | -------------------------------------------------------------------------------- /src/main/java/eastcircle/terasort/TotalOrderPartitioner.java: -------------------------------------------------------------------------------- 1 | /** 2 | * Licensed to the Apache Software Foundation (ASF) under one 3 | * or more contributor license agreements. See the NOTICE file 4 | * distributed with this work for additional information 5 | * regarding copyright ownership. The ASF licenses this file 6 | * to you under the Apache License, Version 2.0 (the 7 | * "License"); you may not use this file except in compliance 8 | * with the License. You may obtain a copy of the License at 9 | * 10 | * http://www.apache.org/licenses/LICENSE-2.0 11 | * 12 | * Unless required by applicable law or agreed to in writing, software 13 | * distributed under the License is distributed on an "AS IS" BASIS, 14 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 15 | * See the License for the specific language governing permissions and 16 | * limitations under the License. 17 | */ 18 | 19 | package eastcircle.terasort; 20 | 21 | import java.io.DataInputStream; 22 | import java.io.IOException; 23 | import java.io.ObjectInputStream; // eastcirclek 24 | import java.io.ObjectOutputStream; // eastcirclek 25 | import java.io.PrintStream; 26 | import java.io.Serializable; 27 | 28 | import org.apache.hadoop.conf.Configuration; 29 | import org.apache.hadoop.fs.FileSystem; 30 | import org.apache.hadoop.fs.Path; 31 | import org.apache.hadoop.io.Text; 32 | 33 | import org.apache.hadoop.mapreduce.MRJobConfig; 34 | 35 | /** 36 | * eastcirclek : 37 | * I extract this class from org.apache.hadoop.examples.terasort.TeraSort 38 | * and make little modification to use it for TeraSort on Spark and Flink. 39 | * - We no longer implement org.apache.hadoop.mapreduce.Partitioner 40 | * as we are going to use this class for custom partitioners in 41 | * Spark (org.apache.spark.Partitioner) and 42 | * Flink (org.apache.flink.api.common.functions.Partitioner). 43 | * - We implement java.io.Serializable because Spark and Flink pass 44 | * partitioner objects to tasks by serializing/deserializing the objects. 45 | * - We remove setConf() from the origial TotalOrderPartitioner 46 | * as it assumes that TeraInputFormat.PARTITION_FILENAME is 47 | * localized to PWD on which a MapReduce task is running. 48 | **/ 49 | 50 | /** 51 | * A partitioner that splits text keys into roughly equal partitions 52 | * in a global sorted order. 53 | */ 54 | public class TotalOrderPartitioner implements Serializable{ 55 | 56 | private Configuration conf; 57 | private TrieNode trie; 58 | 59 | private Path partFile; // eastcirclek 60 | 61 | /** 62 | * A generic trie node 63 | */ 64 | static abstract class TrieNode { 65 | private int level; 66 | TrieNode(int level) { 67 | this.level = level; 68 | } 69 | abstract int findPartition(Text key); 70 | abstract void print(PrintStream strm) throws IOException; 71 | int getLevel() { 72 | return level; 73 | } 74 | } 75 | 76 | /** 77 | * An inner trie node that contains 256 children based on the next 78 | * character. 79 | */ 80 | static class InnerTrieNode extends TrieNode { 81 | private TrieNode[] child = new TrieNode[256]; 82 | 83 | InnerTrieNode(int level) { 84 | super(level); 85 | } 86 | int findPartition(Text key) { 87 | int level = getLevel(); 88 | if (key.getLength() <= level) { 89 | return child[0].findPartition(key); 90 | } 91 | return child[key.getBytes()[level] & 0xff].findPartition(key); 92 | } 93 | void setChild(int idx, TrieNode child) { 94 | this.child[idx] = child; 95 | } 96 | void print(PrintStream strm) throws IOException { 97 | for(int ch=0; ch < 256; ++ch) { 98 | for(int i = 0; i < 2*getLevel(); ++i) { 99 | strm.print(' '); 100 | } 101 | strm.print(ch); 102 | strm.println(" ->"); 103 | if (child[ch] != null) { 104 | child[ch].print(strm); 105 | } 106 | } 107 | } 108 | } 109 | 110 | /** 111 | * A leaf trie node that does string compares to figure out where the given 112 | * key belongs between lower..upper. 113 | */ 114 | static class LeafTrieNode extends TrieNode { 115 | int lower; 116 | int upper; 117 | Text[] splitPoints; 118 | LeafTrieNode(int level, Text[] splitPoints, int lower, int upper) { 119 | super(level); 120 | this.splitPoints = splitPoints; 121 | this.lower = lower; 122 | this.upper = upper; 123 | } 124 | int findPartition(Text key) { 125 | for(int i=lower; i= 0) { 127 | return i; 128 | } 129 | } 130 | return upper; 131 | } 132 | void print(PrintStream strm) throws IOException { 133 | for(int i = 0; i < 2*getLevel(); ++i) { 134 | strm.print(' '); 135 | } 136 | strm.print(lower); 137 | strm.print(", "); 138 | strm.println(upper); 139 | } 140 | } 141 | 142 | 143 | /** 144 | * Read the cut points from the given sequence file. 145 | * @param fs the file system 146 | * @param p the path to read 147 | * @param job the job config 148 | * @return the strings to split the partitions on 149 | * @throws IOException 150 | */ 151 | private static Text[] readPartitions(FileSystem fs, Path p, Configuration conf) throws IOException { 152 | int reduces = conf.getInt(MRJobConfig.NUM_REDUCES, 1); 153 | Text[] result = new Text[reduces - 1]; 154 | DataInputStream reader = fs.open(p); 155 | for(int i=0; i < reduces - 1; ++i) { 156 | result[i] = new Text(); 157 | result[i].readFields(reader); 158 | } 159 | reader.close(); 160 | return result; 161 | } 162 | 163 | /** 164 | * Given a sorted set of cut points, build a trie that will find the correct 165 | * partition quickly. 166 | * @param splits the list of cut points 167 | * @param lower the lower bound of partitions 0..numPartitions-1 168 | * @param upper the upper bound of partitions 0..numPartitions-1 169 | * @param prefix the prefix that we have already checked against 170 | * @param maxDepth the maximum depth we will build a trie for 171 | * @return the trie node that will divide the splits correctly 172 | */ 173 | private static TrieNode buildTrie(Text[] splits, int lower, int upper, Text prefix, int maxDepth) { 174 | int depth = prefix.getLength(); 175 | if (depth >= maxDepth || lower == upper) { 176 | return new LeafTrieNode(depth, splits, lower, upper); 177 | } 178 | InnerTrieNode result = new InnerTrieNode(depth); 179 | Text trial = new Text(prefix); 180 | // append an extra byte on to the prefix 181 | trial.append(new byte[1], 0, 1); 182 | int currentBound = lower; 183 | for(int ch = 0; ch < 255; ++ch) { 184 | trial.getBytes()[depth] = (byte) (ch + 1); 185 | lower = currentBound; 186 | while (currentBound < upper) { 187 | if (splits[currentBound].compareTo(trial) >= 0) { 188 | break; 189 | } 190 | currentBound += 1; 191 | } 192 | trial.getBytes()[depth] = (byte) ch; 193 | result.child[ch] = buildTrie(splits, lower, currentBound, trial, 194 | maxDepth); 195 | } 196 | // pick up the rest 197 | trial.getBytes()[depth] = (byte) 255; 198 | result.child[255] = buildTrie(splits, currentBound, upper, trial, 199 | maxDepth); 200 | return result; 201 | } 202 | 203 | // eastcirclek 204 | private static TrieNode buildTrieFromHDFS(Configuration conf, Path hdfsPath) throws IOException{ 205 | FileSystem fs = hdfsPath.getFileSystem(conf); 206 | Text[] splitPoints = readPartitions(fs, hdfsPath, conf); 207 | return buildTrie(splitPoints, 0, splitPoints.length, new Text(), 2); 208 | } 209 | 210 | // eastcirclek 211 | public TotalOrderPartitioner(Configuration conf, 212 | Path partFile) throws IOException{ 213 | this.conf = conf; 214 | this.partFile = partFile; 215 | this.trie = buildTrieFromHDFS(conf, partFile); 216 | } 217 | 218 | // eastcirclek 219 | public int getPartition(Text key){ 220 | return trie.findPartition(key); 221 | } 222 | 223 | // eastcirclek for serialization 224 | private void writeObject(ObjectOutputStream out) throws IOException{ 225 | out.writeUTF(conf.get("fs.defaultFS")); 226 | out.writeInt(conf.getInt(MRJobConfig.NUM_REDUCES, 2)); 227 | /** 228 | * Instead of serializing the trie, 229 | * we serialize the filename containing sampling points 230 | * so that we can rebuild the trie in each task. 231 | */ 232 | out.writeUTF(this.partFile.toString()); 233 | } 234 | 235 | // eastcirclek for deserialization 236 | private void readObject(ObjectInputStream in) throws IOException{ 237 | this.conf = new Configuration(); 238 | conf.set("fs.defaultFS", (String)in.readUTF()); 239 | conf.setInt(MRJobConfig.NUM_REDUCES, (int)in.readInt()); 240 | this.partFile = new Path((String)in.readUTF()); 241 | this.trie = buildTrieFromHDFS(conf, partFile); 242 | } 243 | } 244 | -------------------------------------------------------------------------------- /src/main/java/eastcircle/terasort/TeraInputFormat.java: -------------------------------------------------------------------------------- 1 | /** 2 | * Licensed to the Apache Software Foundation (ASF) under one 3 | * or more contributor license agreements. See the NOTICE file 4 | * distributed with this work for additional information 5 | * regarding copyright ownership. The ASF licenses this file 6 | * to you under the Apache License, Version 2.0 (the 7 | * "License"); you may not use this file except in compliance 8 | * with the License. You may obtain a copy of the License at 9 | * 10 | * http://www.apache.org/licenses/LICENSE-2.0 11 | * 12 | * Unless required by applicable law or agreed to in writing, software 13 | * distributed under the License is distributed on an "AS IS" BASIS, 14 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 15 | * See the License for the specific language governing permissions and 16 | * limitations under the License. 17 | */ 18 | 19 | package eastcircle.terasort; 20 | 21 | import java.io.DataOutputStream; 22 | import java.io.EOFException; 23 | import java.io.IOException; 24 | import java.util.ArrayList; 25 | import java.util.List; 26 | 27 | import org.apache.hadoop.conf.Configuration; 28 | import org.apache.hadoop.fs.FSDataInputStream; 29 | import org.apache.hadoop.fs.FileSystem; 30 | import org.apache.hadoop.fs.Path; 31 | import org.apache.hadoop.io.Text; 32 | import org.apache.hadoop.mapreduce.InputSplit; 33 | import org.apache.hadoop.mapreduce.JobContext; 34 | import org.apache.hadoop.mapreduce.MRJobConfig; 35 | import org.apache.hadoop.mapreduce.RecordReader; 36 | import org.apache.hadoop.mapreduce.TaskAttemptContext; 37 | import org.apache.hadoop.mapreduce.TaskAttemptID; 38 | import org.apache.hadoop.mapreduce.lib.input.FileInputFormat; 39 | import org.apache.hadoop.mapreduce.lib.input.FileSplit; 40 | import org.apache.hadoop.mapreduce.task.TaskAttemptContextImpl; 41 | import org.apache.hadoop.util.IndexedSortable; 42 | import org.apache.hadoop.util.QuickSort; 43 | import org.apache.hadoop.util.StringUtils; 44 | 45 | /** 46 | * An input format that reads the first 10 characters of each line as the key 47 | * and the rest of the line as the value. Both key and value are represented 48 | * as Text. 49 | */ 50 | public class TeraInputFormat extends FileInputFormat { 51 | 52 | static final String PARTITION_FILENAME = "_partition.lst"; 53 | private static final String NUM_PARTITIONS = 54 | "mapreduce.terasort.num.partitions"; 55 | private static final String SAMPLE_SIZE = 56 | "mapreduce.terasort.partitions.sample"; 57 | static final int KEY_LENGTH = 10; 58 | static final int VALUE_LENGTH = 90; 59 | static final int RECORD_LENGTH = KEY_LENGTH + VALUE_LENGTH; 60 | private static MRJobConfig lastContext = null; 61 | private static List lastResult = null; 62 | 63 | static class TextSampler implements IndexedSortable { 64 | private ArrayList records = new ArrayList(); 65 | 66 | public int compare(int i, int j) { 67 | Text left = records.get(i); 68 | Text right = records.get(j); 69 | return left.compareTo(right); 70 | } 71 | 72 | public void swap(int i, int j) { 73 | Text left = records.get(i); 74 | Text right = records.get(j); 75 | records.set(j, left); 76 | records.set(i, right); 77 | } 78 | 79 | public void addKey(Text key) { 80 | synchronized (this) { 81 | records.add(new Text(key)); 82 | } 83 | } 84 | 85 | /** 86 | * Find the split points for a given sample. The sample keys are sorted 87 | * and down sampled to find even split points for the partitions. The 88 | * returned keys should be the start of their respective partitions. 89 | * @param numPartitions the desired number of partitions 90 | * @return an array of size numPartitions - 1 that holds the split points 91 | */ 92 | Text[] createPartitions(int numPartitions) { 93 | int numRecords = records.size(); 94 | System.out.println("Making " + numPartitions + " from " + numRecords + 95 | " sampled records"); 96 | if (numPartitions > numRecords) { 97 | throw new IllegalArgumentException 98 | ("Requested more partitions than input keys (" + numPartitions + 99 | " > " + numRecords + ")"); 100 | } 101 | new QuickSort().sort(this, 0, records.size()); 102 | float stepSize = numRecords / (float) numPartitions; 103 | Text[] result = new Text[numPartitions-1]; 104 | for(int i=1; i < numPartitions; ++i) { 105 | result[i-1] = records.get(Math.round(stepSize * i)); 106 | } 107 | return result; 108 | } 109 | } 110 | 111 | /** 112 | * Use the input splits to take samples of the input and generate sample 113 | * keys. By default reads 100,000 keys from 10 locations in the input, sorts 114 | * them and picks N-1 keys to generate N equally sized partitions. 115 | * @param job the job to sample 116 | * @param partFile where to write the output file to 117 | * @throws Throwable if something goes wrong 118 | */ 119 | public static void writePartitionFile(final JobContext job, 120 | Path partFile) throws Throwable { 121 | long t1 = System.currentTimeMillis(); 122 | Configuration conf = job.getConfiguration(); 123 | final TeraInputFormat inFormat = new TeraInputFormat(); 124 | final TextSampler sampler = new TextSampler(); 125 | int partitions = job.getNumReduceTasks(); 126 | long sampleSize = conf.getLong(SAMPLE_SIZE, 100000); 127 | final List splits = inFormat.getSplits(job); 128 | long t2 = System.currentTimeMillis(); 129 | System.out.println("Computing input splits took " + (t2 - t1) + "ms"); 130 | int samples = Math.min(conf.getInt(NUM_PARTITIONS, 10), splits.size()); 131 | System.out.println("Sampling " + samples + " splits of " + splits.size()); 132 | final long recordsPerSample = sampleSize / samples; 133 | final int sampleStep = splits.size() / samples; 134 | Thread[] samplerReader = new Thread[samples]; 135 | SamplerThreadGroup threadGroup = new SamplerThreadGroup("Sampler Reader Thread Group"); 136 | // take N samples from different parts of the input 137 | for(int i=0; i < samples; ++i) { 138 | final int idx = i; 139 | samplerReader[i] = 140 | new Thread (threadGroup,"Sampler Reader " + idx) { 141 | { 142 | setDaemon(true); 143 | } 144 | public void run() { 145 | long records = 0; 146 | try { 147 | TaskAttemptContext context = new TaskAttemptContextImpl( 148 | job.getConfiguration(), new TaskAttemptID()); 149 | RecordReader reader = 150 | inFormat.createRecordReader(splits.get(sampleStep * idx), 151 | context); 152 | reader.initialize(splits.get(sampleStep * idx), context); 153 | while (reader.nextKeyValue()) { 154 | sampler.addKey(new Text(reader.getCurrentKey())); 155 | records += 1; 156 | if (recordsPerSample <= records) { 157 | break; 158 | } 159 | } 160 | } catch (IOException ie){ 161 | System.err.println("Got an exception while reading splits " + 162 | StringUtils.stringifyException(ie)); 163 | throw new RuntimeException(ie); 164 | } catch (InterruptedException e) { 165 | 166 | } 167 | } 168 | }; 169 | samplerReader[i].start(); 170 | } 171 | FileSystem outFs = partFile.getFileSystem(conf); 172 | DataOutputStream writer = outFs.create(partFile, true, 64*1024, (short) 10, 173 | outFs.getDefaultBlockSize(partFile)); 174 | for (int i = 0; i < samples; i++) { 175 | try { 176 | samplerReader[i].join(); 177 | if(threadGroup.getThrowable() != null){ 178 | throw threadGroup.getThrowable(); 179 | } 180 | } catch (InterruptedException e) { 181 | } 182 | } 183 | for(Text split : sampler.createPartitions(partitions)) { 184 | split.write(writer); 185 | } 186 | writer.close(); 187 | long t3 = System.currentTimeMillis(); 188 | System.out.println("Computing parititions took " + (t3 - t2) + "ms"); 189 | } 190 | 191 | static class SamplerThreadGroup extends ThreadGroup{ 192 | 193 | private Throwable throwable; 194 | 195 | public SamplerThreadGroup(String s) { 196 | super(s); 197 | } 198 | 199 | @Override 200 | public void uncaughtException(Thread thread, Throwable throwable) { 201 | this.throwable = throwable; 202 | } 203 | 204 | public Throwable getThrowable() { 205 | return this.throwable; 206 | } 207 | 208 | } 209 | 210 | static class TeraRecordReader extends RecordReader { 211 | private FSDataInputStream in; 212 | private long offset; 213 | private long length; 214 | private static final int RECORD_LENGTH = KEY_LENGTH + VALUE_LENGTH; 215 | private byte[] buffer = new byte[RECORD_LENGTH]; 216 | private Text key; 217 | private Text value; 218 | 219 | public TeraRecordReader() throws IOException { 220 | } 221 | 222 | public void initialize(InputSplit split, TaskAttemptContext context) 223 | throws IOException, InterruptedException { 224 | Path p = ((FileSplit)split).getPath(); 225 | FileSystem fs = p.getFileSystem(context.getConfiguration()); 226 | in = fs.open(p); 227 | long start = ((FileSplit)split).getStart(); 228 | // find the offset to start at a record boundary 229 | offset = (RECORD_LENGTH - (start % RECORD_LENGTH)) % RECORD_LENGTH; 230 | in.seek(start + offset); 231 | length = ((FileSplit)split).getLength(); 232 | } 233 | 234 | public void close() throws IOException { 235 | in.close(); 236 | } 237 | 238 | public Text getCurrentKey() { 239 | return key; 240 | } 241 | 242 | public Text getCurrentValue() { 243 | return value; 244 | } 245 | 246 | public float getProgress() throws IOException { 247 | return (float) offset / length; 248 | } 249 | 250 | public boolean nextKeyValue() throws IOException { 251 | if (offset >= length) { 252 | return false; 253 | } 254 | int read = 0; 255 | while (read < RECORD_LENGTH) { 256 | long newRead = in.read(buffer, read, RECORD_LENGTH - read); 257 | if (newRead == -1) { 258 | if (read == 0) { 259 | return false; 260 | } else { 261 | throw new EOFException("read past eof"); 262 | } 263 | } 264 | read += newRead; 265 | } 266 | if (key == null) { 267 | key = new Text(); 268 | } 269 | if (value == null) { 270 | value = new Text(); 271 | } 272 | key.set(buffer, 0, KEY_LENGTH); 273 | value.set(buffer, KEY_LENGTH, VALUE_LENGTH); 274 | offset += RECORD_LENGTH; 275 | return true; 276 | } 277 | } 278 | 279 | @Override 280 | public RecordReader 281 | createRecordReader(InputSplit split, TaskAttemptContext context) 282 | throws IOException { 283 | return new TeraRecordReader(); 284 | } 285 | 286 | @Override 287 | public List getSplits(JobContext job) throws IOException { 288 | if (job == lastContext) { 289 | return lastResult; 290 | } 291 | long t1, t2, t3; 292 | t1 = System.currentTimeMillis(); 293 | lastContext = job; 294 | lastResult = super.getSplits(job); 295 | t2 = System.currentTimeMillis(); 296 | System.out.println("Spent " + (t2 - t1) + "ms computing base-splits."); 297 | if (job.getConfiguration().getBoolean(TeraScheduler.USE, true)) { 298 | TeraScheduler scheduler = new TeraScheduler( 299 | lastResult.toArray(new FileSplit[0]), job.getConfiguration()); 300 | lastResult = scheduler.getNewFileSplits(); 301 | t3 = System.currentTimeMillis(); 302 | System.out.println("Spent " + (t3 - t2) + "ms computing TeraScheduler splits."); 303 | } 304 | return lastResult; 305 | } 306 | } 307 | --------------------------------------------------------------------------------