You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@pig.apache.org by xu...@apache.org on 2016/09/09 04:50:48 UTC

svn commit: r1759940 - in /pig/branches/spark/src/org/apache/pig/backend/hadoop/executionengine: physicalLayer/plans/ physicalLayer/relationalOperators/ spark/ spark/converter/

Author: xuefu
Date: Fri Sep  9 04:50:47 2016
New Revision: 1759940

URL: http://svn.apache.org/viewvc?rev=1759940&view=rev
Log:
PIG-5024: add a physical operator to broadcast small RDDs (Xianda via Xuefu)

Added:
    pig/branches/spark/src/org/apache/pig/backend/hadoop/executionengine/physicalLayer/relationalOperators/POBroadcastSpark.java
    pig/branches/spark/src/org/apache/pig/backend/hadoop/executionengine/spark/converter/BroadcastConverter.java
Modified:
    pig/branches/spark/src/org/apache/pig/backend/hadoop/executionengine/physicalLayer/plans/PhyPlanVisitor.java
    pig/branches/spark/src/org/apache/pig/backend/hadoop/executionengine/spark/JobGraphBuilder.java
    pig/branches/spark/src/org/apache/pig/backend/hadoop/executionengine/spark/SparkUtil.java

Modified: pig/branches/spark/src/org/apache/pig/backend/hadoop/executionengine/physicalLayer/plans/PhyPlanVisitor.java
URL: http://svn.apache.org/viewvc/pig/branches/spark/src/org/apache/pig/backend/hadoop/executionengine/physicalLayer/plans/PhyPlanVisitor.java?rev=1759940&r1=1759939&r2=1759940&view=diff
==============================================================================
--- pig/branches/spark/src/org/apache/pig/backend/hadoop/executionengine/physicalLayer/plans/PhyPlanVisitor.java (original)
+++ pig/branches/spark/src/org/apache/pig/backend/hadoop/executionengine/physicalLayer/plans/PhyPlanVisitor.java Fri Sep  9 04:50:47 2016
@@ -73,6 +73,7 @@ import org.apache.pig.backend.hadoop.exe
 import org.apache.pig.backend.hadoop.executionengine.physicalLayer.relationalOperators.POStore;
 import org.apache.pig.backend.hadoop.executionengine.physicalLayer.relationalOperators.POStream;
 import org.apache.pig.backend.hadoop.executionengine.physicalLayer.relationalOperators.POUnion;
+import org.apache.pig.backend.hadoop.executionengine.physicalLayer.relationalOperators.POBroadcastSpark;
 import org.apache.pig.impl.plan.PlanVisitor;
 import org.apache.pig.impl.plan.PlanWalker;
 import org.apache.pig.impl.plan.VisitorException;
@@ -363,4 +364,7 @@ public class PhyPlanVisitor extends Plan
 
     public void visitPoissonSample(POPoissonSample poissonSample) throws VisitorException {
     }
+
+    public void visitBroadcastSpark(POBroadcastSpark poBroadcastSpark) {
+    }
 }

Added: pig/branches/spark/src/org/apache/pig/backend/hadoop/executionengine/physicalLayer/relationalOperators/POBroadcastSpark.java
URL: http://svn.apache.org/viewvc/pig/branches/spark/src/org/apache/pig/backend/hadoop/executionengine/physicalLayer/relationalOperators/POBroadcastSpark.java?rev=1759940&view=auto
==============================================================================
--- pig/branches/spark/src/org/apache/pig/backend/hadoop/executionengine/physicalLayer/relationalOperators/POBroadcastSpark.java (added)
+++ pig/branches/spark/src/org/apache/pig/backend/hadoop/executionengine/physicalLayer/relationalOperators/POBroadcastSpark.java Fri Sep  9 04:50:47 2016
@@ -0,0 +1,80 @@
+/*
+ * 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.physicalLayer.relationalOperators;
+
+import org.apache.pig.backend.executionengine.ExecException;
+import org.apache.pig.backend.hadoop.executionengine.physicalLayer.PhysicalOperator;
+import org.apache.pig.backend.hadoop.executionengine.physicalLayer.plans.PhyPlanVisitor;
+import org.apache.pig.data.Tuple;
+import org.apache.pig.impl.plan.OperatorKey;
+import org.apache.pig.impl.plan.VisitorException;
+
+public class POBroadcastSpark extends PhysicalOperator {
+    private static final long serialVersionUID = 1L;
+
+    protected String broadcastedVariableName;
+
+    public POBroadcastSpark(OperatorKey k) {
+        super(k);
+    }
+
+    public POBroadcastSpark(POBroadcastSpark copy)
+            throws ExecException {
+        super(copy);
+    }
+
+    /**
+     * Set your broadcast variable name so that
+     * BroadcastConverter can put this broadcasted variable in a map
+     * which can be referenced by other functions / closures in Converters
+     *
+     * @param varName
+     */
+    public void setBroadcastedVariableName(String varName) {
+        broadcastedVariableName = varName;
+    }
+
+    public String getBroadcastedVariableName() {
+        return broadcastedVariableName;
+    }
+
+    @Override
+    public Tuple illustratorMarkup(Object in, Object out, int eqClassIndex) {
+        return null;
+    }
+
+    @Override
+    public boolean supportsMultipleInputs() {
+        return false;
+    }
+
+    @Override
+    public boolean supportsMultipleOutputs() {
+        return false;
+    }
+
+    @Override
+    public String name() {
+        return getAliasString() + "BroadcastSpark - " + mKey.toString();
+    }
+
+    @Override
+    public void visit(PhyPlanVisitor v) throws VisitorException {
+        v.visitBroadcastSpark(this);
+    }
+}

