/**
 * 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 org.apache.hadoop.raid;

import java.io.InputStream;
import java.io.OutputStream;
import java.io.File;
import java.io.FileInputStream;
import java.io.FileOutputStream;
import java.io.IOException;
import java.util.Random;

import org.apache.commons.logging.Log;
import org.apache.commons.logging.LogFactory;
import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.fs.FileStatus;
import org.apache.hadoop.fs.FileSystem;
import org.apache.hadoop.fs.FSDataOutputStream;
import org.apache.hadoop.fs.Path;
import org.apache.hadoop.util.Progressable;

/**
 * Represents a generic encoder that can generate a parity file for a source
 * file.
 */
public class Encoder {
	public static final Log LOG = LogFactory
			.getLog("org.apache.hadoop.raid.Encoder");
	public static final int DEFAULT_PARALLELISM = 4;
	protected Configuration conf;
	protected int parallelism;
	protected Codec codec;
	protected ErasureCode code;
	protected Random rand;
	protected int bufSize;
	protected byte[][] readBufs;
	protected byte[][] writeBufs;
	
	/**
	 * added by jason
	 */
	public static final int DEFAULT_CODING_THREADNUM = 5;
	protected int codingThreadNum;
	public static final boolean DEFAULT_CODING_IFPARALLELISM = true;
	protected boolean ifCodingParallelism;			
	/**
	 * added by jason ended
	 */

	/**
	 * A class that acts as a sink for data, similar to /dev/null.
	 */
	static class NullOutputStream extends OutputStream {
		public void write(byte[] b) throws IOException {
		}

		public void write(int b) throws IOException {
		}

		public void write(byte[] b, int off, int len) throws IOException {
		}
	}

	Encoder(Configuration conf, Codec codec) {
		this.conf = conf;
		this.parallelism = conf.getInt("raid.encoder.parallelism",
				DEFAULT_PARALLELISM);
		this.codec = codec;
		this.code = codec.createErasureCode(conf);
		this.rand = new Random();
		this.bufSize = conf.getInt("raid.encoder.bufsize", 1024 * 1024);
		this.writeBufs = new byte[codec.parityLength][];
		allocateBuffers();
		
		/**
		 * added by jason
		 */
		this.codingThreadNum = conf.getInt("raid.encoder.threadnum", DEFAULT_CODING_THREADNUM);
		this.ifCodingParallelism = conf.getBoolean("raid.encoder.ifparallelism", DEFAULT_CODING_IFPARALLELISM);
		code.initThreadPool(codingThreadNum);
		/**
		 * added by jason ended
		 */
	}

	private void allocateBuffers() {
		for (int i = 0; i < codec.parityLength; i++) {
			writeBufs[i] = new byte[bufSize];
		}
	}

	private void configureBuffers(long blockSize) {
		if ((long) bufSize > blockSize) {
			bufSize = (int) blockSize;
			allocateBuffers();
		} else if (blockSize % bufSize != 0) {
			bufSize = (int) (blockSize / 256L); // heuristic.
			if (bufSize == 0) {
				bufSize = 1024;
			}
			bufSize = Math.min(bufSize, 1024 * 1024);
			allocateBuffers();
		}
	}

	/**
	 * The interface to use to generate a parity file. This method can be called
	 * multiple times with the same Encoder object, thus allowing reuse of the
	 * buffers allocated by the Encoder object.
	 * 
	 * @param fs
	 *            The filesystem containing the source file.
	 * @param srcFile
	 *            The source file.
	 * @param parityFile
	 *            The parity file to be generated.
	 */
	public void encodeFile(Configuration jobConf, FileSystem fs, Path srcFile,
			FileSystem parityFs, Path parityFile, short parityRepl,
			long numStripes, long blockSize, Progressable reporter,
			StripeReader sReader) throws IOException {

		// --Test
		TimeStatistics.clear();
		long start = System.currentTimeMillis();

		long expectedParityBlocks = numStripes * codec.parityLength;
		long expectedParityFileSize = numStripes * blockSize
				* codec.parityLength;

		// Create a tmp file to which we will write first.
		String jobID = RaidNode.getJobID(jobConf);
		Path tmpDir = new Path(codec.tmpParityDirectory, jobID);
		//Path tmpDir = new Path(codec.tmpParityDirectory);
		if (!parityFs.mkdirs(tmpDir)) {
			throw new IOException("Could not create tmp dir " + tmpDir);
		}
		//Path parityTmp = new Path(tmpDir, parityFile.getName()
		//		+ rand.nextLong());
		Path parityTmp = new Path(tmpDir, parityFile.toString().substring(1));
		//LOG.info("encoder:jobID:" + jobID 
		//		+ ",parityFile:" + parityFile
		//		+ ",parityTmp:" + parityTmp);
		Path parent = parityTmp.getParent();
		if(!parityFs.exists(parent)) {
			if (!parityFs.mkdirs(parent)) {
				throw new IOException("Could not create parent dir of parityTmp " + tmpDir);
			}
		}
		//Path parityTmp = new Path(tmpDir, parityFile);
		// Writing out a large parity file at replication 1 is difficult since
		// some datanode could die and we would not be able to close() the file.
		// So write at replication 2 and then reduce it after close() succeeds.
		short tmpRepl = parityRepl;
		if (expectedParityBlocks >= conf.getInt(
				"raid.encoder.largeparity.blocks", 20)) {
			if (parityRepl == 1) {
				tmpRepl = 2;
			}
		}
		FSDataOutputStream out = parityFs.create(parityTmp, true,
				conf.getInt("io.file.buffer.size", 64 * 1024), tmpRepl,
				blockSize);

		try {
			encodeFileToStream(sReader, blockSize, out, reporter);
			out.close();
			out = null;
			LOG.info("Wrote temp parity file " + parityTmp);
			FileStatus tmpStat = parityFs.getFileStatus(parityTmp);
			if (tmpStat.getLen() != expectedParityFileSize) {
				throw new IOException("Expected parity size "
						+ expectedParityFileSize + " does not match actual "
						+ tmpStat.getLen());
			}

			// delete destination if exists
			if (parityFs.exists(parityFile)) {
				parityFs.delete(parityFile, false);
			}
			parityFs.mkdirs(parityFile.getParent());
			if (tmpRepl > parityRepl) {
				parityFs.setReplication(parityTmp, parityRepl);
			}
			if (!parityFs.rename(parityTmp, parityFile)) {
				String msg = "Unable to rename file " + parityTmp + " to "
						+ parityFile;
				throw new IOException(msg);
			}
			LOG.info(