[jira] [Commented] (HIVE-15580) Replace Spark's groupByKey operator with something with bounded memory

2017-01-19 Thread Hive QA (JIRA)

[ 
https://issues.apache.org/jira/browse/HIVE-15580?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=15829567#comment-15829567
 ] 

Hive QA commented on HIVE-15580:




Here are the results of testing the latest attachment:
https://issues.apache.org/jira/secure/attachment/12848236/HIVE-15580.4.patch

{color:green}SUCCESS:{color} +1 due to 1 test(s) being added or modified.

{color:red}ERROR:{color} -1 due to 83 failed/errored test(s), 10946 tests 
executed
*Failed tests:*
{noformat}
TestDerbyConnector - did not produce a TEST-*.xml file (likely timed out) 
(batchId=234)
org.apache.hadoop.hive.cli.TestAccumuloCliDriver.testCliDriver[accumulo_joins] 
(batchId=218)
org.apache.hadoop.hive.cli.TestAccumuloCliDriver.testCliDriver[accumulo_predicate_pushdown]
 (batchId=218)
org.apache.hadoop.hive.cli.TestAccumuloCliDriver.testCliDriver[accumulo_single_sourced_multi_insert]
 (batchId=218)
org.apache.hadoop.hive.cli.TestBlobstoreCliDriver.testCliDriver[ctas] 
(batchId=230)
org.apache.hadoop.hive.cli.TestBlobstoreCliDriver.testCliDriver[insert_into_dynamic_partitions]
 (batchId=230)
org.apache.hadoop.hive.cli.TestBlobstoreCliDriver.testCliDriver[insert_into_table]
 (batchId=230)
org.apache.hadoop.hive.cli.TestBlobstoreCliDriver.testCliDriver[insert_overwrite_directory]
 (batchId=230)
org.apache.hadoop.hive.cli.TestBlobstoreCliDriver.testCliDriver[insert_overwrite_dynamic_partitions]
 (batchId=230)
org.apache.hadoop.hive.cli.TestBlobstoreCliDriver.testCliDriver[insert_overwrite_table]
 (batchId=230)
org.apache.hadoop.hive.cli.TestBlobstoreCliDriver.testCliDriver[write_final_output_blobstore]
 (batchId=230)
org.apache.hadoop.hive.cli.TestCliDriver.testCliDriver[acid_table_stats] 
(batchId=48)
org.apache.hadoop.hive.cli.TestCliDriver.testCliDriver[analyze_tbl_part] 
(batchId=44)
org.apache.hadoop.hive.cli.TestCliDriver.testCliDriver[input19] (batchId=79)
org.apache.hadoop.hive.cli.TestCliDriver.testCliDriver[insert_overwrite_directory]
 (batchId=25)
org.apache.hadoop.hive.cli.TestCliDriver.testCliDriver[sample5] (batchId=52)
org.apache.hadoop.hive.cli.TestCliDriver.testCliDriver[serde_opencsv] 
(batchId=68)
org.apache.hadoop.hive.cli.TestCliDriver.testCliDriver[specialChar] (batchId=22)
org.apache.hadoop.hive.cli.TestCliDriver.testCliDriver[vectorized_math_funcs] 
(batchId=19)
org.apache.hadoop.hive.cli.TestContribCliDriver.testCliDriver[dboutput] 
(batchId=221)
org.apache.hadoop.hive.cli.TestContribCliDriver.testCliDriver[fileformat_base64]
 (batchId=221)
org.apache.hadoop.hive.cli.TestContribCliDriver.testCliDriver[udf_row_sequence] 
(batchId=221)
org.apache.hadoop.hive.cli.TestContribCliDriver.testCliDriver[url_hook] 
(batchId=221)
org.apache.hadoop.hive.cli.TestContribNegativeCliDriver.testCliDriver[case_with_row_sequence]
 (batchId=224)
org.apache.hadoop.hive.cli.TestContribNegativeCliDriver.testCliDriver[invalid_row_sequence]
 (batchId=224)
org.apache.hadoop.hive.cli.TestContribNegativeCliDriver.testCliDriver[serde_regex]
 (batchId=224)
org.apache.hadoop.hive.cli.TestEncryptedHDFSCliDriver.testCliDriver[encryption_insert_partition_dynamic]
 (batchId=158)
org.apache.hadoop.hive.cli.TestEncryptedHDFSCliDriver.testCliDriver[encryption_insert_partition_static]
 (batchId=156)
org.apache.hadoop.hive.cli.TestEncryptedHDFSCliDriver.testCliDriver[encryption_insert_values]
 (batchId=157)
org.apache.hadoop.hive.cli.TestEncryptedHDFSCliDriver.testCliDriver[encryption_join_unencrypted_tbl]
 (batchId=159)
org.apache.hadoop.hive.cli.TestEncryptedHDFSCliDriver.testCliDriver[encryption_join_with_different_encryption_keys]
 (batchId=159)
org.apache.hadoop.hive.cli.TestEncryptedHDFSCliDriver.testCliDriver[encryption_load_data_to_encrypted_tables]
 (batchId=157)
org.apache.hadoop.hive.cli.TestEncryptedHDFSCliDriver.testCliDriver[encryption_move_tbl]
 (batchId=157)
org.apache.hadoop.hive.cli.TestEncryptedHDFSCliDriver.testCliDriver[encryption_select_read_only_encrypted_tbl]
 (batchId=159)
