[jira] [Commented] (PIG-5246) Modify bin/pig about SPARK_HOME, SPARK_ASSEMBLY_JAR after upgrading spark to 2

2017-06-16 Thread Jeff Zhang (JIRA)

[ 
https://issues.apache.org/jira/browse/PIG-5246?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=16052616#comment-16052616
 ] 

Jeff Zhang commented on PIG-5246:
-

Pig don't need to load all the jars under SPARK_HOME/jars. These jars are only 
needed when spark-submit script is launched, not necessary to be included in 
pig's classpath. Pig has already specify spark dependencies in ivy.



> Modify bin/pig about SPARK_HOME, SPARK_ASSEMBLY_JAR after upgrading spark to 2
> --
>
> Key: PIG-5246
> URL: https://issues.apache.org/jira/browse/PIG-5246
> Project: Pig
>  Issue Type: Bug
>Reporter: liyunzhang_intel
>Assignee: liyunzhang_intel
> Attachments: HBase9498.patch, PIG-5246.1.patch, PIG-5246_2.patch, 
> PIG-5246.patch
>
>
> in bin/pig.
> we copy assembly jar to pig's classpath in spark1.6.
> {code}
> # For spark mode:
> # Please specify SPARK_HOME first so that we can locate 
> $SPARK_HOME/lib/spark-assembly*.jar,
> # we will add spark-assembly*.jar to the classpath.
> if [ "$isSparkMode"  == "true" ]; then
> if [ -z "$SPARK_HOME" ]; then
>echo "Error: SPARK_HOME is not set!"
>exit 1
> fi
> # Please specify SPARK_JAR which is the hdfs path of spark-assembly*.jar 
> to allow YARN to cache spark-assembly*.jar on nodes so that it doesn't need 
> to be distributed each time an application runs.
> if [ -z "$SPARK_JAR" ]; then
>echo "Error: SPARK_JAR is not set, SPARK_JAR stands for the hdfs 
> location of spark-assembly*.jar. This allows YARN to cache 
> spark-assembly*.jar on nodes so that it doesn't need to be distributed each 
> time an application runs."
>exit 1
> fi
> if [ -n "$SPARK_HOME" ]; then
> echo "Using Spark Home: " ${SPARK_HOME}
> SPARK_ASSEMBLY_JAR=`ls ${SPARK_HOME}/lib/spark-assembly*`
> CLASSPATH=${CLASSPATH}:$SPARK_ASSEMBLY_JAR
> fi
> fi
> {code}
> after upgrade to spark2.0, we may modify it



