[ https://issues.apache.org/jira/browse/HIVE-15527?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=15799429#comment-15799429 ]
Hive QA commented on HIVE-15527: -------------------------------- Here are the results of testing the latest attachment: https://issues.apache.org/jira/secure/attachment/12845609/HIVE-15527.4.patch {color:red}ERROR:{color} -1 due to no test(s) being added or modified. {color:red}ERROR:{color} -1 due to 25 failed/errored test(s), 10914 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.TestMiniLlapCliDriver.testCliDriver[orc_ppd_basic] (batchId=134) org.apache.hadoop.hive.cli.TestMiniLlapCliDriver.testCliDriver[orc_ppd_schema_evol_3a] (batchId=135) org.apache.hadoop.hive.cli.TestMiniSparkOnYarnCliDriver.testCliDriver[vector_outer_join5] (batchId=161) org.apache.hadoop.hive.cli.TestSparkCliDriver.testCliDriver[auto_join_without_localtask] (batchId=94) org.apache.hadoop.hive.cli.TestSparkCliDriver.testCliDriver[auto_sortmerge_join_3] (batchId=94) org.apache.hadoop.hive.cli.TestSparkCliDriver.testCliDriver[bucket_map_join_spark4] (batchId=94) org.apache.hadoop.hive.cli.TestSparkCliDriver.testCliDriver[bucketmapjoin4] (batchId=94) org.apache.hadoop.hive.cli.TestSparkCliDriver.testCliDriver[date_join1] (batchId=94) org.apache.hadoop.hive.cli.TestSparkCliDriver.testCliDriver[groupby2_noskew] (batchId=94) org.apache.hadoop.hive.cli.TestSparkCliDriver.testCliDriver[groupby6_noskew] (batchId=94) org.apache.hadoop.hive.cli.TestSparkCliDriver.testCliDriver[mapjoin_test_outer] (batchId=94) org.apache.hadoop.hive.cli.TestSparkCliDriver.testCliDriver[merge2] (batchId=94) org.apache.hadoop.hive.cli.TestSparkCliDriver.testCliDriver[multi_join_union] (batchId=94) org.apache.hadoop.hive.cli.TestSparkCliDriver.testCliDriver[ptf] (batchId=101) org.apache.hadoop.hive.cli.TestSparkCliDriver.testCliDriver[smb_mapjoin_11] (batchId=94) org.apache.hadoop.hive.cli.TestSparkCliDriver.testCliDriver[timestamp_2] (batchId=94) org.apache.hadoop.hive.cli.TestSparkCliDriver.testCliDriver[union21] (batchId=94) org.apache.hadoop.hive.cli.TestSparkCliDriver.testCliDriver[vector_between_in] (batchId=118) org.apache.hadoop.hive.cli.TestSparkCliDriver.testCliDriver[vectorization_9] (batchId=94) org.apache.hadoop.hive.cli.TestSparkCliDriver.testCliDriver[vectorized_ptf] (batchId=121) org.apache.hive.jdbc.TestMultiSessionsHS2WithLocalClusterSpark.testSparkQuery (batchId=215) {noformat} Test results: https://builds.apache.org/job/PreCommit-HIVE-Build/2781/testReport Console output: https://builds.apache.org/job/PreCommit-HIVE-Build/2781/console Test logs: http://104.198.109.242/logs/PreCommit-HIVE-Build-2781/ 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: 25 tests failed {noformat} This message is automatically generated. ATTACHMENT ID: 12845609 - PreCommit-HIVE-Build > Memory usage is unbound in SortByShuffler for Spark > --------------------------------------------------- > > Key: HIVE-15527 > URL: https://issues.apache.org/jira/browse/HIVE-15527 > Project: Hive > Issue Type: Improvement > Components: Spark > Affects Versions: 1.1.0 > Reporter: Xuefu Zhang > Assignee: Chao Sun > Attachments: HIVE-15527.1.patch, HIVE-15527.2.patch, > HIVE-15527.3.patch, HIVE-15527.4.patch, HIVE-15527.patch > > > In SortByShuffler.java, an ArrayList is used to back the iterator for values > that have the same key in shuffled result produced by spark transformation > sortByKey. It's possible that memory can be exhausted because of a large key > group. > {code} > @Override > public Tuple2<HiveKey, Iterable<BytesWritable>> next() { > // TODO: implement this by accumulating rows with the same key > into a list. > // Note that this list needs to improved to prevent excessive > memory usage, but this > // can be done in later phase. > while (it.hasNext()) { > Tuple2<HiveKey, BytesWritable> pair = it.next(); > if (curKey != null && !curKey.equals(pair._1())) { > HiveKey key = curKey; > List<BytesWritable> values = curValues; > curKey = pair._1(); > curValues = new ArrayList<BytesWritable>(); > curValues.add(pair._2()); > return new Tuple2<HiveKey, Iterable<BytesWritable>>(key, > values); > } > curKey = pair._1(); > curValues.add(pair._2()); > } > if (curKey == null) { > throw new NoSuchElementException(); > } > // if we get here, this should be the last element we have > HiveKey key = curKey; > curKey = null; > return new Tuple2<HiveKey, Iterable<BytesWritable>>(key, > curValues); > } > {code} > Since the output from sortByKey is already sorted on key, it's possible to > backup the value iterable using the same input iterator. -- This message was sent by Atlassian JIRA (v6.3.4#6332)