org.apache.hadoop.hive.cli.TestEncryptedHDFSCliDriver.testCliDriver[encryption_select_read_only_unencrypted_tbl]
 (batchId=159)
org.apache.hadoop.hive.cli.TestEncryptedHDFSCliDriver.testCliDriver[encryption_unencrypted_nonhdfs_external_tables]
 (batchId=157)
org.apache.hadoop.hive.cli.TestHBaseNegativeCliDriver.testCliDriver[cascade_dbdrop]
 (batchId=225)
org.apache.hadoop.hive.cli.TestHBaseNegativeCliDriver.testCliDriver[generatehfiles_require_family_path]
 (batchId=225)
org.apache.hadoop.hive.cli.TestMiniLlapCliDriver.testCliDriver[orc_llap_counters]
 (batchId=137)
org.apache.hadoop.hive.cli.TestMiniLlapCliDriver.testCliDriver[orc_ppd_basic] 
(batchId=135)
org.apache.hadoop.hive.cli.TestMiniLlapCliDriver.testCliDriver[orc_ppd_schema_evol_3a]
 (batchId=136)
org.apache.hadoop.hive.cli.TestMiniLlapLocalCliDriver.testCliDriver[escape1] 
(batchId=139)
org.apache.hadoop.hive.cli.TestMiniLlapLocalCliDriver.testCliDriver[escape2] 
(batchId=154)

[jira] [Commented] (HIVE-15580) Replace Spark's groupByKey operator with something with bounded memory

2017-01-18 Thread Hive QA (JIRA)

[ 
https://issues.apache.org/jira/browse/HIVE-15580?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=15829346#comment-15829346
 ] 

Hive QA commented on HIVE-15580:




Here are the results of testing the latest attachment:
https://issues.apache.org/jira/secure/attachment/12848103/HIVE-15580.3.patch

{color:red}ERROR:{color} -1 due to build exiting with an error

Test results: https://builds.apache.org/job/PreCommit-HIVE-Build/3028/testReport
Console output: https://builds.apache.org/job/PreCommit-HIVE-Build/3028/console
Test logs: http://104.198.109.242/logs/PreCommit-HIVE-Build-3028/

Messages:
{noformat}
Executing org.apache.hive.ptest.execution.TestCheckPhase
Executing org.apache.hive.ptest.execution.PrepPhase
Tests exited with: NonZeroExitCodeException
Command 'bash /data/hiveptest/working/scratch/source-prep.sh' failed with exit 
status 1 and output '+ date '+%Y-%m-%d %T.%3N'
2017-01-19 05:35:19.448
+ [[ -n /usr/lib/jvm/java-8-openjdk-amd64 ]]
+ export JAVA_HOME=/usr/lib/jvm/java-8-openjdk-amd64
+ JAVA_HOME=/usr/lib/jvm/java-8-openjdk-amd64
+ export 
PATH=/usr/lib/jvm/java-8-openjdk-amd64/bin/:/usr/local/bin:/usr/bin:/bin:/usr/local/games:/usr/games
+ 
PATH=/usr/lib/jvm/java-8-openjdk-amd64/bin/:/usr/local/bin:/usr/bin:/bin:/usr/local/games:/usr/games
+ export 'ANT_OPTS=-Xmx1g -XX:MaxPermSize=256m '
+ ANT_OPTS='-Xmx1g -XX:MaxPermSize=256m '
+ export 'MAVEN_OPTS=-Xmx1g '
+ MAVEN_OPTS='-Xmx1g '
+ cd /data/hiveptest/working/
+ tee /data/hiveptest/logs/PreCommit-HIVE-Build-3028/source-prep.txt
+ [[ false == \t\r\u\e ]]
+ mkdir -p maven ivy
+ [[ git = \s\v\n ]]
+ [[ git = \g\i\t ]]
+ [[ -z master ]]
+ [[ -d apache-github-source-source ]]
+ [[ ! -d apache-github-source-source/.git ]]
+ [[ ! -d apache-github-source-source ]]
+ date '+%Y-%m-%d %T.%3N'
2017-01-19 05:35:19.451
+ cd apache-github-source-source
+ git fetch origin
+ git reset --hard HEAD
HEAD is now at ef33237  IVE-15297: Hive should not split semicolon within 
quoted string literals (Pengcheng Xiong, reviewed by Ashutosh Chauhan) 
(addendum I)
+ git clean -f -d
+ git checkout master
Already on 'master'
Your branch is up-to-date with 'origin/master'.
+ git reset --hard origin/master
HEAD is now at ef33237  IVE-15297: Hive should not split semicolon within 
quoted string literals (Pengcheng Xiong, reviewed by Ashutosh Chauhan) 
(addendum I)
+ git merge --ff-only origin/master
Already up-to-date.
+ date '+%Y-%m-%d %T.%3N'
2017-01-19 05:35:20.803
+ patchCommandPath=/data/hiveptest/working/scratch/smart-apply-patch.sh
+ patchFilePath=/data/hiveptest/working/scratch/build.patch
+ [[ -f /data/hiveptest/working/scratch/build.patch ]]
+ chmod +x /data/hiveptest/working/scratch/smart-apply-patch.sh
+ /data/hiveptest/working/scratch/smart-apply-patch.sh 
/data/hiveptest/working/scratch/build.patch
error: patch failed: 
ql/src/test/results/clientpositive/spark/union_top_level.q.out:324
error: ql/src/test/results/clientpositive/spark/union_top_level.q.out: patch 
does not apply
The patch does not appear to apply with p0, p1, or p2
+ exit 1
'
{noformat}