--
This message was sent by Atlassian JIRA
(v6.4.14#64029)


[jira] [Commented] (PIG-5246) Modify bin/pig about SPARK_HOME, SPARK_ASSEMBLY_JAR after upgrading spark to 2

2017-06-16 Thread Rohini Palaniswamy (JIRA)

[ 
https://issues.apache.org/jira/browse/PIG-5246?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=16052434#comment-16052434
 ] 

Rohini Palaniswamy commented on PIG-5246:
-

bq. $SPARK_HOME/bin/spark-submit --version >/tmp/spark.version 2>&1
  This is a bad idea as it will launch a jvm which is costly. I would suggest 
checking for presence of spark-tags*.jar which is only present in Spark 2. If 
it is not present, then assume spark 1.

Also instead of doing
{code}
 SPARK_ASSEMBLY_JAR=`ls ${SPARK_HOME}/lib/spark-assembly*`
 CLASSPATH=${CLASSPATH}:$SPARK_ASSEMBLY_JAR
{code}

you should be able to just refer directly with wildcard
{code}
 CLASSPATH=${CLASSPATH}:${SPARK_HOME}/lib/spark-assembly*
{code}

> Modify bin/pig about SPARK_HOME, SPARK_ASSEMBLY_JAR after upgrading spark to 2
> --
>
> Key: PIG-5246
> URL: https://issues.apache.org/jira/browse/PIG-5246
> Project: Pig
>  Issue Type: Bug
>Reporter: liyunzhang_intel
>Assignee: liyunzhang_intel
> Attachments: HBase9498.patch, PIG-5246.1.patch, PIG-5246_2.patch, 
> PIG-5246.patch
>
>
> in bin/pig.
> we copy assembly jar to pig's classpath in spark1.6.
> {code}
> # For spark mode:
> # Please specify SPARK_HOME first so that we can locate 
> $SPARK_HOME/lib/spark-assembly*.jar,
> # we will add spark-assembly*.jar to the classpath.
> if [ "$isSparkMode"  == "true" ]; then
> if [ -z "$SPARK_HOME" ]; then
>echo "Error: SPARK_HOME is not set!"
>exit 1
> fi
> # Please specify SPARK_JAR which is the hdfs path of spark-assembly*.jar 
> to allow YARN to cache spark-assembly*.jar on nodes so that it doesn't need 
> to be distributed each time an application runs.
> if [ -z "$SPARK_JAR" ]; then
>echo "Error: SPARK_JAR is not set, SPARK_JAR stands for the hdfs 
> location of spark-assembly*.jar. This allows YARN to cache 
> spark-assembly*.jar on nodes so that it doesn't need to be distributed each 
> time an application runs."
>exit 1
> fi
> if [ -n "$SPARK_HOME" ]; then
> echo "Using Spark Home: " ${SPARK_HOME}
> SPARK_ASSEMBLY_JAR=`ls ${SPARK_HOME}/lib/spark-assembly*`
> CLASSPATH=${CLASSPATH}:$SPARK_ASSEMBLY_JAR
> fi
> fi
> {code}
> after upgrade to spark2.0, we may modify it



--
This message was sent by Atlassian JIRA
(v6.4.14#64029)


[jira] [Commented] (PIG-5191) Pig HBase 2.0.0 support

2017-06-16 Thread Rohini Palaniswamy (JIRA)

[ 
https://issues.apache.org/jira/browse/PIG-5191?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=16052419#comment-16052419
 ] 

Rohini Palaniswamy commented on PIG-5191:
-

Just to update, approach that will be taken will be 
https://issues.apache.org/jira/browse/PIG-5246?focusedCommentId=16035045=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel#comment-16035045

> Pig HBase 2.0.0 support
> ---
>
> Key: PIG-5191
> URL: https://issues.apache.org/jira/browse/PIG-5191
> Project: Pig
>  Issue Type: Improvement
>Reporter: Nandor Kollar
>Assignee: Nandor Kollar
> Fix For: 0.18.0
>
>
> Pig doesn't support HBase 2.0.0. Since the new HBase API introduces several 
> API changes, we should find a way to support both 1.x and 2.x HBase API.



--
This message was sent by Atlassian JIRA
(v6.4.14#64029)


Re: Review Request 59530: PIG-5157 Upgrade to Spark 2.0

2017-06-16 Thread Rohini Palaniswamy

---
This is an automatically generated e-mail. To reply, visit:
https://reviews.apache.org/r/59530/#review178126
---



Looks good. Just minor changes for class names and ant target.


build.xml
Lines 795 (patched)


Can you rename the current "jar" target to "jar-simple" and actually make 
this one part of the jar target? HBase and Hadoop3 can be added to it in the 
future. This way anyone who builds with "ant clean jar" as today gets a pig jar 
usable across all HBase/Spark/Hadoop3 versions. 
Developers like us can use the jar-simple target to save time.



src/org/apache/pig/backend/hadoop/executionengine/spark/SparkShim.java
Lines 38 (patched)


SparkShims



src/org/apache/pig/backend/hadoop/executionengine/spark/SparkShim.java
Lines 68 (patched)


getInstance(). get() makes you think it is a ThreadLocal variable.



src/org/apache/pig/backend/hadoop/executionengine/spark/SparkShim1.java
Lines 59 (patched)


Spark1Shims



src/org/apache/pig/backend/hadoop/executionengine/spark/SparkShim2.java
Lines 46 (patched)


Spark2Shims



src/org/apache/pig/tools/pigstats/spark/SparkJobStats1.java
Lines 32 (patched)


Spark1JobStats



src/org/apache/pig/tools/pigstats/spark/SparkJobStats2.java
Lines 31 (patched)


Spark2JobStats


- Rohini Palaniswamy


On June 14, 2017, 9:13 a.m., Nandor Kollar wrote:
> 
> ---
> This is an automatically generated e-mail. To reply, visit:
> https://reviews.apache.org/r/59530/
> ---
> 
> (Updated June 14, 2017, 9:13 a.m.)
> 
> 
> Review request for pig, liyun zhang, Rohini Palaniswamy, and Adam Szita.
> 
> 
> Repository: pig-git
> 
> 
> Description
> ---
> 
> Upgrade to Spark 2.1 API using shims.
> 
> 
> Diffs
> -
> 
>   build.xml bba2b52d9354ab909ad26f969480806f6d91911c 
>   ivy.xml 3f2c94373ba9455bbb6a3c96bfd61fc6cfaab588 
>   ivy/libraries.properties c2aed45a3244dfd108a255c7308a7dcb0dabd3b5 
>   
> src/org/apache/pig/backend/hadoop/executionengine/spark/FlatMapFunctionAdapter.java
>  PRE-CREATION 
>   
> src/org/apache/pig/backend/hadoop/executionengine/spark/JobMetricsListener.java
>  f81341233447203abc4800cc7b22a4f419e10262 
>   
> src/org/apache/pig/backend/hadoop/executionengine/spark/PairFlatMapFunctionAdapter.java
>  PRE-CREATION 
>   src/org/apache/pig/backend/hadoop/executionengine/spark/SparkLauncher.java 
> 237fd9431a16226234d91059088f91aab346b83c 
>   src/org/apache/pig/backend/hadoop/executionengine/spark/SparkShim.java 
> PRE-CREATION 
>   src/org/apache/pig/backend/hadoop/executionengine/spark/SparkShim1.java 
> PRE-CREATION 
>   src/org/apache/pig/backend/hadoop/executionengine/spark/SparkShim2.java 
> PRE-CREATION 
>   
> src/org/apache/pig/backend/hadoop/executionengine/spark/converter/CollectedGroupConverter.java
>  83311dfa5bb25209a5366c2db7e8d483c31d94cd 
>   
> src/org/apache/pig/backend/hadoop/executionengine/spark/converter/FRJoinConverter.java
>  382258e7ff9105aa397c5a2888df0c11e9562ec9 
>   
> src/org/apache/pig/backend/hadoop/executionengine/spark/converter/ForEachConverter.java
>  b58415e7e18ca4cf1331beef06e9214600a51424 
>   
> src/org/apache/pig/backend/hadoop/executionengine/spark/converter/GlobalRearrangeConverter.java
>  130c8b9a747b176ce2b649ca6d5260527595fb76 
>   
> src/org/apache/pig/backend/hadoop/executionengine/spark/converter/LimitConverter.java
>  fe1b54c8f128661d7d19c276d3bb2de7874d3086 
>   
> src/org/apache/pig/backend/hadoop/executionengine/spark/converter/MergeCogroupConverter.java
>  adf78ecab0da10d3b1a7fdde8af2b42dd899810f 
>   
> src/org/apache/pig/backend/hadoop/executionengine/spark/converter/MergeJoinConverter.java
>  d1c43b1e06adc4c9fe45a83b8110402e3756 
>   
> src/org/apache/pig/backend/hadoop/executionengine/spark/converter/PoissonSampleConverter.java
>  e003bbd95763b2d189ff9ec540c89abe52592420 
>   
> src/org/apache/pig/backend/hadoop/executionengine/spark/converter/SecondaryKeySortUtil.java
>  00d29b44848546ed16dde2baa8c61b36939971b2 
>   
> src/org/apache/pig/backend/hadoop/executionengine/spark/converter/SkewedJoinConverter.java
>  c55ba3145495a53d69db2dd56434dcc9b3bf8ed5 
>   
> src/org/apache/pig/backend/hadoop/executionengine/spark/converter/SortConverter.java
>  baabfa090323e3bef087e259ce19df2e4c34dd63 
>   
> src/org/apache/pig/backend/hadoop/executionengine/spark/converter/SparkSampleSortConverter.java
>  3166fdc31745c013380492e089c83f3e853a3e6e 
>   
> 

[jira] [Resolved] (PIG-5260) Separate bloom filter for each reducer of the join

2017-06-16 Thread Rohini Palaniswamy (JIRA)

 [ 
https://issues.apache.org/jira/browse/PIG-5260?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel
 ]

Rohini Palaniswamy resolved PIG-5260.
-
Resolution: Invalid

This is totally invalid. Writing something when you totally lack sleep is a bad 
thing. Had totally interchanged the left side load vertex with the join vertex. 
It can still be valid for the case when the left side is a intermediate reducer 
and result of group by on same key as the join key. But not worth the effort.

> Separate bloom filter for each reducer of the join
> --
>
> Key: PIG-5260
> URL: https://issues.apache.org/jira/browse/PIG-5260
> Project: Pig
>  Issue Type: New Feature
>Reporter: Rohini Palaniswamy
>
>Currently bloom join allows specifying the number of bloom filters and all 
> of them are broadcast to each join vertex. The bloom filter partition logic 
> is joinkey hashcode % num_filters. The reducer partition logic is joinkey 
> hashcode % num_reducers. If we made the number of bloom filters equal to 
> number of reducers in the join we can just broadcast bloom filter  0 to 
> reducer 0, bloom filter 1 to reducer 1 and so on. one-one edge will most 
> likely prevent auto-reduce parallelism from being applied for the 
> scatter-gather edge. So need to see if we need a custom one-one broadcast 
> edge for this.



--
This message was sent by Atlassian JIRA
(v6.4.14#64029)


[jira] [Commented] (PIG-5246) Modify bin/pig about SPARK_HOME, SPARK_ASSEMBLY_JAR after upgrading spark to 2

2017-06-16 Thread liyunzhang_intel (JIRA)

[ 
https://issues.apache.org/jira/browse/PIG-5246?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=16051474#comment-16051474
 ] 

liyunzhang_intel commented on PIG-5246:
---

[~nkollar]:
  bq. For Spark 2.x do we have to add all jar under $SPARK_HOME/jars?
some guy suggested to to add all jar under $SPARK_HOME/jars in Hive on 
Spark([HIVE-15302|https://issues.apache.org/jira/browse/HIVE-15302]), It seems 
this is not accepted by [~vanzin]. But in [Hive 
wiki|https://cwiki.apache.org/confluence/display/Hive/Hive+on+Spark%3A+Getting+Started],
 it is said that we need not append all jars under $SPARK_HOME/jars.
{noformat}
Configuring Hive
To add the Spark dependency to Hive:
Prior to Hive 2.2.0, link the spark-assembly jar to HIVE_HOME/lib.
Since Hive 2.2.0, Hive on Spark runs with Spark 2.0.0 and above, which doesn't 
have an assembly jar.
To run with YARN mode (either yarn-client or yarn-cluster), link the following 
jars to HIVE_HOME/lib.
scala-library
spark-core
spark-network-common
To run with LOCAL mode (for debugging only), link the following jars in 
addition to those above to HIVE_HOME/lib.
chill-java  chill  jackson-module-paranamer  jackson-module-scala  
jersey-container-servlet-core
jersey-server  json4s-ast  kryo-shaded  minlog  scala-xml  spark-launcher
spark-network-shuffle  spark-unsafe  xbean-asm5-shaded
{noformat}

 I don't know whether there is performance influence if we append all jar under 
$SPARK_HOME/jars to the pig classpath.
bq.Could we avoid creating temp files? Instead of creating spark.version, would 
something like this work?
yes, this works, thanks for suggestion.

> Modify bin/pig about SPARK_HOME, SPARK_ASSEMBLY_JAR after upgrading spark to 2
> --
>
> Key: PIG-5246
> URL: https://issues.apache.org/jira/browse/PIG-5246
> Project: Pig
>  Issue Type: Bug
>Reporter: liyunzhang_intel
>Assignee: liyunzhang_intel
> Attachments: HBase9498.patch, PIG-5246.1.patch, PIG-5246_2.patch, 
> PIG-5246.patch
>
>
> in bin/pig.
> we copy assembly jar to pig's classpath in spark1.6.
> {code}
> # For spark mode:
> # Please specify SPARK_HOME first so that we can locate 
> $SPARK_HOME/lib/spark-assembly*.jar,
> # we will add spark-assembly*.jar to the classpath.
> if [ "$isSparkMode"  == "true" ]; then
> if [ -z "$SPARK_HOME" ]; then
>echo "Error: SPARK_HOME is not set!"
>exit 1
> fi
> # Please specify SPARK_JAR which is the hdfs path of spark-assembly*.jar 
> to allow YARN to cache spark-assembly*.jar on nodes so that it doesn't need 
> to be distributed each time an application runs.
> if [ -z "$SPARK_JAR" ]; then
>echo "Error: SPARK_JAR is not set, SPARK_JAR stands for the hdfs 
> location of spark-assembly*.jar. This allows YARN to cache 
> spark-assembly*.jar on nodes so that it doesn't need to be distributed each 
> time an application runs."
>exit 1
> fi
> if [ -n "$SPARK_HOME" ]; then
> echo "Using Spark Home: " ${SPARK_HOME}
> SPARK_ASSEMBLY_JAR=`ls ${SPARK_HOME}/lib/spark-assembly*`
> CLASSPATH=${CLASSPATH}:$SPARK_ASSEMBLY_JAR
> fi
> fi
> {code}
> after upgrade to spark2.0, we may modify it



--
This message was sent by Atlassian JIRA
(v6.4.14#64029)


[jira] Subscription: PIG patch available

2017-06-16 Thread jira
Issue Subscription
Filter: PIG patch available (31 issues)

Subscriber: pigdaily

Key Summary
PIG-5246Modify bin/pig about SPARK_HOME, SPARK_ASSEMBLY_JAR after upgrading 
spark to 2
https://issues.apache.org/jira/browse/PIG-5246
PIG-5160SchemaTupleFrontend.java is not thread safe, cause PigServer thrown 
NPE in multithread env
https://issues.apache.org/jira/browse/PIG-5160
PIG-5157Upgrade to Spark 2.0
https://issues.apache.org/jira/browse/PIG-5157
PIG-5115Builtin AvroStorage generates incorrect avro schema when the same 
pig field name appears in the alias
https://issues.apache.org/jira/browse/PIG-5115
PIG-5106Optimize when mapreduce.input.fileinputformat.input.dir.recursive 
set to true
https://issues.apache.org/jira/browse/PIG-5106
PIG-5081Can not run pig on spark source code distribution
https://issues.apache.org/jira/browse/PIG-5081
PIG-5080Support store alias as spark table
https://issues.apache.org/jira/browse/PIG-5080
PIG-5057IndexOutOfBoundsException when pig reducer processOnePackageOutput
https://issues.apache.org/jira/browse/PIG-5057
PIG-5029Optimize sort case when data is skewed
https://issues.apache.org/jira/browse/PIG-5029
PIG-4926Modify the content of start.xml for spark mode
https://issues.apache.org/jira/browse/PIG-4926
PIG-4913Reduce jython function initiation during compilation
https://issues.apache.org/jira/browse/PIG-4913
PIG-4849pig on tez will cause tez-ui to crash,because the content from 
timeline server is too long. 
https://issues.apache.org/jira/browse/PIG-4849
PIG-4750REPLACE_MULTI should compile Pattern once and reuse it
https://issues.apache.org/jira/browse/PIG-4750
PIG-4684Exception should be changed to warning when job diagnostics cannot 
be fetched
https://issues.apache.org/jira/browse/PIG-4684
PIG-4656Improve String serialization and comparator performance in 
BinInterSedes
https://issues.apache.org/jira/browse/PIG-4656
PIG-4598Allow user defined plan optimizer rules
https://issues.apache.org/jira/browse/PIG-4598
PIG-4551Partition filter is not pushed down in case of SPLIT
https://issues.apache.org/jira/browse/PIG-4551
PIG-4539New PigUnit
https://issues.apache.org/jira/browse/PIG-4539
PIG-4515org.apache.pig.builtin.Distinct throws ClassCastException
https://issues.apache.org/jira/browse/PIG-4515
PIG-4323PackageConverter hanging in Spark
https://issues.apache.org/jira/browse/PIG-4323
PIG-4313StackOverflowError in LIMIT operation on Spark
https://issues.apache.org/jira/browse/PIG-4313
PIG-4251Pig on Storm
https://issues.apache.org/jira/browse/PIG-4251
PIG-4002Disable combiner when map-side aggregation is used
https://issues.apache.org/jira/browse/PIG-4002
PIG-3952PigStorage accepts '-tagSplit' to return full split information
https://issues.apache.org/jira/browse/PIG-3952
PIG-3911Define unique fields with @OutputSchema
https://issues.apache.org/jira/browse/PIG-3911
PIG-3877Getting Geo Latitude/Longitude from Address Lines
https://issues.apache.org/jira/browse/PIG-3877
PIG-3873Geo distance calculation using Haversine
https://issues.apache.org/jira/browse/PIG-3873
PIG-3864ToDate(userstring, format, timezone) computes DateTime with strange 
handling of Daylight Saving Time with location based timezones
https://issues.apache.org/jira/browse/PIG-3864
PIG-3668COR built-in function when atleast one of the coefficient values is 
NaN
https://issues.apache.org/jira/browse/PIG-3668
PIG-3587add functionality for rolling over dates
https://issues.apache.org/jira/browse/PIG-3587
PIG-1804Alow Jython function to implement Algebraic and/or Accumulator 
interfaces
https://issues.apache.org/jira/browse/PIG-1804

You may edit this subscription at:
https://issues.apache.org/jira/secure/FilterSubscription!default.jspa?subId=16328=12322384