package org.apache.hadoop.hdfs.server.datanode;

import java.io.BufferedOutputStream;
import java.io.DataInputStream;
import java.io.DataOutputStream;
import java.io.FileOutputStream;
import java.io.IOException;
import java.io.InterruptedIOException;
import java.io.OutputStream;
import java.nio.channels.ClosedByInterruptException;
import java.nio.channels.ClosedChannelException;
import java.util.ArrayList;
import java.util.Arrays;
import java.util.LinkedList;
import java.util.List;
import java.util.concurrent.ArrayBlockingQueue;
import java.util.concurrent.BlockingQueue;
import java.util.concurrent.ExecutorService;
import java.util.concurrent.Executors;
import java.util.zip.CRC32;
import java.util.zip.Checksum;

import org.apache.commons.logging.Log;
import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.fs.FSInputChecker;
import org.apache.hadoop.fs.FSOutputSummer;
import org.apache.hadoop.hdfs.DataTransferPacket;
import org.apache.hadoop.hdfs.protocol.Block;
import org.apache.hadoop.hdfs.protocol.FSConstants;
import org.apache.hadoop.hdfs.util.DataTransferThrottler;
import org.apache.hadoop.io.IOUtils;
import org.apache.hadoop.util.ChecksumUtil;
import org.apache.hadoop.util.DataChecksum;
import org.apache.hadoop.util.PureJavaCrc32;
import org.apache.hadoop.util.StringUtils;

/**
 * A class that merges a block and writes to its own disk, or transfer to its
 * parent site. If a throttler is provided, streaming throttling is also supported.
 **/
abstract public class BlockXCodingMerger implements java.io.Closeable, FSConstants {
	static {
		Configuration.addDefaultResource("raid-site.xml");
	}
	protected static final Log LOG = DataNode.LOG;
	static final Log ClientTraceLog = DataNode.ClientTraceLog;

	protected Block block; 	// the block that is being reconstructed
	protected int namespaceId; // the namespace the block being reconstructed belongs to
	protected DataInputStream[] childInputStreams = null; // from where data are read
	protected DataChecksum checksum;   // the checksum of data read
	protected int bytesPerChecksum = 0;
	protected int checksumSize = 0;
	protected long offsetInBlock;
	protected long length;
	protected final String[] childAddrs;
	protected final String myAddr;
	private DataTransferThrottler throttler;
	protected int mergerLevel; // the level of merger in the recovery tree
							   // the level of root is 0
	int packetSize = 4096;

	public BlockXCodingMerger(Block block, int namespaceId,
			DataInputStream[] childInputStreams, long offsetInBlock,
			long length, String[] childAddrs, String myAddr,
			DataTransferThrottler throttler,
			int mergerLevel) throws IOException{
		super();
		this.block = block;
		this.namespaceId = namespaceId;
		this.childInputStreams = childInputStreams;
		this.offsetInBlock = offsetInBlock;
		this.length = length;
		this.childAddrs = childAddrs;
		this.myAddr = myAddr;
		this.throttler = throttler;
		this.mergerLevel = mergerLevel;
		Configuration conf = new Configuration();
		this.packetSize = conf.getInt("raid.blockreconstruct.packetsize", 4096);
		this.bytesPerChecksum = conf.getInt("io.bytes.per.checksum", 512);
		this.checksum = DataChecksum.newDataChecksum(DataChecksum.CHECKSUM_CRC32,
				bytesPerChecksum, new PureJavaCrc32());
		this.checksumSize = checksum.getChecksumSize();
	}
	
	/**
	 * Write the data packet merged from child sites somewhere
	 * 
	 * @param buffer
	 * 		the ByteBuffer of packet merged
	 * @param packetInfo
	 * 		the information of the packet merged
	 * @throws IOException
	 */
	abstract void writePacket(DataTransferPacket packet) throws IOException;
	abstract void finalizeWrite() throws IOException;
	abstract void handleException(Exception e) throws IOException;


	class PacketReader implements Runnable {
		int id;
		int bufferSize;
		DataInputStream inputStream;
		BlockingQueue<DataTransferPacket> readQueue;
		
		PacketReader(DataInputStream inputStream, int bufferSize,
				BlockingQueue<DataTransferPacket> readQueue, int id){
			this.inputStream = inputStream;
			this.readQueue = readQueue;
			this.id = id;
			this.bufferSize = bufferSize;
		}
		
		public void run() {
			boolean running = true;
			try {
				while(running && !Thread.interrupted()) {
					DataTransferPacket packet = new DataTransferPacket(bufferSize);
					try {
						packet.read(inputStream);
					} catch (IOException e) {
						packet.reset();
						packet.dataLength = -1;
						LOG.error("NTar: " + this + " : Exception occured during reading packet:" 
								+ StringUtils.stringifyException(e));
						packet.errMessage = this + " : Exception occured during reading packet:" 
								+ StringUtils.stringifyException(e);
					}
					readQueue.put(packet);
					if(packet.dataLength <= 0)
						running = false;
				}
			} catch (InterruptedException e) {
				//LOG.error("NTar: " + this + " : interrupted:" + StringUtils.stringifyException(e));	
			}
		}
		
		public String toString() {
			return "Packet Reader [" + id + "] of BlockXCodingMerger";
		}
	}
	
	class PacketMerger implements Runnable {
		int bufferSize;
		List<BlockingQueue<DataTransferPacket>> readQueues;
		BlockingQueue<DataTransferPacket> mergeQueue;
		
		PacketMerger(List<BlockingQueue<DataTransferPacket>> readQueues, int bufferSize,
				BlockingQueue<DataTransferPacket> mergeQueue) {
			this.readQueues =  new ArrayList<BlockingQueue<DataTransferPacket>>(readQueues.size());
			this.readQueues.addAll(readQueues);
			this.mergeQueue = mergeQueue;
			this.bufferSize = bufferSize;
		}
		
		@Override
		public void run() {
			boolean running = true;
			List<BlockingQueue<DataTransferPacket>> finishedQueue = 
					new LinkedList<BlockingQueue<DataTransferPacket>>();
			try {
				while (running && !Thread.interrupted()) {
					DataTransferPacket mergedPacket = null;
					
					for (BlockingQueue<DataTransferPacket> queue : readQueues) {
						DataTransferPacket packet = queue.take();
						if (packet.dataLength < 0) {
							mergedPacket = packet;
							break;
						}
						if (packet.dataLength == 0) {
							finishedQueue.add(queue);
						}
						
						if (mergedPacket == null) {
							mergedPacket = packet;
							continue;
						}
						
						if ((packet.dataLength > 0) && (mergedPacket.dataLength > 0)
								&& (mergedPacket.offsetInBlock != packet.offsetInBlock)) {
							String errMessage = this + ":offsetInBlock does not match "
									+ mergedPacket.offsetInBlock + " vs " + packet.offsetInBlock;
							mergedPacket.reset();
							mergedPacket.dataLength = -1;
							mergedPacket.errMessage = errMessage;
							break;
						}
						if (mergedPacket.dataLength < packet.dataLength) {
							DataTransferPacket tempPacket = mergedPacket;
							mergedPacket = p