Modified: pig/branches/spark/src/org/apache/pig/backend/hadoop/executionengine/spark/JobGraphBuilder.java
URL: http://svn.apache.org/viewvc/pig/branches/spark/src/org/apache/pig/backend/hadoop/executionengine/spark/JobGraphBuilder.java?rev=1759940&r1=1759939&r2=1759940&view=diff
==============================================================================
--- pig/branches/spark/src/org/apache/pig/backend/hadoop/executionengine/spark/JobGraphBuilder.java (original)
+++ pig/branches/spark/src/org/apache/pig/backend/hadoop/executionengine/spark/JobGraphBuilder.java Fri Sep  9 04:50:47 2016
@@ -63,6 +63,7 @@ import org.apache.pig.impl.plan.VisitorE
 import org.apache.pig.tools.pigstats.spark.SparkPigStats;
 import org.apache.pig.tools.pigstats.spark.SparkStatsUtil;
 import org.apache.spark.api.java.JavaSparkContext;
+import org.apache.spark.broadcast.Broadcast;
 import org.apache.spark.rdd.RDD;
 
 import com.google.common.collect.Lists;

Modified: pig/branches/spark/src/org/apache/pig/backend/hadoop/executionengine/spark/SparkUtil.java
URL: http://svn.apache.org/viewvc/pig/branches/spark/src/org/apache/pig/backend/hadoop/executionengine/spark/SparkUtil.java?rev=1759940&r1=1759939&r2=1759940&view=diff
==============================================================================
--- pig/branches/spark/src/org/apache/pig/backend/hadoop/executionengine/spark/SparkUtil.java (original)
+++ pig/branches/spark/src/org/apache/pig/backend/hadoop/executionengine/spark/SparkUtil.java Fri Sep  9 04:50:47 2016
@@ -23,7 +23,9 @@ import java.util.LinkedList;
 import java.util.List;
 import java.util.Random;
 import java.util.UUID;
+import java.util.concurrent.ConcurrentHashMap;
 
+import org.apache.spark.broadcast.Broadcast;
 import scala.Product2;
 import scala.Tuple2;
 import scala.collection.JavaConversions;
@@ -57,6 +59,8 @@ import org.apache.spark.rdd.RDD;
 
 public class SparkUtil {
 
+    private static ConcurrentHashMap<String, Broadcast<List<Tuple>>> broadcastedVars = new ConcurrentHashMap() ;
+
     public static <T> ClassTag<T> getManifest(Class<T> clazz) {
         return ClassTag$.MODULE$.apply(clazz);
     }
@@ -165,4 +169,8 @@ public class SparkUtil {
         //POSort is added to sort the index tuples genereated by MergeJoinIndexer.More detail, see PIG-4601
         baseSparkOp.physicalPlan.addAsLeaf(sort);
     }
+
+    static public ConcurrentHashMap<String, Broadcast<List<Tuple>>> getBroadcastedVars() {
+        return broadcastedVars;
+    }
 }
\ No newline at end of file

Added: pig/branches/spark/src/org/apache/pig/backend/hadoop/executionengine/spark/converter/BroadcastConverter.java
URL: http://svn.apache.org/viewvc/pig/branches/spark/src/org/apache/pig/backend/hadoop/executionengine/spark/converter/BroadcastConverter.java?rev=1759940&view=auto
==============================================================================
--- pig/branches/spark/src/org/apache/pig/backend/hadoop/executionengine/spark/converter/BroadcastConverter.java (added)
+++ pig/branches/spark/src/org/apache/pig/backend/hadoop/executionengine/spark/converter/BroadcastConverter.java Fri Sep  9 04:50:47 2016
@@ -0,0 +1,54 @@
+/*
+ * 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 org.apache.pig.backend.hadoop.executionengine.physicalLayer.relationalOperators.POBroadcastSpark;
+import org.apache.pig.backend.hadoop.executionengine.spark.SparkUtil;
+import org.apache.pig.data.Tuple;
+import org.apache.spark.api.java.JavaRDD;
+import org.apache.spark.api.java.JavaSparkContext;
+import org.apache.spark.broadcast.Broadcast;
+import org.apache.spark.rdd.RDD;
+
+import java.util.List;
+
+public class BroadcastConverter implements RDDConverter<Tuple, Tuple, POBroadcastSpark> {
+
+    private final JavaSparkContext sc;
+
+    public BroadcastConverter(JavaSparkContext sc) {
+        this.sc = sc;
+    }
+
+    @Override
+    public RDD<Tuple> convert(List<RDD<Tuple>> predecessors, POBroadcastSpark po) {
+        SparkUtil.assertPredecessorSize(predecessors, po, 1);
+        RDD<Tuple> rdd = predecessors.get(0);
+
+        // Just collect the predecessor RDD, and broadcast it
+        JavaRDD<Tuple> javaRDD = new JavaRDD<>(rdd, SparkUtil.getManifest(Tuple.class));
+        Broadcast<List<Tuple>> broadcastedRDD = sc.broadcast(javaRDD.collect());
+
+        // Save the broadcast variable to broadcastedVars map, so that this
+        // broadcasted variable can be referenced by the driver client.
+        SparkUtil.getBroadcastedVars().put(po.getBroadcastedVariableName(), broadcastedRDD);
+
+        return rdd;
+    }
+
+}