blob: 2be49a6db544f8004ffbd72e84e579918adf8a0c [file] [log] [blame]
/*
* 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.pig.backend.hadoop.executionengine.spark.converter;
import java.io.IOException;
import java.util.List;
import scala.collection.JavaConversions;
import org.apache.pig.backend.hadoop.executionengine.physicalLayer.relationalOperators.POUnion;
import org.apache.pig.backend.hadoop.executionengine.spark.SparkUtil;
import org.apache.pig.data.Tuple;
import org.apache.spark.SparkContext;
import org.apache.spark.rdd.RDD;
import org.apache.spark.rdd.UnionRDD;
public class UnionConverter implements RDDConverter<Tuple, Tuple, POUnion> {
private final SparkContext sc;
public UnionConverter(SparkContext sc) {
this.sc = sc;
}
@Override
public RDD<Tuple> convert(List<RDD<Tuple>> predecessors,
POUnion physicalOperator) throws IOException {
SparkUtil.assertPredecessorSizeGreaterThan(predecessors,
physicalOperator, 0);
UnionRDD<Tuple> unionRDD = new UnionRDD<Tuple>(sc,
JavaConversions.asScalaBuffer(predecessors),
SparkUtil.getManifest(Tuple.class));
return unionRDD;
}
}