This message is automatically generated.

ATTACHMENT ID: 12848103 - PreCommit-HIVE-Build

> Replace Spark's groupByKey operator with something with bounded memory
> --
>
> Key: HIVE-15580
> URL: https://issues.apache.org/jira/browse/HIVE-15580
> Project: Hive
>  Issue Type: Improvement
>  Components: Spark
>Reporter: Xuefu Zhang
>Assignee: Xuefu Zhang
> Attachments: HIVE-15580.1.patch, HIVE-15580.1.patch, 
> HIVE-15580.2.patch, HIVE-15580.2.patch, HIVE-15580.3.patch, HIVE-15580.patch
>
>




--
This message was sent by Atlassian JIRA
(v6.3.4#6332)


[jira] [Commented] (HIVE-15580) Replace Spark's groupByKey operator with something with bounded memory

2017-01-18 Thread Dapeng Sun (JIRA)

[ 
https://issues.apache.org/jira/browse/HIVE-15580?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=15829218#comment-15829218
 ] 

Dapeng Sun commented on HIVE-15580:
---

Thank [~xuefuz] for the suggestion, currently the heap size is 290G for each 
executor, I will try to do more turning on it.

> Replace Spark's groupByKey operator with something with bounded memory
> --
>
> Key: HIVE-15580
> URL: https://issues.apache.org/jira/browse/HIVE-15580
> Project: Hive
>  Issue Type: Improvement
>  Components: Spark
>Reporter: Xuefu Zhang
>Assignee: Xuefu Zhang
> Attachments: HIVE-15580.1.patch, HIVE-15580.1.patch, 
> HIVE-15580.2.patch, HIVE-15580.2.patch, HIVE-15580.3.patch, HIVE-15580.patch
>
>




--
This message was sent by Atlassian JIRA
(v6.3.4#6332)


[jira] [Commented] (HIVE-15580) Replace Spark's groupByKey operator with something with bounded memory

2017-01-18 Thread Xuefu Zhang (JIRA)

[ 
https://issues.apache.org/jira/browse/HIVE-15580?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=15829203#comment-15829203
 ] 

Xuefu Zhang commented on HIVE-15580:


[~dapengsun], for the OOM error you get, you can probably increase the executor 
heap size to overcome it.  PartitionedPairBuffer uses a buffer up to 
Int.MaxValue / 2, so the memory usage is big but bounded.

> Replace Spark's groupByKey operator with something with bounded memory
> --
>
> Key: HIVE-15580
> URL: https://issues.apache.org/jira/browse/HIVE-15580
> Project: Hive
>  Issue Type: Improvement
>  Components: Spark
>Reporter: Xuefu Zhang
>Assignee: Xuefu Zhang
> Attachments: HIVE-15580.1.patch, HIVE-15580.1.patch, 
> HIVE-15580.2.patch, HIVE-15580.2.patch, HIVE-15580.3.patch, HIVE-15580.patch
>
>




--
This message was sent by Atlassian JIRA
(v6.3.4#6332)


[jira] [Commented] (HIVE-15580) Replace Spark's groupByKey operator with something with bounded memory

2017-01-18 Thread Dapeng Sun (JIRA)

[ 
https://issues.apache.org/jira/browse/HIVE-15580?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=15829154#comment-15829154
 ] 

Dapeng Sun commented on HIVE-15580:
---

Thank [~xuefuz], [~csun] and [~Ferd], we are running a 100TB test case about 
data skew on 50 nodes(TPC-xBB q21), before the patch, spark tasks are failed 
with following error:
{noformat}
java.lang.OutOfMemoryError: Requested array size exceeds VM limit
 at java.util.Arrays.copyOf(Arrays.java:3181)
 at java.util.ArrayList.grow(ArrayList.java:261)
 at java.util.ArrayList.ensureExplicitCapacity(ArrayList.java:235)
 at java.util.ArrayList.ensureCapacityInternal(ArrayList.java:227)
 at java.util.ArrayList.add(ArrayList.java:458)
 at 
org.apache.hadoop.hive.ql.exec.spark.SortByShuffler$ShuffleFunction$1.next(SortByShuffler.java:100)
 at 
org.apache.hadoop.hive.ql.exec.spark.SortByShuffler$ShuffleFunction$1.next(SortByShuffler.java:75)
 at 
org.apache.hadoop.hive.ql.exec.spark.HiveBaseFunctionResultList$ResultIterator.hasNext(HiveBaseFunctionResultList.java:95)
 at 
scala.collection.convert.Wrappers$JIteratorWrapper.hasNext(Wrappers.scala:41)
 at 
org.apache.spark.util.collection.ExternalSorter.insertAll(ExternalSorter.scala:200)
 at 
org.apache.spark.shuffle.sort.SortShuffleWriter.write(SortShuffleWriter.scala:64)
 at 
org.apache.spark.scheduler.ShuffleMapTask.runTask(ShuffleMapTask.scala:73)
 at 
org.apache.spark.scheduler.ShuffleMapTask.runTask(ShuffleMapTask.scala:41)
 at org.apache.spark.scheduler.Task.run(Task.scala:89)
 at 
org.apache.spark.executor.Executor$TaskRunner.run(Executor.scala:214)
 at 
java.util.concurrent.ThreadPoolExecutor.runWorker(ThreadPoolExecutor.java:1142)
 at 
java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:617)
 at java.lang.Thread.run(Thread.java:745)
{noformat}
after apply the patch, the arraylist are fixed, but PartitionedPairBuffer also 
cause OOM, here are the task failed exception:
{noformat}
java.lang.OutOfMemoryError: Requested array size exceeds VM limit
at 
org.apache.spark.util.collection.PartitionedPairBuffer.growArray(PartitionedPairBuffer.scala:67)
at 
org.apache.spark.util.collection.PartitionedPairBuffer.insert(PartitionedPairBuffer.scala:48)
at 
org.apache.spark.util.collection.ExternalSorter.insertAll(ExternalSorter.scala:203)
at 
org.apache.spark.shuffle.BlockStoreShuffleReader.read(BlockStoreShuffleReader.scala:111)
at org.apache.spark.rdd.ShuffledRDD.compute(ShuffledRDD.scala:98)
at org.apache.spark.rdd.RDD.computeOrReadCheckpoint(RDD.scala:306)
at org.apache.spark.rdd.RDD.iterator(RDD.scala:270)
at 
org.apache.spark.rdd.MapPartitionsRDD.compute(MapPartitionsRDD.scala:38)
at org.apache.spark.rdd.RDD.computeOrReadCheckpoint(RDD.scala:306)
at org.apache.spark.rdd.RDD.iterator(RDD.scala:270)
at 
org.apache.spark.rdd.MapPartitionsRDD.compute(MapPartitionsRDD.scala:38)
at org.apache.spark.rdd.RDD.computeOrReadCheckpoint(RDD.scala:306)
at org.apache.spark.rdd.RDD.iterator(RDD.scala:270)
at 
org.apache.spark.scheduler.ShuffleMapTask.runTask(ShuffleMapTask.scala:73)
at 
org.apache.spark.scheduler.ShuffleMapTask.runTask(ShuffleMapTask.scala:41)
at org.apache.spark.scheduler.Task.run(Task.scala:89)
at org.apache.spark.executor.Executor$TaskRunner.run(Executor.scala:214)
at 
java.util.concurrent.ThreadPoolExecutor.runWorker(ThreadPoolExecutor.java:1142)
at 
java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:617)
at java.lang.Thread.run(Thread.java:745)
{noformat}

> Replace Spark's groupByKey operator with something with bounded memory
> --
>
> Key: HIVE-15580
> URL: https://issues.apache.org/jira/browse/HIVE-15580
> Project: Hive
>  Issue Type: Improvement
>  Components: Spark
>Reporter: Xuefu Zhang
>Assignee: Xuefu Zhang
> Attachments: HIVE-15580.1.patch, HIVE-15580.1.patch, 
> HIVE-15580.2.patch, HIVE-15580.2.patch, HIVE-15580.3.patch, HIVE-15580.patch
>
>




--
This message was sent by Atlassian JIRA
(v6.3.4#6332)


[jira] [Commented] (HIVE-15580) Replace Spark's groupByKey operator with something with bounded memory

2017-01-18 Thread Ferdinand Xu (JIRA)

[ 
https://issues.apache.org/jira/browse/HIVE-15580?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=15829099#comment-15829099
 ] 

Ferdinand Xu commented on HIVE-15580:
-

[~xuefuz], patch in HIVE-15527 solved the issue and we're trying the patch in 
HIVE-15580. Will keep you post.

> Replace Spark's groupByKey operator with something with bounded memory
> --
>
> Key: HIVE-15580
> URL: https://issues.apache.org/jira/browse/HIVE-15580
> Project: Hive
>  Issue Type: Improvement
>  Components: Spark
>Reporter: Xuefu Zhang
>Assignee: Xuefu Zhang
> Attachments: HIVE-15580.1.patch, HIVE-15580.1.patch, 
> HIVE-15580.2.patch, HIVE-15580.2.patch, HIVE-15580.3.patch, HIVE-15580.patch
>
>




--
This message was sent by Atlassian JIRA
(v6.3.4#6332)


[jira] [Commented] (HIVE-15580) Replace Spark's groupByKey operator with something with bounded memory

2017-01-18 Thread Rui Li (JIRA)

[ 
https://issues.apache.org/jira/browse/HIVE-15580?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=15828249#comment-15828249
 ] 

Rui Li commented on HIVE-15580:
---

Hmm Spark does make it clear groupByKey can't handle large keys:
https://github.com/apache/spark/pull/3936

> Replace Spark's groupByKey operator with something with bounded memory
> --
>
> Key: HIVE-15580
> URL: https://issues.apache.org/jira/browse/HIVE-15580
> Project: Hive
>  Issue Type: Improvement
>  Components: Spark
>Reporter: Xuefu Zhang
>Assignee: Xuefu Zhang
> Attachments: HIVE-15580.1.patch, HIVE-15580.1.patch, 
> HIVE-15580.2.patch, HIVE-15580.patch
>
>




--
This message was sent by Atlassian JIRA
(v6.3.4#6332)


[jira] [Commented] (HIVE-15580) Replace Spark's groupByKey operator with something with bounded memory

2017-01-18 Thread Rui Li (JIRA)

[ 
https://issues.apache.org/jira/browse/HIVE-15580?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=15828183#comment-15828183
 ] 

Rui Li commented on HIVE-15580:
---

[~xuefuz], thanks for your explanations. It makes sense. So in general, the 
input to reducers doesn't have to be  right? I think one 
drawback of this is we have to shuffle more data over network. And I'm curious, 
will this happen to MR too, i.e. does MR also spills at key-group boundary?

> Replace Spark's groupByKey operator with something with bounded memory
> --
>
> Key: HIVE-15580
> URL: https://issues.apache.org/jira/browse/HIVE-15580
> Project: Hive
>  Issue Type: Improvement
>  Components: Spark
>Reporter: Xuefu Zhang
>Assignee: Xuefu Zhang
> Attachments: HIVE-15580.1.patch, HIVE-15580.1.patch, 
> HIVE-15580.2.patch, HIVE-15580.patch
>
>




--
This message was sent by Atlassian JIRA
(v6.3.4#6332)


[jira] [Commented] (HIVE-15580) Replace Spark's groupByKey operator with something with bounded memory

2017-01-18 Thread Xuefu Zhang (JIRA)

[ 
https://issues.apache.org/jira/browse/HIVE-15580?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=15828155#comment-15828155
 ] 

Xuefu Zhang commented on HIVE-15580:


Hi [~lirui], your understanding is correct.

And yes, groupByKey uses unbounded memory. While Spark can split to disk for 
this, but the spilling has to be at the key/group boundary. For a big key 
group, Spark can still run out of memory.

Ref: 
http://apache-spark-user-list.1001560.n3.nabble.com/Understanding-RDD-GroupBy-OutOfMemory-Exceptions-td11427.html

> Replace Spark's groupByKey operator with something with bounded memory
> --
>
> Key: HIVE-15580
> URL: https://issues.apache.org/jira/browse/HIVE-15580
> Project: Hive
>  Issue Type: Improvement
>  Components: Spark
>Reporter: Xuefu Zhang
>Assignee: Xuefu Zhang
> Attachments: HIVE-15580.1.patch, HIVE-15580.1.patch, 
> HIVE-15580.2.patch, HIVE-15580.patch
>
>




--
This message was sent by Atlassian JIRA
(v6.3.4#6332)


[jira] [Commented] (HIVE-15580) Replace Spark's groupByKey operator with something with bounded memory

2017-01-18 Thread Rui Li (JIRA)

[ 
https://issues.apache.org/jira/browse/HIVE-15580?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=15828143#comment-15828143
 ] 

Rui Li commented on HIVE-15580:
---

Hi [~xuefuz], I'd like to check my understanding too. Before the patch, we have 
3 kinds of shuffle: groupByKey, sortByKey and 
repartitionAndSortWithinPartitions. For the last two, we do the grouping 
ourselves (because reducer expects ). This grouping uses 
unbounded memory, which is the root cause of HIVE-15527.

With the patch, we'll replace groupByKey with 
repartitionAndSortWithinPartitions. And we don't have to do the grouping 
ourselves because GBY operator will do that for us. Is this correct?
BTW, is there any doc indicating Spark's groupByKey uses unbounded memory? I 
think Spark can spill the shuffled data to disk if it's too large.

> Replace Spark's groupByKey operator with something with bounded memory
> --
>
> Key: HIVE-15580
> URL: https://issues.apache.org/jira/browse/HIVE-15580
> Project: Hive
>  Issue Type: Improvement
>  Components: Spark
>Reporter: Xuefu Zhang
>Assignee: Xuefu Zhang
> Attachments: HIVE-15580.1.patch, HIVE-15580.1.patch, 
> HIVE-15580.2.patch, HIVE-15580.patch
>
>




--
This message was sent by Atlassian JIRA
(v6.3.4#6332)


[jira] [Commented] (HIVE-15580) Replace Spark's groupByKey operator with something with bounded memory

2017-01-17 Thread Xuefu Zhang (JIRA)

[ 
https://issues.apache.org/jira/browse/HIVE-15580?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=15827470#comment-15827470
 ] 

Xuefu Zhang commented on HIVE-15580:


[~Ferd], Functionally, I don't see anything bad because groupByKey was used in 
Hive for aggregation. Hive's groupby operator is able to process one row at a 
time with this patch. Performance wise, I'm not sure if this will improve or 
degrade. That depends on the performance difference of groupByKey() + value 
iterator and repartitionAndSortWithinPartitions() + dummy value iterator. It 
would be great if you guys can find out.

The obvious benefit of this change is that Hive on Spark overcomes the 
unbounded memory usage of groupByKey(). The patch also solves the problem in 
HIVE-15527.

Please note that this patch is WIP. We will improve it, for example getting 
ride of the dummy value iterator created per row.

I manually ran all spark tests with this patch, and there was only one test 
failure which needs investigation.

> Replace Spark's groupByKey operator with something with bounded memory
> --
>
> Key: HIVE-15580
> URL: https://issues.apache.org/jira/browse/HIVE-15580
> Project: Hive
>  Issue Type: Improvement
>  Components: Spark
>Reporter: Xuefu Zhang
>Assignee: Xuefu Zhang
> Attachments: HIVE-15580.1.patch, HIVE-15580.1.patch, 
> HIVE-15580.2.patch, HIVE-15580.patch
>
>




--
This message was sent by Atlassian JIRA
(v6.3.4#6332)


[jira] [Commented] (HIVE-15580) Replace Spark's groupByKey operator with something with bounded memory

2017-01-17 Thread Ferdinand Xu (JIRA)

[ 
https://issues.apache.org/jira/browse/HIVE-15580?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=15827399#comment-15827399
 ] 

Ferdinand Xu commented on HIVE-15580:
-

Hi [~xuefuz], the main change is about replacing *groupByKey* with 
*repartitionAndSortWithinPartitions*. Just help me to have a better understand. 
Before this patch:
e.g. GroupByShuffle will lead to the following result:
K1 -> iterator of {V11,V12,V13...}
K2 -> iterator of {V21,V22,V23...}
...

With this patch:
K1 -> V11
K1 -> V12
K1 -> V13
...
K2 -> V21
...

And we process them one by one without fetching the value from iterator. If so, 
is there any side effect by changing this?


> Replace Spark's groupByKey operator with something with bounded memory
> --
>
> Key: HIVE-15580
> URL: https://issues.apache.org/jira/browse/HIVE-15580
> Project: Hive
>  Issue Type: Improvement
>  Components: Spark
>Reporter: Xuefu Zhang
>Assignee: Xuefu Zhang
> Attachments: HIVE-15580.1.patch, HIVE-15580.1.patch, 
> HIVE-15580.2.patch, HIVE-15580.patch
>
>




--
This message was sent by Atlassian JIRA
(v6.3.4#6332)


[jira] [Commented] (HIVE-15580) Replace Spark's groupByKey operator with something with bounded memory

2017-01-11 Thread Hive QA (JIRA)

[ 
https://issues.apache.org/jira/browse/HIVE-15580?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=15819638#comment-15819638
 ] 

Hive QA commented on HIVE-15580:




Here are the results of testing the latest attachment:
https://issues.apache.org/jira/secure/attachment/12847067/HIVE-15580.1.patch

{color:red}ERROR:{color} -1 due to no test(s) being added or modified.

{color:red}ERROR:{color} -1 due to 15 failed/errored test(s), 10941 tests 
executed
*Failed tests:*
{noformat}
TestDerbyConnector - did not produce a TEST-*.xml file (likely timed out) 
(batchId=234)
org.apache.hadoop.hive.cli.TestMiniLlapLocalCliDriver.testCliDriver[schema_evol_text_vec_part]
 (batchId=148)
org.apache.hadoop.hive.cli.TestMiniSparkOnYarnCliDriver.testCliDriver[vector_outer_join5]
 (batchId=161)
org.apache.hadoop.hive.cli.TestSparkCliDriver.testCliDriver[lateral_view_explode2]
 (batchId=130)
org.apache.hadoop.hive.cli.TestSparkCliDriver.testCliDriver[union_remove_25] 
(batchId=132)
org.apache.hadoop.hive.cli.TestSparkCliDriver.testCliDriver[union_top_level] 
(batchId=120)
org.apache.hadoop.hive.llap.cache.TestLowLevelCacheImpl.testCacheMetrics 
(batchId=282)
org.apache.hadoop.hive.llap.cache.TestLowLevelCacheImpl.testGetPut (batchId=282)
org.apache.hadoop.hive.llap.cache.TestLowLevelCacheImpl.testMTTWithCleanup 
(batchId=282)
org.apache.hadoop.hive.llap.cache.TestLowLevelCacheImpl.testMultiMatch 
(batchId=282)
org.apache.hadoop.hive.llap.cache.TestLowLevelCacheImpl.testMultiMatchNonGranular
 (batchId=282)
org.apache.hadoop.hive.llap.cache.TestLowLevelCacheImpl.testStaleValueGet 
(batchId=282)
org.apache.hadoop.hive.llap.cache.TestLowLevelCacheImpl.testStaleValueReplace 
(batchId=282)
org.apache.hadoop.hive.ql.security.authorization.plugin.TestHiveAuthorizerCheckInvocation.org.apache.hadoop.hive.ql.security.authorization.plugin.TestHiveAuthorizerCheckInvocation
 (batchId=208)
org.apache.hadoop.hive.ql.security.authorization.plugin.TestHiveAuthorizerShowFilters.org.apache.hadoop.hive.ql.security.authorization.plugin.TestHiveAuthorizerShowFilters
 (batchId=208)
{noformat}

Test results: https://builds.apache.org/job/PreCommit-HIVE-Build/2894/testReport
Console output: https://builds.apache.org/job/PreCommit-HIVE-Build/2894/console
Test logs: http://104.198.109.242/logs/PreCommit-HIVE-Build-2894/

Messages:
{noformat}
Executing org.apache.hive.ptest.execution.TestCheckPhase
Executing org.apache.hive.ptest.execution.PrepPhase
Executing org.apache.hive.ptest.execution.ExecutionPhase
Executing org.apache.hive.ptest.execution.ReportingPhase
Tests exited with: TestsFailedException: 15 tests failed
{noformat}

This message is automatically generated.

ATTACHMENT ID: 12847067 - PreCommit-HIVE-Build

> Replace Spark's groupByKey operator with something with bounded memory
> --
>
> Key: HIVE-15580
> URL: https://issues.apache.org/jira/browse/HIVE-15580
> Project: Hive
>  Issue Type: Improvement
>  Components: Spark
>Reporter: Xuefu Zhang
>Assignee: Xuefu Zhang
> Attachments: HIVE-15580.1.patch, HIVE-15580.1.patch, HIVE-15580.patch
>
>




--
This message was sent by Atlassian JIRA
(v6.3.4#6332)


[jira] [Commented] (HIVE-15580) Replace Spark's groupByKey operator with something with bounded memory

2017-01-11 Thread Hive QA (JIRA)

[ 
https://issues.apache.org/jira/browse/HIVE-15580?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=15819356#comment-15819356
 ] 

Hive QA commented on HIVE-15580:




Here are the results of testing the latest attachment:
https://issues.apache.org/jira/secure/attachment/12847067/HIVE-15580.1.patch

{color:red}ERROR:{color} -1 due to no test(s) being added or modified.

{color:red}ERROR:{color} -1 due to 14 failed/errored test(s), 10954 tests 
executed
*Failed tests:*
{noformat}
TestDerbyConnector - did not produce a TEST-*.xml file (likely timed out) 
(batchId=233)
org.apache.hadoop.hive.cli.TestMiniLlapLocalCliDriver.testCliDriver[schema_evol_text_vec_part]
 (batchId=148)
org.apache.hadoop.hive.cli.TestMiniLlapLocalCliDriver.testCliDriver[vector_varchar_simple]
 (batchId=151)
org.apache.hadoop.hive.cli.TestMiniSparkOnYarnCliDriver.testCliDriver[vector_outer_join5]
 (batchId=161)
org.apache.hadoop.hive.cli.TestSparkCliDriver.testCliDriver[lateral_view_explode2]
 (batchId=130)
org.apache.hadoop.hive.cli.TestSparkCliDriver.testCliDriver[union_remove_25] 
(batchId=132)
org.apache.hadoop.hive.cli.TestSparkCliDriver.testCliDriver[union_top_level] 
(batchId=120)
org.apache.hadoop.hive.llap.cache.TestLowLevelCacheImpl.testCacheMetrics 
(batchId=281)
org.apache.hadoop.hive.llap.cache.TestLowLevelCacheImpl.testGetPut (batchId=281)
org.apache.hadoop.hive.llap.cache.TestLowLevelCacheImpl.testMTTWithCleanup 
(batchId=281)
org.apache.hadoop.hive.llap.cache.TestLowLevelCacheImpl.testMultiMatch 
(batchId=281)
org.apache.hadoop.hive.llap.cache.TestLowLevelCacheImpl.testMultiMatchNonGranular
 (batchId=281)
org.apache.hadoop.hive.llap.cache.TestLowLevelCacheImpl.testStaleValueGet 
(batchId=281)
org.apache.hadoop.hive.llap.cache.TestLowLevelCacheImpl.testStaleValueReplace 
(batchId=281)
{noformat}

Test results: https://builds.apache.org/job/PreCommit-HIVE-Build/2890/testReport
Console output: https://builds.apache.org/job/PreCommit-HIVE-Build/2890/console
Test logs: http://104.198.109.242/logs/PreCommit-HIVE-Build-2890/

Messages:
{noformat}
Executing org.apache.hive.ptest.execution.TestCheckPhase
Executing org.apache.hive.ptest.execution.PrepPhase
Executing org.apache.hive.ptest.execution.ExecutionPhase
Executing org.apache.hive.ptest.execution.ReportingPhase
Tests exited with: TestsFailedException: 14 tests failed
{noformat}

This message is automatically generated.

ATTACHMENT ID: 12847067 - PreCommit-HIVE-Build

> Replace Spark's groupByKey operator with something with bounded memory
> --
>
> Key: HIVE-15580
> URL: https://issues.apache.org/jira/browse/HIVE-15580
> Project: Hive
>  Issue Type: Improvement
>  Components: Spark
>Reporter: Xuefu Zhang
>Assignee: Xuefu Zhang
> Attachments: HIVE-15580.1.patch, HIVE-15580.1.patch, HIVE-15580.patch
>
>




--
This message was sent by Atlassian JIRA
(v6.3.4#6332)


[jira] [Commented] (HIVE-15580) Replace Spark's groupByKey operator with something with bounded memory

2017-01-11 Thread Hive QA (JIRA)

[ 
https://issues.apache.org/jira/browse/HIVE-15580?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=15818740#comment-15818740
 ] 

Hive QA commented on HIVE-15580:




Here are the results of testing the latest attachment:
https://issues.apache.org/jira/secure/attachment/12846754/HIVE-15580.patch

{color:red}ERROR:{color} -1 due to no test(s) being added or modified.

{color:red}ERROR:{color} -1 due to 56 failed/errored test(s), 10940 tests 
executed
*Failed tests:*
{noformat}
TestDerbyConnector - did not produce a TEST-*.xml file (likely timed out) 
(batchId=233)
org.apache.hadoop.hive.cli.TestCliDriver.testCliDriver[case_sensitivity] 
(batchId=61)
org.apache.hadoop.hive.cli.TestCliDriver.testCliDriver[input_testxpath] 
(batchId=28)
org.apache.hadoop.hive.cli.TestCliDriver.testCliDriver[udf_coalesce] 
(batchId=75)
org.apache.hadoop.hive.cli.TestMiniLlapLocalCliDriver.testCliDriver[schema_evol_text_vec_part]
 (batchId=148)
org.apache.hadoop.hive.cli.TestMiniSparkOnYarnCliDriver.testCliDriver[infer_bucket_sort_map_operators]
 (batchId=160)
org.apache.hadoop.hive.cli.TestMiniSparkOnYarnCliDriver.testCliDriver[infer_bucket_sort_num_buckets]
 (batchId=160)
org.apache.hadoop.hive.cli.TestMiniSparkOnYarnCliDriver.testCliDriver[infer_bucket_sort_reducers_power_two]
 (batchId=159)
org.apache.hadoop.hive.cli.TestMiniSparkOnYarnCliDriver.testCliDriver[vector_outer_join5]
 (batchId=161)
org.apache.hadoop.hive.cli.TestSparkCliDriver.org.apache.hadoop.hive.cli.TestSparkCliDriver
 (batchId=96)
org.apache.hadoop.hive.cli.TestSparkCliDriver.testCliDriver[count] (batchId=105)
org.apache.hadoop.hive.cli.TestSparkCliDriver.testCliDriver[groupby1] 
(batchId=102)
org.apache.hadoop.hive.cli.TestSparkCliDriver.testCliDriver[groupby1_map_nomap] 
(batchId=127)
org.apache.hadoop.hive.cli.TestSparkCliDriver.testCliDriver[groupby1_noskew] 
(batchId=108)
org.apache.hadoop.hive.cli.TestSparkCliDriver.testCliDriver[groupby4] 
(batchId=120)
org.apache.hadoop.hive.cli.TestSparkCliDriver.testCliDriver[groupby4_noskew] 
(batchId=119)
org.apache.hadoop.hive.cli.TestSparkCliDriver.testCliDriver[groupby5] 
(batchId=111)
org.apache.hadoop.hive.cli.TestSparkCliDriver.testCliDriver[groupby5_noskew] 
(batchId=131)
org.apache.hadoop.hive.cli.TestSparkCliDriver.testCliDriver[groupby6] 
(batchId=117)
org.apache.hadoop.hive.cli.TestSparkCliDriver.testCliDriver[groupby6_noskew] 
(batchId=94)
org.apache.hadoop.hive.cli.TestSparkCliDriver.testCliDriver[groupby7] 
(batchId=123)
org.apache.hadoop.hive.cli.TestSparkCliDriver.testCliDriver[groupby7_noskew] 
(batchId=130)
org.apache.hadoop.hive.cli.TestSparkCliDriver.testCliDriver[groupby_grouping_id2]
 (batchId=109)
org.apache.hadoop.hive.cli.TestSparkCliDriver.testCliDriver[groupby_sort_1_23] 
(batchId=128)
org.apache.hadoop.hive.cli.TestSparkCliDriver.testCliDriver[groupby_sort_skew_1_23]
 (batchId=98)
org.apache.hadoop.hive.cli.TestSparkCliDriver.testCliDriver[lateral_view_explode2]
 (batchId=130)
org.apache.hadoop.hive.cli.TestSparkCliDriver.testCliDriver[limit_pushdown] 
(batchId=123)
org.apache.hadoop.hive.cli.TestSparkCliDriver.testCliDriver[mapjoin_distinct] 
(batchId=117)
org.apache.hadoop.hive.cli.TestSparkCliDriver.testCliDriver[mapjoin_mapjoin] 
(batchId=115)
org.apache.hadoop.hive.cli.TestSparkCliDriver.testCliDriver[merge2] (batchId=94)
org.apache.hadoop.hive.cli.TestSparkCliDriver.testCliDriver[udaf_collect_set] 
(batchId=98)
org.apache.hadoop.hive.cli.TestSparkCliDriver.testCliDriver[udf_percentile] 
(batchId=121)
org.apache.hadoop.hive.cli.TestSparkCliDriver.testCliDriver[union15] 
(batchId=132)
org.apache.hadoop.hive.cli.TestSparkCliDriver.testCliDriver[union17] 
(batchId=124)
org.apache.hadoop.hive.cli.TestSparkCliDriver.testCliDriver[union21] 
(batchId=94)
org.apache.hadoop.hive.cli.TestSparkCliDriver.testCliDriver[union24] 
(batchId=119)
org.apache.hadoop.hive.cli.TestSparkCliDriver.testCliDriver[union26] 
(batchId=122)
org.apache.hadoop.hive.cli.TestSparkCliDriver.testCliDriver[union31] 
(batchId=95)
org.apache.hadoop.hive.cli.TestSparkCliDriver.testCliDriver[union_remove_10] 
(batchId=104)
org.apache.hadoop.hive.cli.TestSparkCliDriver.testCliDriver[union_remove_13] 
(batchId=131)
org.apache.hadoop.hive.cli.TestSparkCliDriver.testCliDriver[union_remove_19] 
(batchId=103)
org.apache.hadoop.hive.cli.TestSparkCliDriver.testCliDriver[union_remove_1] 
(batchId=108)
org.apache.hadoop.hive.cli.TestSparkCliDriver.testCliDriver[union_remove_20] 
(batchId=130)
org.apache.hadoop.hive.cli.TestSparkCliDriver.testCliDriver[union_remove_21] 
(batchId=120)
org.apache.hadoop.hive.cli.TestSparkCliDriver.testCliDriver[union_remove_22] 
(batchId=127)
org.apache.hadoop.hive.cli.TestSparkCliDriver.testCliDriver[union_remove_23] 
(batchId=125)
org.apache.hadoop.hive.cli.TestSparkCliDriver.testCliDriver[union_remove_24] 
(batchId=100)
org.apache.hadoop.hive.cli.TestSparkCliDriver.testCliDriver[union_remove_25] 
(batchId=132)