package org.apache.pig.backend.hadoop.executionengine.spark.converter;
import java.io.IOException;
import java.io.Serializable;
import java.util.List;
import org.apache.pig.backend.executionengine.ExecException;
import org.apache.pig.backend.hadoop.executionengine.physicalLayer.Result;
import org.apache.pig.backend.hadoop.executionengine.physicalLayer.relationalOperators.POLimit;
import org.apache.pig.backend.hadoop.executionengine.spark.ScalaUtil;
import org.apache.pig.backend.hadoop.executionengine.spark.SparkUtil;
import org.apache.pig.data.Tuple;
import org.apache.spark.rdd.RDD;
import scala.collection.Iterator;
import scala.collection.JavaConversions;
import scala.runtime.AbstractFunction1;
public class LimitConverter implements POConverter<Tuple, Tuple, POLimit> {
@Override
public RDD<Tuple> convert(List<RDD<Tuple>> predecessors, POLimit poLimit)
throws IOException {
SparkUtil.assertPredecessorSize(predecessors, poLimit, 1);
RDD<Tuple> rdd = predecessors.get(0);
LimitFunction limitFunction = new LimitFunction(poLimit);
return rdd.mapPartitions(limitFunction, false, ScalaUtil.getClassTag(Tuple.class));
}
@SuppressWarnings("serial")
private static class LimitFunction extends AbstractFunction1<Iterator<Tuple>, Iterator<Tuple>> implements Serializable {
private final POLimit poLimit;
public LimitFunction(POLimit poLimit) {
this.poLimit = poLimit;
}
@Override
public Iterator<Tuple> apply(Iterator<Tuple> i) {
final java.util.Iterator<Tuple> tuples = JavaConversions.asJavaIterator(i);
return JavaConversions.asScalaIterator(new POOutputConsumerIterator(tuples) {
protected void attach(Tuple tuple) {
poLimit.setInputs(null);
poLimit.attachInput(tuple);
}
protected Result getNextResult() throws ExecException {
return poLimit.getNextTuple();
}
});
}
}
}