Commit graph

6815 commits

Author SHA1 Message Date
Xiao Li a9928277da
[SPARK-24420][BUILD][FOLLOW-UP] Upgrade ASM6 APIs
## What changes were proposed in this pull request?
Use ASM 6 APIs after we upgrading it to ASM6.

## How was this patch tested?
N/A

Closes #22082 from gatorsmile/asm6.

Authored-by: Xiao Li <gatorsmile@gmail.com>
Signed-off-by: DB Tsai <d_tsai@apple.com>
2018-08-13 05:59:08 +00:00
10129659 02d0a1ffd9 [SPARK-25069][CORE] Using UnsafeAlignedOffset to make the entire record of 8 byte Items aligned like which is used in UnsafeExternalSorter
## What changes were proposed in this pull request?

The class of UnsafeExternalSorter used UnsafeAlignedOffset to make the entire record of 8 byte Items aligned, but ShuffleExternalSorter not.
The SPARC platform requires this because using a 4 byte Int for record lengths causes the entire record of 8 byte Items to become misaligned by 4 bytes. Using a 8 byte long for record length keeps things 8 byte aligned.

## How was this patch tested?
Existing Test.

Closes #22053 from eatoncys/UnsafeAlignedOffset.

Authored-by: 10129659 <chen.yanshan@zte.com.cn>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2018-08-13 09:09:25 +08:00
Kazuhiro Sera 8ec25cd67e Fix typos detected by github.com/client9/misspell
## What changes were proposed in this pull request?

Fixing typos is sometimes very hard. It's not so easy to visually review them. Recently, I discovered a very useful tool for it, [misspell](https://github.com/client9/misspell).

This pull request fixes minor typos detected by [misspell](https://github.com/client9/misspell) except for the false positives. If you would like me to work on other files as well, let me know.

## How was this patch tested?

### before

```
$ misspell . | grep -v '.js'
R/pkg/R/SQLContext.R:354:43: "definiton" is a misspelling of "definition"
R/pkg/R/SQLContext.R:424:43: "definiton" is a misspelling of "definition"
R/pkg/R/SQLContext.R:445:43: "definiton" is a misspelling of "definition"
R/pkg/R/SQLContext.R:495:43: "definiton" is a misspelling of "definition"
NOTICE-binary:454:16: "containd" is a misspelling of "contained"
R/pkg/R/context.R:46:43: "definiton" is a misspelling of "definition"
R/pkg/R/context.R:74:43: "definiton" is a misspelling of "definition"
R/pkg/R/DataFrame.R:591:48: "persistance" is a misspelling of "persistence"
R/pkg/R/streaming.R:166:44: "occured" is a misspelling of "occurred"
R/pkg/inst/worker/worker.R:65:22: "ouput" is a misspelling of "output"
R/pkg/tests/fulltests/test_utils.R:106:25: "environemnt" is a misspelling of "environment"
common/kvstore/src/test/java/org/apache/spark/util/kvstore/InMemoryStoreSuite.java:38:39: "existant" is a misspelling of "existent"
common/kvstore/src/test/java/org/apache/spark/util/kvstore/LevelDBSuite.java:83:39: "existant" is a misspelling of "existent"
common/network-common/src/main/java/org/apache/spark/network/crypto/TransportCipher.java:243:46: "transfered" is a misspelling of "transferred"
common/network-common/src/main/java/org/apache/spark/network/sasl/SaslEncryption.java:234:19: "transfered" is a misspelling of "transferred"
common/network-common/src/main/java/org/apache/spark/network/sasl/SaslEncryption.java:238:63: "transfered" is a misspelling of "transferred"
common/network-common/src/main/java/org/apache/spark/network/sasl/SaslEncryption.java:244:46: "transfered" is a misspelling of "transferred"
common/network-common/src/main/java/org/apache/spark/network/sasl/SaslEncryption.java:276:39: "transfered" is a misspelling of "transferred"
common/network-common/src/main/java/org/apache/spark/network/util/AbstractFileRegion.java:27:20: "transfered" is a misspelling of "transferred"
common/unsafe/src/test/scala/org/apache/spark/unsafe/types/UTF8StringPropertyCheckSuite.scala:195:15: "orgin" is a misspelling of "origin"
core/src/main/scala/org/apache/spark/api/python/PythonRDD.scala:621:39: "gauranteed" is a misspelling of "guaranteed"
core/src/main/scala/org/apache/spark/status/storeTypes.scala:113:29: "ect" is a misspelling of "etc"
core/src/main/scala/org/apache/spark/storage/DiskStore.scala:282:18: "transfered" is a misspelling of "transferred"
core/src/main/scala/org/apache/spark/util/ListenerBus.scala:64:17: "overriden" is a misspelling of "overridden"
core/src/test/scala/org/apache/spark/ShuffleSuite.scala:211:7: "substracted" is a misspelling of "subtracted"
core/src/test/scala/org/apache/spark/scheduler/DAGSchedulerSuite.scala:1922:49: "agriculteur" is a misspelling of "agriculture"
core/src/test/scala/org/apache/spark/scheduler/DAGSchedulerSuite.scala:2468:84: "truely" is a misspelling of "truly"
core/src/test/scala/org/apache/spark/storage/FlatmapIteratorSuite.scala:25:18: "persistance" is a misspelling of "persistence"
core/src/test/scala/org/apache/spark/storage/FlatmapIteratorSuite.scala:26:69: "persistance" is a misspelling of "persistence"
data/streaming/AFINN-111.txt:1219:0: "humerous" is a misspelling of "humorous"
dev/run-pip-tests:55:28: "enviroments" is a misspelling of "environments"
dev/run-pip-tests:91:37: "virutal" is a misspelling of "virtual"
dev/merge_spark_pr.py:377:72: "accross" is a misspelling of "across"
dev/merge_spark_pr.py:378:66: "accross" is a misspelling of "across"
dev/run-pip-tests:126:25: "enviroments" is a misspelling of "environments"
docs/configuration.md:1830:82: "overriden" is a misspelling of "overridden"
docs/structured-streaming-programming-guide.md:525:45: "processs" is a misspelling of "processes"
docs/structured-streaming-programming-guide.md:1165:61: "BETWEN" is a misspelling of "BETWEEN"
docs/sql-programming-guide.md:1891:810: "behaivor" is a misspelling of "behavior"
examples/src/main/python/sql/arrow.py:98:8: "substract" is a misspelling of "subtract"
examples/src/main/python/sql/arrow.py:103:27: "substract" is a misspelling of "subtract"
licenses/LICENSE-heapq.txt:5:63: "Stichting" is a misspelling of "Stitching"
licenses/LICENSE-heapq.txt:6:2: "Mathematisch" is a misspelling of "Mathematics"
licenses/LICENSE-heapq.txt:262:29: "Stichting" is a misspelling of "Stitching"
licenses/LICENSE-heapq.txt:262:39: "Mathematisch" is a misspelling of "Mathematics"
licenses/LICENSE-heapq.txt:269:49: "Stichting" is a misspelling of "Stitching"
licenses/LICENSE-heapq.txt:269:59: "Mathematisch" is a misspelling of "Mathematics"
licenses/LICENSE-heapq.txt:274:2: "STICHTING" is a misspelling of "STITCHING"
licenses/LICENSE-heapq.txt:274:12: "MATHEMATISCH" is a misspelling of "MATHEMATICS"
licenses/LICENSE-heapq.txt:276:29: "STICHTING" is a misspelling of "STITCHING"
licenses/LICENSE-heapq.txt:276:39: "MATHEMATISCH" is a misspelling of "MATHEMATICS"
licenses-binary/LICENSE-heapq.txt:5:63: "Stichting" is a misspelling of "Stitching"
licenses-binary/LICENSE-heapq.txt:6:2: "Mathematisch" is a misspelling of "Mathematics"
licenses-binary/LICENSE-heapq.txt:262:29: "Stichting" is a misspelling of "Stitching"
licenses-binary/LICENSE-heapq.txt:262:39: "Mathematisch" is a misspelling of "Mathematics"
licenses-binary/LICENSE-heapq.txt:269:49: "Stichting" is a misspelling of "Stitching"
licenses-binary/LICENSE-heapq.txt:269:59: "Mathematisch" is a misspelling of "Mathematics"
licenses-binary/LICENSE-heapq.txt:274:2: "STICHTING" is a misspelling of "STITCHING"
licenses-binary/LICENSE-heapq.txt:274:12: "MATHEMATISCH" is a misspelling of "MATHEMATICS"
licenses-binary/LICENSE-heapq.txt:276:29: "STICHTING" is a misspelling of "STITCHING"
licenses-binary/LICENSE-heapq.txt:276:39: "MATHEMATISCH" is a misspelling of "MATHEMATICS"
mllib/src/main/resources/org/apache/spark/ml/feature/stopwords/hungarian.txt:170:0: "teh" is a misspelling of "the"
mllib/src/main/resources/org/apache/spark/ml/feature/stopwords/portuguese.txt:53:0: "eles" is a misspelling of "eels"
mllib/src/main/scala/org/apache/spark/ml/stat/Summarizer.scala:99:20: "Euclidian" is a misspelling of "Euclidean"
mllib/src/main/scala/org/apache/spark/ml/stat/Summarizer.scala:539:11: "Euclidian" is a misspelling of "Euclidean"
mllib/src/main/scala/org/apache/spark/mllib/clustering/LDAOptimizer.scala:77:36: "Teh" is a misspelling of "The"
mllib/src/main/scala/org/apache/spark/mllib/clustering/StreamingKMeans.scala:230:24: "inital" is a misspelling of "initial"
mllib/src/main/scala/org/apache/spark/mllib/stat/MultivariateOnlineSummarizer.scala:276:9: "Euclidian" is a misspelling of "Euclidean"
mllib/src/test/scala/org/apache/spark/ml/clustering/KMeansSuite.scala:237:26: "descripiton" is a misspelling of "descriptions"
python/pyspark/find_spark_home.py:30:13: "enviroment" is a misspelling of "environment"
python/pyspark/context.py:937:12: "supress" is a misspelling of "suppress"
python/pyspark/context.py:938:12: "supress" is a misspelling of "suppress"
python/pyspark/context.py:939:12: "supress" is a misspelling of "suppress"
python/pyspark/context.py:940:12: "supress" is a misspelling of "suppress"
python/pyspark/heapq3.py:6:63: "Stichting" is a misspelling of "Stitching"
python/pyspark/heapq3.py:7:2: "Mathematisch" is a misspelling of "Mathematics"
python/pyspark/heapq3.py:263:29: "Stichting" is a misspelling of "Stitching"
python/pyspark/heapq3.py:263:39: "Mathematisch" is a misspelling of "Mathematics"
python/pyspark/heapq3.py:270:49: "Stichting" is a misspelling of "Stitching"
python/pyspark/heapq3.py:270:59: "Mathematisch" is a misspelling of "Mathematics"
python/pyspark/heapq3.py:275:2: "STICHTING" is a misspelling of "STITCHING"
python/pyspark/heapq3.py:275:12: "MATHEMATISCH" is a misspelling of "MATHEMATICS"
python/pyspark/heapq3.py:277:29: "STICHTING" is a misspelling of "STITCHING"
python/pyspark/heapq3.py:277:39: "MATHEMATISCH" is a misspelling of "MATHEMATICS"
python/pyspark/heapq3.py:713:8: "probabilty" is a misspelling of "probability"
python/pyspark/ml/clustering.py:1038:8: "Currenlty" is a misspelling of "Currently"
python/pyspark/ml/stat.py:339:23: "Euclidian" is a misspelling of "Euclidean"
python/pyspark/ml/regression.py:1378:20: "paramter" is a misspelling of "parameter"
python/pyspark/mllib/stat/_statistics.py:262:8: "probabilty" is a misspelling of "probability"
python/pyspark/rdd.py:1363:32: "paramter" is a misspelling of "parameter"
python/pyspark/streaming/tests.py:825:42: "retuns" is a misspelling of "returns"
python/pyspark/sql/tests.py:768:29: "initalization" is a misspelling of "initialization"
python/pyspark/sql/tests.py:3616:31: "initalize" is a misspelling of "initialize"
resource-managers/mesos/src/main/scala/org/apache/spark/scheduler/cluster/mesos/MesosSchedulerBackendUtil.scala:120:39: "arbitary" is a misspelling of "arbitrary"
resource-managers/mesos/src/test/scala/org/apache/spark/deploy/mesos/MesosClusterDispatcherArgumentsSuite.scala:26:45: "sucessfully" is a misspelling of "successfully"
resource-managers/mesos/src/main/scala/org/apache/spark/scheduler/cluster/mesos/MesosSchedulerUtils.scala:358:27: "constaints" is a misspelling of "constraints"
resource-managers/yarn/src/test/scala/org/apache/spark/deploy/yarn/YarnClusterSuite.scala:111:24: "senstive" is a misspelling of "sensitive"
sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/catalog/SessionCatalog.scala:1063:5: "overwirte" is a misspelling of "overwrite"
sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/datetimeExpressions.scala:1348:17: "compatability" is a misspelling of "compatibility"
sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/basicLogicalOperators.scala:77:36: "paramter" is a misspelling of "parameter"
sql/catalyst/src/main/scala/org/apache/spark/sql/internal/SQLConf.scala:1374:22: "precendence" is a misspelling of "precedence"
sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/analysis/AnalysisSuite.scala:238:27: "unnecassary" is a misspelling of "unnecessary"
sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/ConditionalExpressionSuite.scala:212:17: "whn" is a misspelling of "when"
sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/StreamingSymmetricHashJoinHelper.scala:147:60: "timestmap" is a misspelling of "timestamp"
sql/core/src/test/scala/org/apache/spark/sql/TPCDSQuerySuite.scala:150:45: "precentage" is a misspelling of "percentage"
sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/csv/CSVInferSchemaSuite.scala:135:29: "infered" is a misspelling of "inferred"
sql/hive/src/test/resources/golden/udf_instr-1-2e76f819563dbaba4beb51e3a130b922:1:52: "occurance" is a misspelling of "occurrence"
sql/hive/src/test/resources/golden/udf_instr-2-32da357fc754badd6e3898dcc8989182:1:52: "occurance" is a misspelling of "occurrence"
sql/hive/src/test/resources/golden/udf_locate-1-6e41693c9c6dceea4d7fab4c02884e4e:1:63: "occurance" is a misspelling of "occurrence"
sql/hive/src/test/resources/golden/udf_locate-2-d9b5934457931447874d6bb7c13de478:1:63: "occurance" is a misspelling of "occurrence"
sql/hive/src/test/resources/golden/udf_translate-2-f7aa38a33ca0df73b7a1e6b6da4b7fe8:9:79: "occurence" is a misspelling of "occurrence"
sql/hive/src/test/resources/golden/udf_translate-2-f7aa38a33ca0df73b7a1e6b6da4b7fe8:13:110: "occurence" is a misspelling of "occurrence"
sql/hive/src/test/resources/ql/src/test/queries/clientpositive/annotate_stats_join.q:46:105: "distint" is a misspelling of "distinct"
sql/hive/src/test/resources/ql/src/test/queries/clientpositive/auto_sortmerge_join_11.q:29:3: "Currenly" is a misspelling of "Currently"
sql/hive/src/test/resources/ql/src/test/queries/clientpositive/avro_partitioned.q:72:15: "existant" is a misspelling of "existent"
sql/hive/src/test/resources/ql/src/test/queries/clientpositive/decimal_udf.q:25:3: "substraction" is a misspelling of "subtraction"
sql/hive/src/test/resources/ql/src/test/queries/clientpositive/groupby2_map_multi_distinct.q:16:51: "funtion" is a misspelling of "function"
sql/hive/src/test/resources/ql/src/test/queries/clientpositive/groupby_sort_8.q:15:30: "issueing" is a misspelling of "issuing"
sql/hive/src/test/scala/org/apache/spark/sql/sources/HadoopFsRelationTest.scala:669:52: "wiht" is a misspelling of "with"
sql/hive-thriftserver/src/main/java/org/apache/hive/service/cli/session/HiveSessionImpl.java:474:9: "Refering" is a misspelling of "Referring"
```

### after

```
$ misspell . | grep -v '.js'
common/network-common/src/main/java/org/apache/spark/network/util/AbstractFileRegion.java:27:20: "transfered" is a misspelling of "transferred"
core/src/main/scala/org/apache/spark/status/storeTypes.scala:113:29: "ect" is a misspelling of "etc"
core/src/test/scala/org/apache/spark/scheduler/DAGSchedulerSuite.scala:1922:49: "agriculteur" is a misspelling of "agriculture"
data/streaming/AFINN-111.txt:1219:0: "humerous" is a misspelling of "humorous"
licenses/LICENSE-heapq.txt:5:63: "Stichting" is a misspelling of "Stitching"
licenses/LICENSE-heapq.txt:6:2: "Mathematisch" is a misspelling of "Mathematics"
licenses/LICENSE-heapq.txt:262:29: "Stichting" is a misspelling of "Stitching"
licenses/LICENSE-heapq.txt:262:39: "Mathematisch" is a misspelling of "Mathematics"
licenses/LICENSE-heapq.txt:269:49: "Stichting" is a misspelling of "Stitching"
licenses/LICENSE-heapq.txt:269:59: "Mathematisch" is a misspelling of "Mathematics"
licenses/LICENSE-heapq.txt:274:2: "STICHTING" is a misspelling of "STITCHING"
licenses/LICENSE-heapq.txt:274:12: "MATHEMATISCH" is a misspelling of "MATHEMATICS"
licenses/LICENSE-heapq.txt:276:29: "STICHTING" is a misspelling of "STITCHING"
licenses/LICENSE-heapq.txt:276:39: "MATHEMATISCH" is a misspelling of "MATHEMATICS"
licenses-binary/LICENSE-heapq.txt:5:63: "Stichting" is a misspelling of "Stitching"
licenses-binary/LICENSE-heapq.txt:6:2: "Mathematisch" is a misspelling of "Mathematics"
licenses-binary/LICENSE-heapq.txt:262:29: "Stichting" is a misspelling of "Stitching"
licenses-binary/LICENSE-heapq.txt:262:39: "Mathematisch" is a misspelling of "Mathematics"
licenses-binary/LICENSE-heapq.txt:269:49: "Stichting" is a misspelling of "Stitching"
licenses-binary/LICENSE-heapq.txt:269:59: "Mathematisch" is a misspelling of "Mathematics"
licenses-binary/LICENSE-heapq.txt:274:2: "STICHTING" is a misspelling of "STITCHING"
licenses-binary/LICENSE-heapq.txt:274:12: "MATHEMATISCH" is a misspelling of "MATHEMATICS"
licenses-binary/LICENSE-heapq.txt:276:29: "STICHTING" is a misspelling of "STITCHING"
licenses-binary/LICENSE-heapq.txt:276:39: "MATHEMATISCH" is a misspelling of "MATHEMATICS"
mllib/src/main/resources/org/apache/spark/ml/feature/stopwords/hungarian.txt:170:0: "teh" is a misspelling of "the"
mllib/src/main/resources/org/apache/spark/ml/feature/stopwords/portuguese.txt:53:0: "eles" is a misspelling of "eels"
mllib/src/main/scala/org/apache/spark/ml/stat/Summarizer.scala:99:20: "Euclidian" is a misspelling of "Euclidean"
mllib/src/main/scala/org/apache/spark/ml/stat/Summarizer.scala:539:11: "Euclidian" is a misspelling of "Euclidean"
mllib/src/main/scala/org/apache/spark/mllib/clustering/LDAOptimizer.scala:77:36: "Teh" is a misspelling of "The"
mllib/src/main/scala/org/apache/spark/mllib/stat/MultivariateOnlineSummarizer.scala:276:9: "Euclidian" is a misspelling of "Euclidean"
python/pyspark/heapq3.py:6:63: "Stichting" is a misspelling of "Stitching"
python/pyspark/heapq3.py:7:2: "Mathematisch" is a misspelling of "Mathematics"
python/pyspark/heapq3.py:263:29: "Stichting" is a misspelling of "Stitching"
python/pyspark/heapq3.py:263:39: "Mathematisch" is a misspelling of "Mathematics"
python/pyspark/heapq3.py:270:49: "Stichting" is a misspelling of "Stitching"
python/pyspark/heapq3.py:270:59: "Mathematisch" is a misspelling of "Mathematics"
python/pyspark/heapq3.py:275:2: "STICHTING" is a misspelling of "STITCHING"
python/pyspark/heapq3.py:275:12: "MATHEMATISCH" is a misspelling of "MATHEMATICS"
python/pyspark/heapq3.py:277:29: "STICHTING" is a misspelling of "STITCHING"
python/pyspark/heapq3.py:277:39: "MATHEMATISCH" is a misspelling of "MATHEMATICS"
python/pyspark/ml/stat.py:339:23: "Euclidian" is a misspelling of "Euclidean"
```

Closes #22070 from seratch/fix-typo.

Authored-by: Kazuhiro Sera <seratch@gmail.com>
Signed-off-by: Sean Owen <srowen@gmail.com>
2018-08-11 21:23:36 -05:00
Xingbo Jiang 4855d5c4b9 [SPARK-24822][PYSPARK] Python support for barrier execution mode
## What changes were proposed in this pull request?

This PR add python support for barrier execution mode, thus enable launch a job containing barrier stage(s) from PySpark.

We just forked the existing `RDDBarrier` and `RDD.barrier()` in Python api.

## How was this patch tested?

Manually tested:
```
>>> rdd = sc.parallelize([1, 2, 3, 4])
>>> def f(iterator): yield sum(iterator)
...
>>> rdd.barrier().mapPartitions(f).isBarrier() == True
True
```

Unit tests will be added in a follow-up PR that implements BarrierTaskContext on python side.

Closes #22011 from jiangxb1987/python.

Authored-by: Xingbo Jiang <xingbo.jiang@databricks.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2018-08-11 21:44:45 +08:00
Shixiong Zhu f5aba65739
[SPARK-25081][CORE] Nested spill in ShuffleExternalSorter should not access released memory page
## What changes were proposed in this pull request?

This issue is pretty similar to [SPARK-21907](https://issues.apache.org/jira/browse/SPARK-21907).

"allocateArray" in [ShuffleInMemorySorter.reset](9b8521e53e/core/src/main/java/org/apache/spark/shuffle/sort/ShuffleInMemorySorter.java (L99)) may trigger a spill and cause ShuffleInMemorySorter access the released `array`. Another task may get the same memory page from the pool. This will cause two tasks access the same memory page. When a task reads memory written by another task, many types of failures may happen. Here are some examples I  have seen:

- JVM crash. (This is easy to reproduce in a unit test as we fill newly allocated and deallocated memory with 0xa5 and 0x5a bytes which usually points to an invalid memory address)
- java.lang.IllegalArgumentException: Comparison method violates its general contract!
- java.lang.NullPointerException at org.apache.spark.memory.TaskMemoryManager.getPage(TaskMemoryManager.java:384)
- java.lang.UnsupportedOperationException: Cannot grow BufferHolder by size -536870912 because the size after growing exceeds size limitation 2147483632

This PR resets states in `ShuffleInMemorySorter.reset` before calling `allocateArray` to fix the issue.

## How was this patch tested?

The new unit test will make JVM crash without the fix.

Closes #22062 from zsxwing/SPARK-25081.

Authored-by: Shixiong Zhu <zsxwing@gmail.com>
Signed-off-by: Shixiong Zhu <zsxwing@gmail.com>
2018-08-10 10:53:44 -07:00
Liang-Chi Hsieh 4f17585098 [SPARK-19355][SQL] Use map output statistics to improve global limit's parallelism
## What changes were proposed in this pull request?

A logical `Limit` is performed physically by two operations `LocalLimit` and `GlobalLimit`.

Most of time, we gather all data into a single partition in order to run `GlobalLimit`. If we use a very big limit number, shuffling data causes performance issue also reduces parallelism.

We can avoid shuffling into single partition if we don't care data ordering. This patch implements this idea by doing a map stage during global limit. It collects the info of row numbers at each partition. For each partition, we locally retrieves limited data without any shuffling to finish this global limit.

For example, we have three partitions with rows (100, 100, 50) respectively. In global limit of 100 rows, we may take (34, 33, 33) rows for each partition locally. After global limit we still have three partitions.

If the data partition has certain ordering, we can't distribute required rows evenly to each partitions because it could change data ordering. But we still can avoid shuffling.

## How was this patch tested?

Jenkins tests.

Author: Liang-Chi Hsieh <viirya@gmail.com>

Closes #16677 from viirya/improve-global-limit-parallelism.
2018-08-10 11:32:15 +02:00
Kazuaki Ishizaki 56e9e97073 [MINOR][DOC] Fix typo
## What changes were proposed in this pull request?

This PR fixes typo regarding `auxiliary verb + verb[s]`. This is a follow-on of #21956.

## How was this patch tested?

N/A

Closes #22040 from kiszk/spellcheck1.

Authored-by: Kazuaki Ishizaki <ishizaki@jp.ibm.com>
Signed-off-by: hyukjinkwon <gurwls223@apache.org>
2018-08-09 20:10:17 +08:00
Xingbo Jiang d90f1336d8 [SPARK-25045][CORE] Make RDDBarrier.mapParititions similar to RDD.mapPartitions
## What changes were proposed in this pull request?

Signature of the function passed to `RDDBarrier.mapPartitions()` is different from that of `RDD.mapPartitions`. The later doesn’t take a `TaskContext`. We shall make the function signature the same to avoid confusion and misusage.

This PR proposes the following API changes:
- In `RDDBarrier`, migrate `mapPartitions` from
   ```
        def mapPartitions[S: ClassTag](
            f: (Iterator[T], BarrierTaskContext) => Iterator[S],
            preservesPartitioning: Boolean = false): RDD[S]
        }
   ```
    to
   ```
        def mapPartitions[S: ClassTag](
            f: Iterator[T] => Iterator[S],
            preservesPartitioning: Boolean = false): RDD[S]
        }
   ```
- Add new static method to get a `BarrierTaskContext`:
   ```
        object BarrierTaskContext {
           def get(): BarrierTaskContext
        }
   ```

## How was this patch tested?

Existing test cases.

Author: Xingbo Jiang <xingbo.jiang@databricks.com>

Closes #22026 from jiangxb1987/mapPartitions.
2018-08-07 17:32:41 -07:00
Sean Owen 66699c5c30 [SPARK-25029][TESTS] Scala 2.12 issues: TaskNotSerializable and Janino "Two non-abstract methods ..." errors
## What changes were proposed in this pull request?

Fixes for test issues that arose after Scala 2.12 support was added -- ones that only affect the 2.12 build.

## How was this patch tested?

Existing tests.

Closes #22004 from srowen/SPARK-25029.

Authored-by: Sean Owen <srowen@gmail.com>
Signed-off-by: Sean Owen <srowen@gmail.com>
2018-08-07 17:30:37 -05:00
Maxim Gekk 131ca146ed [SPARK-24005][CORE] Remove usage of Scala’s parallel collection
## What changes were proposed in this pull request?

In the PR, I propose to replace Scala parallel collections by new methods `parmap()`. The methods use futures to transform a sequential collection by applying a lambda function to each element in parallel. The result of `parmap` is another regular (sequential) collection.

The proposed `parmap` method aims to solve the problem of impossibility to interrupt parallel Scala collection. This possibility is needed for reliable task preemption.

## How was this patch tested?

A test was added to `ThreadUtilsSuite`

Closes #21913 from MaxGekk/par-map.

Authored-by: Maxim Gekk <maxim.gekk@databricks.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2018-08-07 17:14:30 +08:00
Xingbo Jiang 388f5a0635 [SPARK-24817][CORE] Implement BarrierTaskContext.barrier()
## What changes were proposed in this pull request?

Implement BarrierTaskContext.barrier(), to support global sync between all the tasks in a barrier stage.
The function set a global barrier and waits until all tasks in this stage hit this barrier. Similar to MPI_Barrier function in MPI, the barrier() function call blocks until all tasks in the same stage have reached this routine. The global sync shall finish immediately once all tasks in the same barrier stage reaches the same barrier.

This PR implements BarrierTaskContext.barrier() based on netty-based RPC client, introduces new `BarrierCoordinator` and new `BarrierCoordinatorMessage`, and new config to handle timeout issue.

## How was this patch tested?
Add `BarrierTaskContextSuite` to test `BarrierTaskContext.barrier()`

Closes #21898 from jiangxb1987/taskcontext.barrier.

Authored-by: Xingbo Jiang <xingbo.jiang@databricks.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2018-08-07 15:06:32 +08:00
deshanxiao 1076e4f002 [MINOR][DOCS] Fix grammatical error in SortShuffleManager
## What changes were proposed in this pull request?

Fix a grammatical error in the comment of SortShuffleManager.

## How was this patch tested?

N/A

Closes #21956 from deshanxiao/master.

Authored-by: deshanxiao <42019462+deshanxiao@users.noreply.github.com>
Signed-off-by: hyukjinkwon <gurwls223@apache.org>
2018-08-07 09:36:37 +08:00
Marco Gaido 3c96937c7b [SPARK-24948][SHS] Delegate check access permissions to the file system
## What changes were proposed in this pull request?

In `SparkHadoopUtil. checkAccessPermission`,  we consider only basic permissions in order to check wether a user can access a file or not. This is not a complete check, as it ignores ACLs and other policies a file system may apply in its internal. So this can result in returning wrongly that a user cannot access a file (despite he actually can).

The PR proposes to delegate to the filesystem the check whether a file is accessible or not, in order to return the right result. A caching layer is added for performance reasons.

## How was this patch tested?

modified UTs

Author: Marco Gaido <marcogaido91@gmail.com>

Closes #21895 from mgaido91/SPARK-24948.
2018-08-06 14:29:05 -07:00
Hieu Huynh 51e2b38d93 [SPARK-24992][CORE] spark should randomize yarn local dir selection
**Description: [SPARK-24992](https://issues.apache.org/jira/browse/SPARK-24992)**
Utils.getLocalDir is used to get path of a temporary directory. However, it always returns the the same directory, which is the first element in the array localRootDirs. When running on YARN, this might causes the case that we always write to one disk, which makes it busy while other disks are free. We should randomize the selection to spread out the loads.

**What changes were proposed in this pull request?**
This PR randomized the selection of local directory inside the method Utils.getLocalDir. This change affects the Utils.fetchFile method since it based on the fact that Utils.getLocalDir always return the same directory to cache file. Therefore, a new variable cachedLocalDir is used to cache the first localDirectory that it gets from Utils.getLocalDir. Also, when getting the configured local directories (inside Utils. getConfiguredLocalDirs), in case we are in yarn mode, the array of directories are also randomized before return.

Author: Hieu Huynh <“Hieu.huynh@oath.com”>

Closes #21953 from hthuynh2/SPARK_24992.
2018-08-06 13:58:28 -05:00
Hieu Huynh 35700bb7f2 [SPARK-24981][CORE] ShutdownHook timeout causes job to fail when succeeded when SparkContext stop() not called by user program
**Description**
The issue is described in [SPARK-24981](https://issues.apache.org/jira/browse/SPARK-24981).

**How does this PR fix the issue?**
This PR catch the Exception that is thrown while the Sparkcontext.stop() is running (when it is called by the ShutdownHookManager).

**How was this patch tested?**
I manually tested it by adding delay (60s) inside the stop(). This make the shutdownHookManger interrupt the thread that is running stop(). The Interrupted Exception was catched and the job succeed.

Author: Hieu Huynh <“Hieu.huynh@oath.com”>
Author: Hieu Tri Huynh <hthieu96@gmail.com>

Closes #21936 from hthuynh2/SPARK_24981.
2018-08-06 09:01:51 -05:00
hyukjinkwon 55e3ae6930 [SPARK-25001][BUILD] Fix miscellaneous build warnings
## What changes were proposed in this pull request?

There are many warnings in the current build (for instance see https://amplab.cs.berkeley.edu/jenkins/job/spark-master-test-sbt-hadoop-2.7/4734/console).

**common**:

```
[warn] /home/jenkins/workspace/spark-master-test-maven-hadoop-2.7/common/kvstore/src/main/java/org/apache/spark/util/kvstore/LevelDB.java:237: warning: [rawtypes] found raw type: LevelDBIterator
[warn]   void closeIterator(LevelDBIterator it) throws IOException {
[warn]                      ^

[warn]   missing type arguments for generic class LevelDBIterator<T>
[warn]   where T is a type-variable:
[warn]     T extends Object declared in class LevelDBIterator
[warn] /home/jenkins/workspace/spark-master-test-maven-hadoop-2.7/common/network-common/src/main/java/org/apache/spark/network/server/TransportServer.java:151: warning: [deprecation] group() in AbstractBootstrap has been deprecated
[warn]     if (bootstrap != null && bootstrap.group() != null) {
[warn]                                       ^

[warn] /home/jenkins/workspace/spark-master-test-maven-hadoop-2.7/common/network-common/src/main/java/org/apache/spark/network/server/TransportServer.java:152: warning: [deprecation] group() in AbstractBootstrap has been deprecated
[warn]       bootstrap.group().shutdownGracefully();
[warn]                ^

[warn] /home/jenkins/workspace/spark-master-test-maven-hadoop-2.7/common/network-common/src/main/java/org/apache/spark/network/server/TransportServer.java:154: warning: [deprecation] childGroup() in ServerBootstrap has been deprecated
[warn]     if (bootstrap != null && bootstrap.childGroup() != null) {
[warn]                                       ^

[warn] /home/jenkins/workspace/spark-master-test-maven-hadoop-2.7/common/network-common/src/main/java/org/apache/spark/network/server/TransportServer.java:155: warning: [deprecation] childGroup() in ServerBootstrap has been deprecated
[warn]       bootstrap.childGroup().shutdownGracefully();
[warn]                ^

[warn] /home/jenkins/workspace/spark-master-test-maven-hadoop-2.7/common/network-common/src/main/java/org/apache/spark/network/util/NettyUtils.java:112: warning: [deprecation] PooledByteBufAllocator(boolean,int,int,int,int,int,int,int) in PooledByteBufAllocator has been deprecated
[warn]     return new PooledByteBufAllocator(
[warn]            ^

[warn] /home/jenkins/workspace/spark-master-test-maven-hadoop-2.7/common/network-common/src/main/java/org/apache/spark/network/client/TransportClient.java:321: warning: [rawtypes] found raw type: Future
[warn]     public void operationComplete(Future future) throws Exception {
[warn]                                   ^

[warn]   missing type arguments for generic class Future<V>
[warn]   where V is a type-variable:
[warn]     V extends Object declared in interface Future
[warn] /home/jenkins/workspace/spark-master-test-maven-hadoop-2.7/common/network-common/src/main/java/org/apache/spark/network/client/TransportResponseHandler.java:215: warning: [rawtypes] found raw type: StreamInterceptor
[warn]           StreamInterceptor interceptor = new StreamInterceptor(this, resp.streamId, resp.byteCount,
[warn]           ^

[warn]   missing type arguments for generic class StreamInterceptor<T>
[warn]   where T is a type-variable:
[warn]     T extends Message declared in class StreamInterceptor
[warn] /home/jenkins/workspace/spark-master-test-maven-hadoop-2.7/common/network-common/src/main/java/org/apache/spark/network/client/TransportResponseHandler.java:215: warning: [rawtypes] found raw type: StreamInterceptor
[warn]           StreamInterceptor interceptor = new StreamInterceptor(this, resp.streamId, resp.byteCount,
[warn]                                               ^

[warn]   missing type arguments for generic class StreamInterceptor<T>
[warn]   where T is a type-variable:
[warn]     T extends Message declared in class StreamInterceptor
[warn] /home/jenkins/workspace/spark-master-test-maven-hadoop-2.7/common/network-common/src/main/java/org/apache/spark/network/client/TransportResponseHandler.java:215: warning: [unchecked] unchecked call to StreamInterceptor(MessageHandler<T>,String,long,StreamCallback) as a member of the raw type StreamInterceptor
[warn]           StreamInterceptor interceptor = new StreamInterceptor(this, resp.streamId, resp.byteCount,
[warn]                                           ^

[warn]   where T is a type-variable:
[warn]     T extends Message declared in class StreamInterceptor
[warn] /home/jenkins/workspace/spark-master-test-maven-hadoop-2.7/common/network-common/src/main/java/org/apache/spark/network/server/TransportRequestHandler.java:255: warning: [rawtypes] found raw type: StreamInterceptor
[warn]         StreamInterceptor interceptor = new StreamInterceptor(this, wrappedCallback.getID(),
[warn]         ^

[warn]   missing type arguments for generic class StreamInterceptor<T>
[warn]   where T is a type-variable:
[warn]     T extends Message declared in class StreamInterceptor
[warn] /home/jenkins/workspace/spark-master-test-maven-hadoop-2.7/common/network-common/src/main/java/org/apache/spark/network/server/TransportRequestHandler.java:255: warning: [rawtypes] found raw type: StreamInterceptor
[warn]         StreamInterceptor interceptor = new StreamInterceptor(this, wrappedCallback.getID(),
[warn]                                             ^

[warn]   missing type arguments for generic class StreamInterceptor<T>
[warn]   where T is a type-variable:
[warn]     T extends Message declared in class StreamInterceptor
[warn] /home/jenkins/workspace/spark-master-test-maven-hadoop-2.7/common/network-common/src/main/java/org/apache/spark/network/server/TransportRequestHandler.java:255: warning: [unchecked] unchecked call to StreamInterceptor(MessageHandler<T>,String,long,StreamCallback) as a member of the raw type StreamInterceptor
[warn]         StreamInterceptor interceptor = new StreamInterceptor(this, wrappedCallback.getID(),
[warn]                                         ^

[warn]   where T is a type-variable:
[warn]     T extends Message declared in class StreamInterceptor
[warn] /home/jenkins/workspace/spark-master-test-maven-hadoop-2.7/common/network-common/src/main/java/org/apache/spark/network/crypto/TransportCipher.java:270: warning: [deprecation] transfered() in FileRegion has been deprecated
[warn]         region.transferTo(byteRawChannel, region.transfered());
[warn]                                                 ^

[warn] /home/jenkins/workspace/spark-master-test-maven-hadoop-2.7/common/network-common/src/main/java/org/apache/spark/network/sasl/SaslEncryption.java:304: warning: [deprecation] transfered() in FileRegion has been deprecated
[warn]         region.transferTo(byteChannel, region.transfered());
[warn]                                              ^

[warn] /home/jenkins/workspace/spark-master-test-maven-hadoop-2.7/common/network-common/src/test/java/org/apache/spark/network/ProtocolSuite.java:119: warning: [deprecation] transfered() in FileRegion has been deprecated
[warn]       while (in.transfered() < in.count()) {
[warn]                ^

[warn] /home/jenkins/workspace/spark-master-test-maven-hadoop-2.7/common/network-common/src/test/java/org/apache/spark/network/ProtocolSuite.java:120: warning: [deprecation] transfered() in FileRegion has been deprecated
[warn]         in.transferTo(channel, in.transfered());
[warn]                                  ^

[warn] /home/jenkins/workspace/spark-master-test-maven-hadoop-2.7/common/unsafe/src/test/java/org/apache/spark/unsafe/hash/Murmur3_x86_32Suite.java:80: warning: [static] static method should be qualified by type name, Murmur3_x86_32, instead of by an expression
[warn]     Assert.assertEquals(-300363099, hasher.hashUnsafeWords(bytes, offset, 16, 42));
[warn]                                           ^

[warn] /home/jenkins/workspace/spark-master-test-maven-hadoop-2.7/common/unsafe/src/test/java/org/apache/spark/unsafe/hash/Murmur3_x86_32Suite.java:84: warning: [static] static method should be qualified by type name, Murmur3_x86_32, instead of by an expression
[warn]     Assert.assertEquals(-1210324667, hasher.hashUnsafeWords(bytes, offset, 16, 42));
[warn]                                            ^

[warn] /home/jenkins/workspace/spark-master-test-maven-hadoop-2.7/common/unsafe/src/test/java/org/apache/spark/unsafe/hash/Murmur3_x86_32Suite.java:88: warning: [static] static method should be qualified by type name, Murmur3_x86_32, instead of by an expression
[warn]     Assert.assertEquals(-634919701, hasher.hashUnsafeWords(bytes, offset, 16, 42));
[warn]                                           ^
```

**launcher**:

```
[warn] Pruning sources from previous analysis, due to incompatible CompileSetup.
[warn] /home/jenkins/workspace/spark-master-test-maven-hadoop-2.7/launcher/src/main/java/org/apache/spark/launcher/AbstractLauncher.java:31: warning: [rawtypes] found raw type: AbstractLauncher
[warn] public abstract class AbstractLauncher<T extends AbstractLauncher> {
[warn]                                                  ^
[warn]   missing type arguments for generic class AbstractLauncher<T>
[warn]   where T is a type-variable:
[warn]     T extends AbstractLauncher declared in class AbstractLauncher
```

**core**:

```
[warn] /home/jenkins/workspace/spark-master-test-maven-hadoop-2.7/core/src/main/scala/org/apache/spark/api/r/RBackend.scala:99: method group in class AbstractBootstrap is deprecated: see corresponding Javadoc for more information.
[warn]     if (bootstrap != null && bootstrap.group() != null) {
[warn]                                        ^

[warn] /home/jenkins/workspace/spark-master-test-maven-hadoop-2.7/core/src/main/scala/org/apache/spark/api/r/RBackend.scala💯 method group in class AbstractBootstrap is deprecated: see corresponding Javadoc for more information.
[warn]       bootstrap.group().shutdownGracefully()
[warn]                 ^

[warn] /home/jenkins/workspace/spark-master-test-maven-hadoop-2.7/core/src/main/scala/org/apache/spark/api/r/RBackend.scala:102: method childGroup in class ServerBootstrap is deprecated: see corresponding Javadoc for more information.
[warn]     if (bootstrap != null && bootstrap.childGroup() != null) {
[warn]                                        ^

[warn] /home/jenkins/workspace/spark-master-test-maven-hadoop-2.7/core/src/main/scala/org/apache/spark/api/r/RBackend.scala:103: method childGroup in class ServerBootstrap is deprecated: see corresponding Javadoc for more information.
[warn]       bootstrap.childGroup().shutdownGracefully()
[warn]                 ^

[warn] /home/jenkins/workspace/spark-master-test-maven-hadoop-2.7/core/src/test/scala/org/apache/spark/util/ClosureCleanerSuite.scala:151: reflective access of structural type member method getData should be enabled
[warn] by making the implicit value scala.language.reflectiveCalls visible.
[warn] This can be achieved by adding the import clause 'import scala.language.reflectiveCalls'
[warn] or by setting the compiler option -language:reflectiveCalls.
[warn] See the Scaladoc for value scala.language.reflectiveCalls for a discussion
[warn] why the feature should be explicitly enabled.
[warn]       val rdd = sc.parallelize(1 to 1).map(concreteObject.getData)
[warn]                                                           ^

[warn] /home/jenkins/workspace/spark-master-test-maven-hadoop-2.7/core/src/test/scala/org/apache/spark/util/ClosureCleanerSuite.scala:175: reflective access of structural type member value innerObject2 should be enabled
[warn] by making the implicit value scala.language.reflectiveCalls visible.
[warn]       val rdd = sc.parallelize(1 to 1).map(concreteObject.innerObject2.getData)
[warn]                                                           ^

[warn] /home/jenkins/workspace/spark-master-test-maven-hadoop-2.7/core/src/test/scala/org/apache/spark/util/ClosureCleanerSuite.scala:175: reflective access of structural type member method getData should be enabled
[warn] by making the implicit value scala.language.reflectiveCalls visible.
[warn]       val rdd = sc.parallelize(1 to 1).map(concreteObject.innerObject2.getData)
[warn]                                                                        ^

[warn] /home/jenkins/workspace/spark-master-test-maven-hadoop-2.7/core/src/test/scala/org/apache/spark/LocalSparkContext.scala:32: constructor Slf4JLoggerFactory in class Slf4JLoggerFactory is deprecated: see corresponding Javadoc for more information.
[warn]     InternalLoggerFactory.setDefaultFactory(new Slf4JLoggerFactory())
[warn]                                             ^

[warn] /home/jenkins/workspace/spark-master-test-maven-hadoop-2.7/core/src/test/scala/org/apache/spark/status/AppStatusListenerSuite.scala:218: value attemptId in class StageInfo is deprecated: Use attemptNumber instead
[warn]         assert(wrapper.stageAttemptId === stages.head.attemptId)
[warn]                                                       ^

[warn] /home/jenkins/workspace/spark-master-test-maven-hadoop-2.7/core/src/test/scala/org/apache/spark/status/AppStatusListenerSuite.scala:261: value attemptId in class StageInfo is deprecated: Use attemptNumber instead
[warn]       stageAttemptId = stages.head.attemptId))
[warn]                                    ^

[warn] /home/jenkins/workspace/spark-master-test-maven-hadoop-2.7/core/src/test/scala/org/apache/spark/status/AppStatusListenerSuite.scala:287: value attemptId in class StageInfo is deprecated: Use attemptNumber instead
[warn]       stageAttemptId = stages.head.attemptId))
[warn]                                    ^

[warn] /home/jenkins/workspace/spark-master-test-maven-hadoop-2.7/core/src/test/scala/org/apache/spark/status/AppStatusListenerSuite.scala:471: value attemptId in class StageInfo is deprecated: Use attemptNumber instead
[warn]       stageAttemptId = stages.last.attemptId))
[warn]                                    ^

[warn] /home/jenkins/workspace/spark-master-test-maven-hadoop-2.7/core/src/test/scala/org/apache/spark/status/AppStatusListenerSuite.scala:966: value attemptId in class StageInfo is deprecated: Use attemptNumber instead
[warn]     listener.onTaskStart(SparkListenerTaskStart(dropped.stageId, dropped.attemptId, task))
[warn]                                                                          ^

[warn] /home/jenkins/workspace/spark-master-test-maven-hadoop-2.7/core/src/test/scala/org/apache/spark/status/AppStatusListenerSuite.scala:972: value attemptId in class StageInfo is deprecated: Use attemptNumber instead
[warn]     listener.onTaskEnd(SparkListenerTaskEnd(dropped.stageId, dropped.attemptId,
[warn]                                                                      ^

[warn] /home/jenkins/workspace/spark-master-test-maven-hadoop-2.7/core/src/test/scala/org/apache/spark/status/AppStatusListenerSuite.scala:976: value attemptId in class StageInfo is deprecated: Use attemptNumber instead
[warn]       .taskSummary(dropped.stageId, dropped.attemptId, Array(0.25d, 0.50d, 0.75d))
[warn]                                             ^

[warn] /home/jenkins/workspace/spark-master-test-maven-hadoop-2.7/core/src/test/scala/org/apache/spark/status/AppStatusListenerSuite.scala:1146: value attemptId in class StageInfo is deprecated: Use attemptNumber instead
[warn]       SparkListenerTaskEnd(stage1.stageId, stage1.attemptId, "taskType", Success, tasks(1), null))
[warn]                                                   ^

[warn] /home/jenkins/workspace/spark-master-test-maven-hadoop-2.7/core/src/test/scala/org/apache/spark/status/AppStatusListenerSuite.scala:1150: value attemptId in class StageInfo is deprecated: Use attemptNumber instead
[warn]       SparkListenerTaskEnd(stage1.stageId, stage1.attemptId, "taskType", Success, tasks(0), null))
[warn]                                                   ^

[warn] /home/jenkins/workspace/spark-master-test-maven-hadoop-2.7/core/src/test/scala/org/apache/spark/storage/DiskStoreSuite.scala:197: method transfered in trait FileRegion is deprecated: see corresponding Javadoc for more information.
[warn]     while (region.transfered() < region.count()) {
[warn]                   ^

[warn] /home/jenkins/workspace/spark-master-test-maven-hadoop-2.7/core/src/test/scala/org/apache/spark/storage/DiskStoreSuite.scala:198: method transfered in trait FileRegion is deprecated: see corresponding Javadoc for more information.
[warn]       region.transferTo(byteChannel, region.transfered())
[warn]                                             ^
```

**sql**:

```
[warn] /home/jenkins/workspace/spark-master-test-maven-hadoop-2.7/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/analysis/AnalysisSuite.scala:534: abstract type T is unchecked since it is eliminated by erasure
[warn]       assert(partitioning.isInstanceOf[T])
[warn]                                       ^

[warn] /home/jenkins/workspace/spark-master-test-maven-hadoop-2.7/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/analysis/AnalysisSuite.scala:534: abstract type T is unchecked since it is eliminated by erasure
[warn]       assert(partitioning.isInstanceOf[T])
[warn]             ^

[warn] /home/jenkins/workspace/spark-master-test-maven-hadoop-2.7/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/ObjectExpressionsSuite.scala:323: inferred existential type Option[Class[_$1]]( forSome { type _$1 }), which cannot be expressed by wildcards,  should be enabled
[warn] by making the implicit value scala.language.existentials visible.
[warn] This can be achieved by adding the import clause 'import scala.language.existentials'
[warn] or by setting the compiler option -language:existentials.
[warn] See the Scaladoc for value scala.language.existentials for a discussion
[warn] why the feature should be explicitly enabled.
[warn]       val optClass = Option(collectionCls)
[warn]                            ^

[warn] /home/jenkins/workspace/spark-master-test-maven-hadoop-2.7/sql/core/src/main/java/org/apache/spark/sql/execution/datasources/parquet/SpecificParquetRecordReaderBase.java:226: warning: [deprecation] ParquetFileReader(Configuration,FileMetaData,Path,List<BlockMetaData>,List<ColumnDescriptor>) in ParquetFileReader has been deprecated
[warn]     this.reader = new ParquetFileReader(
[warn]                   ^

[warn] /home/jenkins/workspace/spark-master-test-maven-hadoop-2.7/sql/core/src/main/java/org/apache/spark/sql/execution/datasources/parquet/VectorizedColumnReader.java:178: warning: [deprecation] getType() in ColumnDescriptor has been deprecated
[warn]             (descriptor.getType() == PrimitiveType.PrimitiveTypeName.INT32 ||
[warn]                        ^

[warn] /home/jenkins/workspace/spark-master-test-maven-hadoop-2.7/sql/core/src/main/java/org/apache/spark/sql/execution/datasources/parquet/VectorizedColumnReader.java:179: warning: [deprecation] getType() in ColumnDescriptor has been deprecated
[warn]             (descriptor.getType() == PrimitiveType.PrimitiveTypeName.INT64  &&
[warn]                        ^

[warn] /home/jenkins/workspace/spark-master-test-maven-hadoop-2.7/sql/core/src/main/java/org/apache/spark/sql/execution/datasources/parquet/VectorizedColumnReader.java:181: warning: [deprecation] getType() in ColumnDescriptor has been deprecated
[warn]             descriptor.getType() == PrimitiveType.PrimitiveTypeName.FLOAT ||
[warn]                       ^

[warn] /home/jenkins/workspace/spark-master-test-maven-hadoop-2.7/sql/core/src/main/java/org/apache/spark/sql/execution/datasources/parquet/VectorizedColumnReader.java:182: warning: [deprecation] getType() in ColumnDescriptor has been deprecated
[warn]             descriptor.getType() == PrimitiveType.PrimitiveTypeName.DOUBLE ||
[warn]                       ^

[warn] /home/jenkins/workspace/spark-master-test-maven-hadoop-2.7/sql/core/src/main/java/org/apache/spark/sql/execution/datasources/parquet/VectorizedColumnReader.java:183: warning: [deprecation] getType() in ColumnDescriptor has been deprecated
[warn]             descriptor.getType() == PrimitiveType.PrimitiveTypeName.BINARY))) {
[warn]                       ^

[warn] /home/jenkins/workspace/spark-master-test-maven-hadoop-2.7/sql/core/src/main/java/org/apache/spark/sql/execution/datasources/parquet/VectorizedColumnReader.java:198: warning: [deprecation] getType() in ColumnDescriptor has been deprecated
[warn]         switch (descriptor.getType()) {
[warn]                           ^

[warn] /home/jenkins/workspace/spark-master-test-maven-hadoop-2.7/sql/core/src/main/java/org/apache/spark/sql/execution/datasources/parquet/VectorizedColumnReader.java:221: warning: [deprecation] getTypeLength() in ColumnDescriptor has been deprecated
[warn]             readFixedLenByteArrayBatch(rowId, num, column, descriptor.getTypeLength());
[warn]                                                                      ^

[warn] /home/jenkins/workspace/spark-master-test-maven-hadoop-2.7/sql/core/src/main/java/org/apache/spark/sql/execution/datasources/parquet/VectorizedColumnReader.java:224: warning: [deprecation] getType() in ColumnDescriptor has been deprecated
[warn]             throw new IOException("Unsupported type: " + descriptor.getType());
[warn]                                                                    ^

[warn] /home/jenkins/workspace/spark-master-test-maven-hadoop-2.7/sql/core/src/main/java/org/apache/spark/sql/execution/datasources/parquet/VectorizedColumnReader.java:246: warning: [deprecation] getType() in ColumnDescriptor has been deprecated
[warn]       descriptor.getType().toString(),
[warn]                 ^

[warn] /home/jenkins/workspace/spark-master-test-maven-hadoop-2.7/sql/core/src/main/java/org/apache/spark/sql/execution/datasources/parquet/VectorizedColumnReader.java:258: warning: [deprecation] getType() in ColumnDescriptor has been deprecated
[warn]     switch (descriptor.getType()) {
[warn]                       ^

[warn] /home/jenkins/workspace/spark-master-test-maven-hadoop-2.7/sql/core/src/main/java/org/apache/spark/sql/execution/datasources/parquet/VectorizedColumnReader.java:384: warning: [deprecation] getType() in ColumnDescriptor has been deprecated
[warn]         throw new UnsupportedOperationException("Unsupported type: " + descriptor.getType());
[warn]                                                                                  ^

[warn] /home/jenkins/workspace/spark-master-test-maven-hadoop-2.7/sql/core/src/main/java/org/apache/spark/sql/vectorized/ArrowColumnVector.java:458: warning: [static] static variable should be qualified by type name, BaseRepeatedValueVector, instead of by an expression
[warn]       int index = rowId * accessor.OFFSET_WIDTH;
[warn]                                   ^
[warn] /home/jenkins/workspace/spark-master-test-maven-hadoop-2.7/sql/core/src/main/java/org/apache/spark/sql/vectorized/ArrowColumnVector.java:460: warning: [static] static variable should be qualified by type name, BaseRepeatedValueVector, instead of by an expression
[warn]       int end = offsets.getInt(index + accessor.OFFSET_WIDTH);
[warn]                                                ^

[warn] /home/jenkins/workspace/spark-master-test-maven-hadoop-2.7/sql/core/src/test/scala/org/apache/spark/sql/BenchmarkQueryTest.scala:57: a pure expression does nothing in statement position; you may be omitting necessary parentheses
[warn]       case s => s
[warn]                 ^

[warn] /home/jenkins/workspace/spark-master-test-maven-hadoop-2.7/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetInteroperabilitySuite.scala:182: inferred existential type org.apache.parquet.column.statistics.Statistics[?0]( forSome { type ?0 <: Comparable[?0] }), which cannot be expressed by wildcards,  should be enabled
[warn] by making the implicit value scala.language.existentials visible.
[warn] This can be achieved by adding the import clause 'import scala.language.existentials'
[warn] or by setting the compiler option -language:existentials.
[warn] See the Scaladoc for value scala.language.existentials for a discussion
[warn] why the feature should be explicitly enabled.
[warn]                 val columnStats = oneBlockColumnMeta.getStatistics
[warn]                                                      ^

[warn] /home/jenkins/workspace/spark-master-test-maven-hadoop-2.7/sql/core/src/test/scala/org/apache/spark/sql/execution/streaming/sources/ForeachBatchSinkSuite.scala:146: implicit conversion method conv should be enabled
[warn] by making the implicit value scala.language.implicitConversions visible.
[warn] This can be achieved by adding the import clause 'import scala.language.implicitConversions'
[warn] or by setting the compiler option -language:implicitConversions.
[warn] See the Scaladoc for value scala.language.implicitConversions for a discussion
[warn] why the feature should be explicitly enabled.
[warn]     implicit def conv(x: (Int, Long)): KV = KV(x._1, x._2)
[warn]                  ^

[warn] /home/jenkins/workspace/spark-master-test-maven-hadoop-2.7/sql/core/src/test/scala/org/apache/spark/sql/streaming/continuous/shuffle/ContinuousShuffleSuite.scala:48: implicit conversion method unsafeRow should be enabled
[warn] by making the implicit value scala.language.implicitConversions visible.
[warn]   private implicit def unsafeRow(value: Int) = {
[warn]                        ^

[warn] /home/jenkins/workspace/spark-master-test-maven-hadoop-2.7/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetInteroperabilitySuite.scala:178: method getType in class ColumnDescriptor is deprecated: see corresponding Javadoc for more information.
[warn]                 assert(oneFooter.getFileMetaData.getSchema.getColumns.get(0).getType() ===
[warn]                                                                              ^

[warn] /home/jenkins/workspace/spark-master-test-maven-hadoop-2.7/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetTest.scala:154: method readAllFootersInParallel in object ParquetFileReader is deprecated: see corresponding Javadoc for more information.
[warn]     ParquetFileReader.readAllFootersInParallel(configuration, fs.getFileStatus(path)).asScala.toSeq
[warn]                       ^

[warn] /home/jenkins/workspace/spark-master-test-maven-hadoop-2.7/sql/hive/src/test/java/org/apache/spark/sql/hive/test/Complex.java:679: warning: [cast] redundant cast to Complex
[warn]     Complex typedOther = (Complex)other;
[warn]                          ^
```

**mllib**:

```
[warn] Pruning sources from previous analysis, due to incompatible CompileSetup.
[warn] /home/jenkins/workspace/spark-master-test-maven-hadoop-2.7/mllib/src/test/scala/org/apache/spark/ml/recommendation/ALSSuite.scala:597: match may not be exhaustive.
[warn] It would fail on the following inputs: None, Some((x: Tuple2[?, ?] forSome x not in (?, ?)))
[warn]     val df = dfs.find {
[warn]                       ^
```

This PR does not target fix all of them since some look pretty tricky to fix and there look too many warnings including false positive (like deprecated API but it's used in its test, etc.)

## How was this patch tested?

Existing tests should cover this.

Author: hyukjinkwon <gurwls223@apache.org>

Closes #21975 from HyukjinKwon/remove-build-warnings.
2018-08-04 11:52:49 -05:00
Nihar Sheth 70462f291b [SPARK-24926][CORE] Ensure numCores is used consistently in all netty configurations
## What changes were proposed in this pull request?

Netty could just ignore user-provided configurations. In particular, spark.driver.cores would be ignored when considering the number of cores available to netty (which would usually just default to Runtime.availableProcessors() ). In transport configurations, the number of threads are based directly on how many cores the system believes it has available, and in yarn cluster mode this would generally overshoot the user-preferred value.

## How was this patch tested?

As this is mostly a configuration change, tests were done manually by adding spark-submit confs and verifying the number of threads started by netty was what was expected.

Passes scalastyle checks from dev/run-tests

Please review http://spark.apache.org/contributing.html before opening a pull request.

Author: Nihar Sheth <niharrsheth@gmail.com>

Closes #21885 from NiharS/usableCores.
2018-08-04 10:27:34 -05:00
Xingbo Jiang 92b48842b9 [SPARK-24954][CORE] Fail fast on job submit if run a barrier stage with dynamic resource allocation enabled
## What changes were proposed in this pull request?

We don't support run a barrier stage with dynamic resource allocation enabled, it shall lead to some confusing behaviors (eg. with dynamic resource allocation enabled, it may happen that we acquire some executors (but not enough to launch all the tasks in a barrier stage) and later release them due to executor idle time expire, and then acquire again).

We perform the check on job submit and fail fast if running a barrier stage with dynamic resource allocation enabled.

## How was this patch tested?

Added new test suite `BarrierStageOnSubmittedSuite` to cover all the fail fast cases that submitted a job containing one or more barrier stages.

Author: Xingbo Jiang <xingbo.jiang@databricks.com>

Closes #21915 from jiangxb1987/SPARK-24954.
2018-08-03 09:36:56 -07:00
Devaraj K 53ca9755db
[SPARK-25009][CORE] Standalone Cluster mode application submit is not working
## What changes were proposed in this pull request?

It seems 'doRunMain()' has been removed accidentally by other PR and due to that the application submission is not happening, this PR adds back the 'doRunMain()' for standalone cluster submission.

## How was this patch tested?

I verified it manually by submitting application in standalone cluster mode, all the applications are submitting to the Master with the change.

Author: Devaraj K <devaraj@apache.org>

Closes #21979 from devaraj-kavali/SPARK-25009.
2018-08-03 07:23:56 +00:00
Xingbo Jiang 29077a1d15 [SPARK-24795][CORE][FOLLOWUP] Combine BarrierTaskContext with BarrierTaskContextImpl
## What changes were proposed in this pull request?

According to https://github.com/apache/spark/pull/21758#discussion_r206746905 , current declaration of `BarrierTaskContext` didn't extend methods from `TaskContext`. Since `TaskContext` is an abstract class and we don't want to change it to a trait, we have to define class `BarrierTaskContext` directly.

## How was this patch tested?

Existing tests.

Author: Xingbo Jiang <xingbo.jiang@databricks.com>

Closes #21972 from jiangxb1987/BarrierTaskContext.
2018-08-02 17:19:42 -07:00
Xingbo Jiang 38e4699c97 [SPARK-24820][SPARK-24821][CORE] Fail fast when submitted job contains a barrier stage with unsupported RDD chain pattern
## What changes were proposed in this pull request?

Check on job submit to make sure we don't launch a barrier stage with unsupported RDD chain pattern. The following patterns are not supported:
- Ancestor RDDs that have different number of partitions from the resulting RDD (eg. union()/coalesce()/first()/PartitionPruningRDD);
- An RDD that depends on multiple barrier RDDs (eg. barrierRdd1.zip(barrierRdd2)).

## How was this patch tested?

Add test cases in `BarrierStageOnSubmittedSuite`.

Author: Xingbo Jiang <xingbo.jiang@databricks.com>

Closes #21927 from jiangxb1987/SPARK-24820.
2018-08-02 09:36:26 -07:00
LucaCanali 15fc237226 Updates to Accumulators 2018-08-02 10:03:22 -05:00
Stavros Kontopoulos a65736996b [SPARK-14540][CORE] Fix remaining major issues for Scala 2.12 Support
## What changes were proposed in this pull request?
This PR addresses issues 2,3 in this [document](https://docs.google.com/document/d/1fbkjEL878witxVQpOCbjlvOvadHtVjYXeB-2mgzDTvk).

* We modified the closure cleaner to identify closures that are implemented via the LambdaMetaFactory mechanism (serializedLambdas) (issue2).

* We also fix the issue due to scala/bug#11016. There are two options for solving the Unit issue, either add () at the end of the closure or use the trick described in the doc. Otherwise overloading resolution does not work (we are not going to eliminate either of the methods) here. Compiler tries to adapt to Unit and makes these two methods candidates for overloading, when there is polymorphic overloading there is no ambiguity (that is the workaround implemented). This does not look that good but it serves its purpose as we need to support two different uses for method: `addTaskCompletionListener`. One that passes a TaskCompletionListener and one that passes a closure that is wrapped with a TaskCompletionListener later on (issue3).

Note: regarding issue 1 in the doc the plan is:

> Do Nothing. Don’t try to fix this as this is only a problem for Java users who would want to use 2.11 binaries. In that case they can cast to MapFunction to be able to utilize lambdas. In Spark 3.0.0 the API should be simplified so that this issue is removed.

## How was this patch tested?
This was manually tested:
```./dev/change-scala-version.sh 2.12
./build/mvn -DskipTests -Pscala-2.12 clean package
./build/mvn -Pscala-2.12 clean package -DwildcardSuites=org.apache.spark.serializer.ProactiveClosureSerializationSuite -Dtest=None
./build/mvn -Pscala-2.12 clean package -DwildcardSuites=org.apache.spark.util.ClosureCleanerSuite -Dtest=None
./build/mvn -Pscala-2.12 clean package -DwildcardSuites=org.apache.spark.streaming.DStreamClosureSuite -Dtest=None```

Author: Stavros Kontopoulos <stavros.kontopoulos@lightbend.com>

Closes #21930 from skonto/scala2.12-sup.
2018-08-02 09:17:09 -05:00
Xingbo Jiang 275415777b [SPARK-24795][CORE][FOLLOWUP] Kill all running tasks when a task in a barrier stage fail
## What changes were proposed in this pull request?

Kill all running tasks when a task in a barrier stage fail in the middle. `TaskScheduler`.`cancelTasks()` will also fail the job, so we implemented a new method `killAllTaskAttempts()` to just kill all running tasks of a stage without cancel the stage/job.

## How was this patch tested?

Add new test cases in `TaskSchedulerImplSuite`.

Author: Xingbo Jiang <xingbo.jiang@databricks.com>

Closes #21943 from jiangxb1987/killAllTasks.
2018-08-02 20:54:36 +08:00
Kazuaki Ishizaki 95a9d5e3a5 [SPARK-23915][SQL] Add array_except function
## What changes were proposed in this pull request?

The PR adds the SQL function `array_except`. The behavior of the function is based on Presto's one.

This function returns returns an array of the elements in array1 but not in array2.

Note: The order of elements in the result is not defined.

## How was this patch tested?

Added UTs.

Author: Kazuaki Ishizaki <ishizaki@jp.ibm.com>

Closes #21103 from kiszk/SPARK-23915.
2018-08-02 02:52:30 +08:00
Marcelo Vanzin 1122754bd9 [SPARK-24653][TESTS] Avoid cross-job pollution in TestUtils / SpillListener.
There is a narrow race in this code that is caused when the code being
run in assertSpilled / assertNotSpilled runs more than a single job.

SpillListener assumed that only a single job was run, and so would only
block waiting for that single job to finish when `numSpilledStages` was
called. But some tests (like SQL tests that call `checkAnswer`) run more
than one job, and so that wait was basically a no-op.

This could cause the next test to install a listener to receive events
from the previous job. Which could cause test failures in certain cases.

The change fixes that race, and also uninstalls listeners after the
test runs, so they don't accumulate when the SparkContext is shared
among multiple tests.

Author: Marcelo Vanzin <vanzin@cloudera.com>

Closes #21639 from vanzin/SPARK-24653.
2018-08-01 15:47:46 +08:00
Xingbo Jiang 3695ba5773 [MINOR][CORE][TEST] Fix afterEach() in TastSetManagerSuite and TaskSchedulerImplSuite
## What changes were proposed in this pull request?

In the `afterEach()` method of both `TastSetManagerSuite` and `TaskSchedulerImplSuite`, `super.afterEach()` shall be called at the end, because it shall stop the SparkContext.

https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/93706/testReport/org.apache.spark.scheduler/TaskSchedulerImplSuite/_It_is_not_a_test_it_is_a_sbt_testing_SuiteSelector_/
The test failure is caused by the above reason, the newly added `barrierCoordinator` required `rpcEnv` which has been stopped before `TaskSchedulerImpl` doing cleanup.

## How was this patch tested?
Existing tests.

Author: Xingbo Jiang <xingbo.jiang@databricks.com>

Closes #21908 from jiangxb1987/afterEach.
2018-07-30 09:58:28 +08:00
Hieu Huynh 5828f41a52 [SPARK-13343] speculative tasks that didn't commit shouldn't be marked as success
**Description**
Currently Speculative tasks that didn't commit can show up as success (depending on timing of commit). This is a bit confusing because that task didn't really succeed in the sense it didn't write anything.
I think these tasks should be marked as KILLED or something that is more obvious to the user exactly what happened. it is happened to hit the timing where it got a commit denied exception then it shows up as failed and counts against your task failures. It shouldn't count against task failures since that failure really doesn't matter.
MapReduce handles these situation so perhaps we can look there for a model.

<img width="1420" alt="unknown" src="https://user-images.githubusercontent.com/15680678/42013170-99db48c2-7a61-11e8-8c7b-ef94c84e36ea.png">

**How can this issue happen?**
When both attempts of a task finish before the driver sends command to kill one of them, both of them send the status update FINISHED to the driver. The driver calls TaskSchedulerImpl to handle one successful task at a time. When it handles the first successful task, it sends the command to kill the other copy of the task, however, because that task is already finished, the executor will ignore the command. After finishing handling the first attempt, it processes the second one, although all actions on the result of this task are skipped, this copy of the task is still marked as SUCCESS. As a result, even though this issue does not affect the result of the job, it might cause confusing to user because both of them appear to be successful.

**How does this PR fix the issue?**
The simple way to fix this issue is that when taskSetManager handles successful task, it checks if any other attempt succeeded. If this is the case, it will call handleFailedTask with state==KILLED and reason==TaskKilled(“another attempt succeeded”) to handle this task as begin killed.

**How was this patch tested?**
I tested this manually by running applications, that caused the issue before, a few times, and observed that the issue does not happen again. Also, I added a unit test in TaskSetManagerSuite to test that if we call handleSuccessfulTask to handle status update for 2 copies of a task, only the one that is handled first will be mark as SUCCESS

Author: Hieu Huynh <“Hieu.huynh@oath.com”>
Author: hthuynh2 <hthieu96@gmail.com>

Closes #21653 from hthuynh2/SPARK_13343.
2018-07-27 12:34:14 -05:00
Imran Rashid 2c82745686 [SPARK-24307][CORE] Add conf to revert to old code.
In case there are any issues in converting FileSegmentManagedBuffer to
ChunkedByteBuffer, add a conf to go back to old code path.

Followup to 7e847646d1

Author: Imran Rashid <irashid@cloudera.com>

Closes #21867 from squito/SPARK-24307-p2.
2018-07-26 12:13:27 -07:00
Xingbo Jiang e3486e1b95 [SPARK-24795][CORE] Implement barrier execution mode
## What changes were proposed in this pull request?

Propose new APIs and modify job/task scheduling to support barrier execution mode, which requires all tasks in a same barrier stage start at the same time, and retry all tasks in case some tasks fail in the middle. The barrier execution mode is useful for some ML/DL workloads.

The proposed API changes include:

- `RDDBarrier` that marks an RDD as barrier (Spark must launch all the tasks together for the current stage).
- `BarrierTaskContext` that support global sync of all tasks in a barrier stage, and provide extra `BarrierTaskInfo`s.

In DAGScheduler, we retry all tasks of a barrier stage in case some tasks fail in the middle, this is achieved by unregistering map outputs for a shuffleId (for ShuffleMapStage) or clear the finished partitions in an active job (for ResultStage).

## How was this patch tested?

Add `RDDBarrierSuite` to ensure we convert RDDs correctly;
Add new test cases in `DAGSchedulerSuite` to ensure we do task scheduling correctly;
Add new test cases in `SparkContextSuite` to ensure the barrier execution mode actually works (both under local mode and local cluster mode).
Add new test cases in `TaskSchedulerImplSuite` to ensure we schedule tasks for barrier taskSet together.

Author: Xingbo Jiang <xingbo.jiang@databricks.com>

Closes #21758 from jiangxb1987/barrier-execution-mode.
2018-07-26 12:09:01 -07:00
Imran Rashid 15fff79032 [SPARK-24297][CORE] Fetch-to-disk by default for > 2gb
Fetch-to-mem is guaranteed to fail if the message is bigger than 2 GB,
so we might as well use fetch-to-disk in that case.  The message includes
some metadata in addition to the block data itself (in particular
UploadBlock has a lot of metadata), so we leave a little room.

Author: Imran Rashid <irashid@cloudera.com>

Closes #21474 from squito/SPARK-24297.
2018-07-25 09:08:42 +08:00
Yuanjian Li 7db81ac8a2 [SPARK-24195][CORE] Ignore the files with "local" scheme in SparkContext.addFile
## What changes were proposed in this pull request?

In Spark "local" scheme means resources are already on the driver/executor nodes, this pr ignore the files with "local" scheme in `SparkContext.addFile` for fixing potential bug.

## How was this patch tested?

Existing tests.

Author: Yuanjian Li <xyliyuanjian@gmail.com>

Closes #21533 from xuanyuanking/SPARK-24195.
2018-07-20 11:25:51 +08:00
Imran Rashid 7e847646d1 [SPARK-24307][CORE] Support reading remote cached partitions > 2gb
(1) Netty's ByteBuf cannot support data > 2gb.  So to transfer data from a
ChunkedByteBuffer over the network, we use a custom version of
FileRegion which is backed by the ChunkedByteBuffer.

(2) On the receiving end, we need to expose all the data in a
FileSegmentManagedBuffer as a ChunkedByteBuffer.  We do that by memory
mapping the entire file in chunks.

Added unit tests.  Ran the randomized test a couple of hundred times on my laptop.  Tests cover the equivalent of SPARK-24107 for the ChunkedByteBufferFileRegion.  Also tested on a cluster with remote cache reads >2gb (in memory and on disk).

Author: Imran Rashid <irashid@cloudera.com>

Closes #21440 from squito/chunked_bb_file_region.
2018-07-20 11:16:53 +08:00
Hieu Huynh 8d707b0600 [SPARK-24755][CORE] Executor loss can cause task to not be resubmitted
**Description**
As described in [SPARK-24755](https://issues.apache.org/jira/browse/SPARK-24755), when speculation is enabled, there is scenario that executor loss can cause task to not be resubmitted.
This patch changes the variable killedByOtherAttempt to keeps track of the taskId of tasks that are killed by other attempt. By doing this, we can still prevent resubmitting task killed by other attempt while resubmit successful attempt when executor lost.

**How was this patch tested?**
A UT is added based on the UT written by xuanyuanking with modification to simulate the scenario described in SPARK-24755.

Author: Hieu Huynh <“Hieu.huynh@oath.com”>

Closes #21729 from hthuynh2/SPARK_24755.
2018-07-19 09:52:07 -05:00
sychen c8bee932cb [SPARK-24677][CORE] Avoid NoSuchElementException from MedianHeap
## What changes were proposed in this pull request?
When speculation is enabled,
TaskSetManager#markPartitionCompleted should write successful task duration to MedianHeap,
not just increase tasksSuccessful.

Otherwise when TaskSetManager#checkSpeculatableTasks,tasksSuccessful non-zero, but MedianHeap is empty.
Then throw an exception successfulTaskDurations.median java.util.NoSuchElementException: MedianHeap is empty.
Finally led to stopping SparkContext.
## How was this patch tested?
TaskSetManagerSuite.scala
unit test:[SPARK-24677] MedianHeap should not be empty when speculation is enabled

Author: sychen <sychen@ctrip.com>

Closes #21656 from cxzl25/fix_MedianHeap_empty.
2018-07-18 13:24:41 -05:00
Nihar Sheth 2694dd2bf0 [MINOR][CORE] Add test cases for RDD.cartesian
## What changes were proposed in this pull request?

While looking through the codebase, it appeared that the scala code for RDD.cartesian does not have any tests for correctness. This adds a couple basic tests to verify cartesian yields correct values. While the implementation for RDD.cartesian is pretty simple, it always helps to have a few tests!

## How was this patch tested?

The new test cases pass, and the scala style tests from running dev/run-tests all pass.

Please review http://spark.apache.org/contributing.html before opening a pull request.

Author: Nihar Sheth <niharrsheth@gmail.com>

Closes #21765 from NiharS/cartesianTests.
2018-07-18 09:14:36 -05:00
sandeep-katta 2603ae30be [SPARK-24558][CORE] wrong Idle Timeout value is used in case of the cacheBlock.
It is corrected as per the configuration.

## What changes were proposed in this pull request?
IdleTimeout info used to print in the logs is taken based on the cacheBlock. If it is cacheBlock then cachedExecutorIdleTimeoutS is considered else executorIdleTimeoutS

## How was this patch tested?
Manual Test
spark-sql> cache table sample;
2018-05-15 14:44:02 INFO  DAGScheduler:54 - Submitting 3 missing tasks from ShuffleMapStage 0 (MapPartitionsRDD[8] at processCmd at CliDriver.java:376) (first 15 tasks are for partitions Vector(0, 1, 2))
2018-05-15 14:44:02 INFO  YarnScheduler:54 - Adding task set 0.0 with 3 tasks
2018-05-15 14:44:03 INFO  ExecutorAllocationManager:54 - Requesting 1 new executor because tasks are backlogged (new desired total will be 1)
...
...
2018-05-15 14:46:10 INFO  YarnClientSchedulerBackend:54 - Actual list of executor(s) to be killed is 1
2018-05-15 14:46:10 INFO  **ExecutorAllocationManager:54 - Removing executor 1 because it has been idle for 120 seconds (new desired total will be 0)**
2018-05-15 14:46:11 INFO  YarnSchedulerBackend$YarnDriverEndpoint:54 - Disabling executor 1.
2018-05-15 14:46:11 INFO  DAGScheduler:54 - Executor lost: 1 (epoch 1)

Author: sandeep-katta <sandeep.katta2007@gmail.com>

Closes #21565 from sandeep-katta/loginfoBug.
2018-07-16 14:52:49 +08:00
Maxim Gekk 69993217fc [SPARK-24807][CORE] Adding files/jars twice: output a warning and add a note
## What changes were proposed in this pull request?

In the PR, I propose to output an warning if the `addFile()` or `addJar()` methods are callled more than once for the same path. Currently, overwriting of already added files is not supported. New comments and warning are reflected the existing behaviour.

Author: Maxim Gekk <maxim.gekk@databricks.com>

Closes #21771 from MaxGekk/warning-on-adding-file.
2018-07-14 22:07:49 -07:00
Dhruve Ashar 1055c94cdf [SPARK-24610] fix reading small files via wholeTextFiles
## What changes were proposed in this pull request?
The `WholeTextFileInputFormat` determines the `maxSplitSize` for the file/s being read using the `wholeTextFiles` method. While this works well for large files, for smaller files where the maxSplitSize is smaller than the defaults being used with configs like hive-site.xml or explicitly passed in the form of `mapreduce.input.fileinputformat.split.minsize.per.node` or `mapreduce.input.fileinputformat.split.minsize.per.rack` , it just throws up an exception.

```java
java.io.IOException: Minimum split size pernode 123456 cannot be larger than maximum split size 9962
at org.apache.hadoop.mapreduce.lib.input.CombineFileInputFormat.getSplits(CombineFileInputFormat.java:200)
at org.apache.spark.rdd.WholeTextFileRDD.getPartitions(WholeTextFileRDD.scala:50)
at org.apache.spark.rdd.RDD$$anonfun$partitions$2.apply(RDD.scala:252)
at org.apache.spark.rdd.RDD$$anonfun$partitions$2.apply(RDD.scala:250)
at scala.Option.getOrElse(Option.scala:121)
at org.apache.spark.rdd.RDD.partitions(RDD.scala:250)
at org.apache.spark.rdd.MapPartitionsRDD.getPartitions(MapPartitionsRDD.scala:35)
at org.apache.spark.rdd.RDD$$anonfun$partitions$2.apply(RDD.scala:252)
at org.apache.spark.rdd.RDD$$anonfun$partitions$2.apply(RDD.scala:250)
at scala.Option.getOrElse(Option.scala:121)
at org.apache.spark.rdd.RDD.partitions(RDD.scala:250)
at org.apache.spark.SparkContext.runJob(SparkContext.scala:2096)
at org.apache.spark.rdd.RDD.count(RDD.scala:1158)
... 48 elided
`

This change checks the maxSplitSize against the minSplitSizePerNode and minSplitSizePerRack and set them if `maxSplitSize < minSplitSizePerNode/Rack`

## How was this patch tested?
Test manually setting the conf while launching the job and added unit test.

Author: Dhruve Ashar <dhruveashar@gmail.com>

Closes #21601 from dhruve/bug/SPARK-24610.
2018-07-12 15:36:02 -05:00
Kazuaki Ishizaki 5ad4735bda [SPARK-24529][BUILD][TEST-MAVEN] Add spotbugs into maven build process
## What changes were proposed in this pull request?

This PR enables a Java bytecode check tool [spotbugs](https://spotbugs.github.io/) to avoid possible integer overflow at multiplication. When an violation is detected, the build process is stopped.
Due to the tool limitation, some other checks will be enabled. In this PR, [these patterns](http://spotbugs-in-kengo-toda.readthedocs.io/en/lqc-list-detectors/detectors.html#findpuzzlers) in `FindPuzzlers` can be detected.

This check is enabled at `compile` phase. Thus, `mvn compile` or `mvn package` launches this check.

## How was this patch tested?

Existing UTs

Author: Kazuaki Ishizaki <ishizaki@jp.ibm.com>

Closes #21542 from kiszk/SPARK-24529.
2018-07-12 09:52:23 +08:00
Rekha Joshi 290c30a53f [SPARK-24470][CORE] RestSubmissionClient to be robust against 404 & non json responses
## What changes were proposed in this pull request?
Added check for 404, to avoid json parsing on not found response and to avoid returning malformed or bad request when it was a not found http response.
Not sure if I need to add an additional check on non json response [if(connection.getHeaderField("Content-Type").contains("text/html")) then exception] as non-json is a subset of malformed json and covered in flow.

## How was this patch tested?
./dev/run-tests

Author: Rekha Joshi <rekhajoshm@gmail.com>

Closes #21684 from rekhajoshm/SPARK-24470.
2018-07-11 13:48:28 -05:00
sharkdtu 6fe32869cc [SPARK-24678][SPARK-STREAMING] Give priority in use of 'PROCESS_LOCAL' for spark-streaming
## What changes were proposed in this pull request?

Currently, `BlockRDD.getPreferredLocations`  only get hosts info of blocks, which results in subsequent schedule level is not better than 'NODE_LOCAL'. We can just make a small changes, the schedule level can be improved to 'PROCESS_LOCAL'

## How was this patch tested?

manual test

Author: sharkdtu <sharkdtu@tencent.com>

Closes #21658 from sharkdtu/master.
2018-07-10 20:18:34 +08:00
jerryshao e2c7e09f74 [SPARK-24646][CORE] Minor change to spark.yarn.dist.forceDownloadSchemes to support wildcard '*'
## What changes were proposed in this pull request?

In the case of getting tokens via customized `ServiceCredentialProvider`, it is required that `ServiceCredentialProvider` be available in local spark-submit process classpath. In this case, all the configured remote sources should be forced to download to local.

For the ease of using this configuration, here propose to add wildcard '*' support to `spark.yarn.dist.forceDownloadSchemes`, also clarify the usage of this configuration.

## How was this patch tested?

New UT added.

Author: jerryshao <sshao@hortonworks.com>

Closes #21633 from jerryshao/SPARK-21917-followup.
2018-07-09 10:21:40 +08:00
Michael Mior e58dadb77e [SPARK-23820][CORE] Enable use of long form of callsite in logs
This adds an option to event logging to include the long form of the callsite instead of the short form.

Author: Michael Mior <mmior@uwaterloo.ca>

Closes #21433 from michaelmior/long-callsite.
2018-07-05 08:32:20 -05:00
Stan Zhai 772060d094 [SPARK-24704][WEBUI] Fix the order of stages in the DAG graph
## What changes were proposed in this pull request?

Before:

![wx20180630-155537](https://user-images.githubusercontent.com/1438757/42123357-2c2e2d84-7c83-11e8-8abd-1c2860f38783.png)

After:

![wx20180630-155604](https://user-images.githubusercontent.com/1438757/42123359-32fae990-7c83-11e8-8a7b-cdcee94f9123.png)

## How was this patch tested?

Manual tests.

Author: Stan Zhai <mail@stanzhai.site>

Closes #21680 from stanzhai/fix-dag-graph.
2018-07-04 10:12:36 +02:00
DB Tsai 5585c5765f
[SPARK-24420][BUILD] Upgrade ASM to 6.1 to support JDK9+
## What changes were proposed in this pull request?

Upgrade ASM to 6.1 to support JDK9+

## How was this patch tested?

Existing tests.

Author: DB Tsai <d_tsai@apple.com>

Closes #21459 from dbtsai/asm.
2018-07-03 10:13:48 -07:00
mcheah 42815548c7 [SPARK-24683][K8S] Fix k8s no resource
## What changes were proposed in this pull request?

Make SparkSubmit pass in the main class even if `SparkLauncher.NO_RESOURCE` is the primary resource.

## How was this patch tested?

New integration test written to capture this case.

Author: mcheah <mcheah@palantir.com>

Closes #21660 from mccheah/fix-k8s-no-resource.
2018-07-02 10:24:04 -07:00
xueyu f71e8da5ef [SPARK-24566][CORE] Fix spark.storage.blockManagerSlaveTimeoutMs default config
This PR use spark.network.timeout in place of spark.storage.blockManagerSlaveTimeoutMs when it is not configured, as configuration doc said

manual test

Author: xueyu <278006819@qq.com>

Closes #21575 from xueyumusic/slaveTimeOutConfig.
2018-06-29 10:44:49 -07:00
Xingbo Jiang 5b05966488 [SPARK-24564][TEST] Add test suite for RecordBinaryComparator
## What changes were proposed in this pull request?

Add a new test suite to test RecordBinaryComparator.

## How was this patch tested?

New test suite.

Author: Xingbo Jiang <xingbo.jiang@databricks.com>

Closes #21570 from jiangxb1987/rbc-test.
2018-06-28 14:19:50 +08:00
Kallman, Steven c5aa54d54b [SPARK-24553][WEB-UI] http 302 fixes for href redirect
## What changes were proposed in this pull request?

Updated URL/href links to include a '/' before '?id' to make links consistent and avoid http 302 redirect errors within UI port 4040 tabs.

## How was this patch tested?

Built a runnable distribution and executed jobs. Validated that http 302 redirects are no longer encountered when clicking on links within UI port 4040 tabs.

Author: Steven Kallman <SJKallmangmail.com>

Author: Kallman, Steven <Steven.Kallman@CapitalOne.com>

Closes #21600 from SJKallman/{Spark-24553}{WEB-UI}-redirect-href-fixes.
2018-06-27 15:36:59 -07:00
Marco Gaido 776befbfd5 [SPARK-24660][SHS] Show correct error pages when downloading logs
## What changes were proposed in this pull request?

SHS is showing bad errors when trying to download logs is not successful. This may happen because the requested application doesn't exist or the user doesn't have permissions for it, for instance.

The PR fixes the response when errors occur, so that they are displayed properly.

## How was this patch tested?

manual tests

**Before the patch:**
 1. Unauthorized user
![screen shot 2018-06-26 at 3 53 33 pm](https://user-images.githubusercontent.com/8821783/41918118-f8b37e70-795b-11e8-91e8-d0250239f09d.png)

 2. Non-existing application
![screen shot 2018-06-26 at 3 25 19 pm](https://user-images.githubusercontent.com/8821783/41918082-e3034c72-795b-11e8-970e-cee4a1eae77f.png)

**After the patch**
 1. Unauthorized user
![screen shot 2018-06-26 at 3 41 29 pm](https://user-images.githubusercontent.com/8821783/41918155-0d950476-795c-11e8-8d26-7b7ce73e6fe1.png)

 2. Non-existing application
![screen shot 2018-06-26 at 3 40 37 pm](https://user-images.githubusercontent.com/8821783/41918175-1a14bb88-795c-11e8-91ab-eadf29190a02.png)

Author: Marco Gaido <marcogaido91@gmail.com>

Closes #21644 from mgaido91/SPARK-24660.
2018-06-27 14:26:08 -07:00
Marcelo Vanzin 6d16b9885d [SPARK-24552][CORE][SQL] Use task ID instead of attempt number for writes.
This passes the unique task attempt id instead of attempt number to v2 data sources because attempt number is reused when stages are retried. When attempt numbers are reused, sources that track data by partition id and attempt number may incorrectly clean up data because the same attempt number can be both committed and aborted.

For v1 / Hadoop writes, generate a unique ID based on available attempt numbers to avoid a similar problem.

Closes #21558

Author: Marcelo Vanzin <vanzin@cloudera.com>
Author: Ryan Blue <blue@apache.org>

Closes #21606 from vanzin/SPARK-24552.2.
2018-06-25 16:54:57 -07:00
Takeshi Yamamuro f596ebe4d3 [SPARK-24327][SQL] Verify and normalize a partition column name based on the JDBC resolved schema
## What changes were proposed in this pull request?
This pr modified JDBC datasource code to verify and normalize a partition column based on the JDBC resolved schema before building `JDBCRelation`.

Closes #20370

## How was this patch tested?
Added tests in `JDBCSuite`.

Author: Takeshi Yamamuro <yamamuro@apache.org>

Closes #21379 from maropu/SPARK-24327.
2018-06-24 23:14:42 -07:00
jerryshao 33e77fa89b [SPARK-24518][CORE] Using Hadoop credential provider API to store password
## What changes were proposed in this pull request?

In our distribution,  because we don't do such fine-grained access control of config file, also configuration file is world readable shared between different components, so password may leak to different users.

Hadoop credential provider API support storing password in a secure way, in which Spark could read it in a secure way, so here propose to add support of using credential provider API to get password.

## How was this patch tested?

Adding tests and verified locally.

Author: jerryshao <sshao@hortonworks.com>

Closes #21548 from jerryshao/SPARK-24518.
2018-06-22 10:14:12 -07:00
Hieu Huynh 39dfaf2fd1 [SPARK-24519] Make the threshold for highly compressed map status configurable
**Problem**
MapStatus uses hardcoded value of 2000 partitions to determine if it should use highly compressed map status. We should make it configurable to allow users to more easily tune their jobs with respect to this without having for them to modify their code to change the number of partitions.  Note we can leave this as an internal/undocumented config for now until we have more advise for the users on how to set this config.
Some of my reasoning:
The config gives you a way to easily change something without the user having to change code, redeploy jar, and then run again. You can simply change the config and rerun. It also allows for easier experimentation. Changing the # of partitions has other side affects, whether good or bad is situation dependent. It can be worse are you could be increasing # of output files when you don't want to be, affects the # of tasks needs and thus executors to run in parallel, etc.
There have been various talks about this number at spark summits where people have told customers to increase it to be 2001 partitions. Note if you just do a search for spark 2000 partitions you will fine various things all talking about this number.  This shows that people are modifying their code to take this into account so it seems to me having this configurable would be better.
Once we have more advice for users we could expose this and document information on it.

**What changes were proposed in this pull request?**
I make the hardcoded value mentioned above to be configurable under the name _SHUFFLE_MIN_NUM_PARTS_TO_HIGHLY_COMPRESS_, which has default value to be 2000. Users can set it to the value they want by setting the property name _spark.shuffle.minNumPartitionsToHighlyCompress_

**How was this patch tested?**
I wrote a unit test to make sure that the default value is 2000, and  _IllegalArgumentException_ will be thrown if user set it to a non-positive value. The unit test also checks that highly compressed map status is correctly used when the number of partition is greater than _SHUFFLE_MIN_NUM_PARTS_TO_HIGHLY_COMPRESS_.

Author: Hieu Huynh <“Hieu.huynh@oath.com”>

Closes #21527 from hthuynh2/spark_branch_1.
2018-06-22 09:16:14 -05:00
Marcelo Vanzin c8e909cd49 [SPARK-24589][CORE] Correctly identify tasks in output commit coordinator.
When an output stage is retried, it's possible that tasks from the previous
attempt are still running. In that case, there would be a new task for the
same partition in the new attempt, and the coordinator would allow both
tasks to commit their output since it did not keep track of stage attempts.

The change adds more information to the stage state tracked by the coordinator,
so that only one task is allowed to commit the output in the above case.
The stage state in the coordinator is also maintained across stage retries,
so that a stray speculative task from a previous stage attempt is not allowed
to commit.

This also removes some code added in SPARK-18113 that allowed for duplicate
commit requests; with the RPC code used in Spark 2, that situation cannot
happen, so there is no need to handle it.

Author: Marcelo Vanzin <vanzin@cloudera.com>

Closes #21577 from vanzin/SPARK-24552.
2018-06-21 13:25:15 -05:00
“attilapiros” b56e9c613f [SPARK-16630][YARN] Blacklist a node if executors won't launch on it
## What changes were proposed in this pull request?

This change extends YARN resource allocation handling with blacklisting functionality.
This handles cases when node is messed up or misconfigured such that a container won't launch on it. Before this change backlisting only focused on task execution but this change introduces YarnAllocatorBlacklistTracker which tracks allocation failures per host (when enabled via "spark.yarn.blacklist.executor.launch.blacklisting.enabled").

## How was this patch tested?

### With unit tests

Including a new suite: YarnAllocatorBlacklistTrackerSuite.

#### Manually

It was tested on a cluster by deleting the Spark jars on one of the node.

#### Behaviour before these changes

Starting Spark as:
```
spark2-shell --master yarn --deploy-mode client --num-executors 4  --conf spark.executor.memory=4g --conf "spark.yarn.max.executor.failures=6"
```

Log is:
```
18/04/12 06:49:36 INFO yarn.ApplicationMaster: Final app status: FAILED, exitCode: 11, (reason: Max number of executor failures (6) reached)
18/04/12 06:49:39 INFO yarn.ApplicationMaster: Unregistering ApplicationMaster with FAILED (diag message: Max number of executor failures (6) reached)
18/04/12 06:49:39 INFO impl.AMRMClientImpl: Waiting for application to be successfully unregistered.
18/04/12 06:49:39 INFO yarn.ApplicationMaster: Deleting staging directory hdfs://apiros-1.gce.test.com:8020/user/systest/.sparkStaging/application_1523459048274_0016
18/04/12 06:49:39 INFO util.ShutdownHookManager: Shutdown hook called
```

#### Behaviour after these changes

Starting Spark as:
```
spark2-shell --master yarn --deploy-mode client --num-executors 4  --conf spark.executor.memory=4g --conf "spark.yarn.max.executor.failures=6" --conf "spark.yarn.blacklist.executor.launch.blacklisting.enabled=true"
```

And the log is:
```
18/04/13 05:37:43 INFO yarn.YarnAllocator: Will request 1 executor container(s), each with 1 core(s) and 4505 MB memory (including 409 MB of overhead)
18/04/13 05:37:43 INFO yarn.YarnAllocator: Submitted 1 unlocalized container requests.
18/04/13 05:37:43 INFO yarn.YarnAllocator: Launching container container_1523459048274_0025_01_000008 on host apiros-4.gce.test.com for executor with ID 6
18/04/13 05:37:43 INFO yarn.YarnAllocator: Received 1 containers from YARN, launching executors on 1 of them.
18/04/13 05:37:43 INFO yarn.YarnAllocator: Completed container container_1523459048274_0025_01_000007 on host: apiros-4.gce.test.com (state: COMPLETE, exit status: 1)
18/04/13 05:37:43 INFO yarn.YarnAllocatorBlacklistTracker: blacklisting host as YARN allocation failed: apiros-4.gce.test.com
18/04/13 05:37:43 INFO yarn.YarnAllocatorBlacklistTracker: adding nodes to YARN application master's blacklist: List(apiros-4.gce.test.com)
18/04/13 05:37:43 WARN yarn.YarnAllocator: Container marked as failed: container_1523459048274_0025_01_000007 on host: apiros-4.gce.test.com. Exit status: 1. Diagnostics: Exception from container-launch.
Container id: container_1523459048274_0025_01_000007
Exit code: 1
Stack trace: ExitCodeException exitCode=1:
        at org.apache.hadoop.util.Shell.runCommand(Shell.java:604)
        at org.apache.hadoop.util.Shell.run(Shell.java:507)
        at org.apache.hadoop.util.Shell$ShellCommandExecutor.execute(Shell.java:789)
        at org.apache.hadoop.yarn.server.nodemanager.DefaultContainerExecutor.launchContainer(DefaultContainerExecutor.java:213)
        at org.apache.hadoop.yarn.server.nodemanager.containermanager.launcher.ContainerLaunch.call(ContainerLaunch.java:302)
        at org.apache.hadoop.yarn.server.nodemanager.containermanager.launcher.ContainerLaunch.call(ContainerLaunch.java:82)
        at java.util.concurrent.FutureTask.run(FutureTask.java:266)
        at java.util.concurrent.ThreadPoolExecutor.runWorker(ThreadPoolExecutor.java:1149)
        at java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:624)
        at java.lang.Thread.run(Thread.java:748)
```

Where the most important part is:

```
18/04/13 05:37:43 INFO yarn.YarnAllocatorBlacklistTracker: blacklisting host as YARN allocation failed: apiros-4.gce.test.com
18/04/13 05:37:43 INFO yarn.YarnAllocatorBlacklistTracker: adding nodes to YARN application master's blacklist: List(apiros-4.gce.test.com)
```

And execution was continued (no shutdown called).

### Testing the backlisting of the whole cluster

Starting Spark with YARN blacklisting enabled then removing a the Spark core jar one by one from all the cluster nodes. Then executing a simple spark job which fails checking the yarn log the expected exit status is contained:

```
18/06/15 01:07:10 INFO yarn.ApplicationMaster: Final app status: FAILED, exitCode: 11, (reason: Due to executor failures all available nodes are blacklisted)
18/06/15 01:07:13 INFO util.ShutdownHookManager: Shutdown hook called
```

Author: “attilapiros” <piros.attila.zsolt@gmail.com>

Closes #21068 from attilapiros/SPARK-16630.
2018-06-21 09:17:18 -05:00
Marco Gaido bc111463a7 [SPARK-23778][CORE] Avoid unneeded shuffle when union gets an empty RDD
## What changes were proposed in this pull request?

When a `union` is invoked on several RDDs of which one is an empty RDD, the result of the operation is a `UnionRDD`. This causes an unneeded extra-shuffle when all the other RDDs have the same partitioning.

The PR ignores incoming empty RDDs in the union method.

## How was this patch tested?

added UT

Author: Marco Gaido <marcogaido91@gmail.com>

Closes #21333 from mgaido91/SPARK-23778.
2018-06-19 22:29:00 -07:00
Kazuaki Ishizaki 90da7dc241 [SPARK-24452][SQL][CORE] Avoid possible overflow in int add or multiple
## What changes were proposed in this pull request?

This PR fixes possible overflow in int add or multiply. In particular, their overflows in multiply are detected by [Spotbugs](https://spotbugs.github.io/)

The following assignments may cause overflow in right hand side. As a result, the result may be negative.
```
long = int * int
long = int + int
```

To avoid this problem, this PR performs cast from int to long in right hand side.

## How was this patch tested?

Existing UTs.

Author: Kazuaki Ishizaki <ishizaki@jp.ibm.com>

Closes #21481 from kiszk/SPARK-24452.
2018-06-15 13:47:48 -07:00
Jacek Laskowski 495d8cf09a [SPARK-24490][WEBUI] Use WebUI.addStaticHandler in web UIs
`WebUI` defines `addStaticHandler` that web UIs don't use (and simply introduce duplication). Let's clean them up and remove duplications.

Local build and waiting for Jenkins

Author: Jacek Laskowski <jacek@japila.pl>

Closes #21510 from jaceklaskowski/SPARK-24490-Use-WebUI.addStaticHandler.
2018-06-15 09:59:02 -07:00
mcheah 270a9a3cac [SPARK-24248][K8S] Use level triggering and state reconciliation in scheduling and lifecycle
## What changes were proposed in this pull request?

Previously, the scheduler backend was maintaining state in many places, not only for reading state but also writing to it. For example, state had to be managed in both the watch and in the executor allocator runnable. Furthermore, one had to keep track of multiple hash tables.

We can do better here by:

1. Consolidating the places where we manage state. Here, we take inspiration from traditional Kubernetes controllers. These controllers tend to follow a level-triggered mechanism. This means that the controller will continuously monitor the API server via watches and polling, and on periodic passes, the controller will reconcile the current state of the cluster with the desired state. We implement this by introducing the concept of a pod snapshot, which is a given state of the executors in the Kubernetes cluster. We operate periodically on snapshots. To prevent overloading the API server with polling requests to get the state of the cluster (particularly for executor allocation where we want to be checking frequently to get executors to launch without unbearably bad latency), we use watches to populate snapshots by applying observed events to a previous snapshot to get a new snapshot. Whenever we do poll the cluster, the polled state replaces any existing snapshot - this ensures eventual consistency and mirroring of the cluster, as is desired in a level triggered architecture.

2. Storing less specialized in-memory state in general. Previously we were creating hash tables to represent the state of executors. Instead, it's easier to represent state solely by the snapshots.

## How was this patch tested?

Integration tests should test there's no regressions end to end. Unit tests to be updated, in particular focusing on different orderings of events, particularly accounting for when events come in unexpected ordering.

Author: mcheah <mcheah@palantir.com>

Closes #21366 from mccheah/event-queue-driven-scheduling.
2018-06-14 15:56:21 -07:00
Xingbo Jiang 534065efeb [MINOR][CORE][TEST] Remove unnecessary sort in UnsafeInMemorySorterSuite
## What changes were proposed in this pull request?

We don't require specific ordering of the input data, the sort action is not necessary and misleading.

## How was this patch tested?

Existing test suite.

Author: Xingbo Jiang <xingbo.jiang@databricks.com>

Closes #21536 from jiangxb1987/sorterSuite.
2018-06-14 14:20:48 +08:00
Jungtaek Lim 4c388bccf1 [SPARK-24485][SS] Measure and log elapsed time for filesystem operations in HDFSBackedStateStoreProvider
## What changes were proposed in this pull request?

This patch measures and logs elapsed time for each operation which communicate with file system (mostly remote HDFS in production) in HDFSBackedStateStoreProvider to help investigating any latency issue.

## How was this patch tested?

Manually tested.

Author: Jungtaek Lim <kabhwan@gmail.com>

Closes #21506 from HeartSaVioR/SPARK-24485.
2018-06-13 12:36:20 +08:00
Li Jin 9786ce66c5 [SPARK-22239][SQL][PYTHON] Enable grouped aggregate pandas UDFs as window functions with unbounded window frames
## What changes were proposed in this pull request?
This PR enables using a grouped aggregate pandas UDFs as window functions. The semantics is the same as using SQL aggregation function as window functions.

```
       >>> from pyspark.sql.functions import pandas_udf, PandasUDFType
       >>> from pyspark.sql import Window
       >>> df = spark.createDataFrame(
       ...     [(1, 1.0), (1, 2.0), (2, 3.0), (2, 5.0), (2, 10.0)],
       ...     ("id", "v"))
       >>> pandas_udf("double", PandasUDFType.GROUPED_AGG)
       ... def mean_udf(v):
       ...     return v.mean()
       >>> w = Window.partitionBy('id')
       >>> df.withColumn('mean_v', mean_udf(df['v']).over(w)).show()
       +---+----+------+
       | id|   v|mean_v|
       +---+----+------+
       |  1| 1.0|   1.5|
       |  1| 2.0|   1.5|
       |  2| 3.0|   6.0|
       |  2| 5.0|   6.0|
       |  2|10.0|   6.0|
       +---+----+------+
```

The scope of this PR is somewhat limited in terms of:
(1) Only supports unbounded window, which acts essentially as group by.
(2) Only supports aggregation functions, not "transform" like window functions (n -> n mapping)

Both of these are left as future work. Especially, (1) needs careful thinking w.r.t. how to pass rolling window data to python efficiently. (2) is a bit easier but does require more changes therefore I think it's better to leave it as a separate PR.

## How was this patch tested?

WindowPandasUDFTests

Author: Li Jin <ice.xelloss@gmail.com>

Closes #21082 from icexelloss/SPARK-22239-window-udf.
2018-06-13 09:10:52 +08:00
Marco Gaido f53818d35b [SPARK-24506][UI] Add UI filters to tabs added after binding
## What changes were proposed in this pull request?

Currently, `spark.ui.filters` are not applied to the handlers added after binding the server. This means that every page which is added after starting the UI will not have the filters configured on it. This can allow unauthorized access to the pages.

The PR adds the filters also to the handlers added after the UI starts.

## How was this patch tested?

manual tests (without the patch, starting the thriftserver with `--conf spark.ui.filters=org.apache.hadoop.security.authentication.server.AuthenticationFilter --conf spark.org.apache.hadoop.security.authentication.server.AuthenticationFilter.params="type=simple"` you can access `http://localhost:4040/sqlserver`; with the patch, 401 is the response as for the other pages).

Author: Marco Gaido <marcogaido91@gmail.com>

Closes #21523 from mgaido91/SPARK-24506.
2018-06-12 16:42:44 -07:00
Fangshi Li cc88d7fad1 [SPARK-24216][SQL] Spark TypedAggregateExpression uses getSimpleName that is not safe in scala
## What changes were proposed in this pull request?

When user create a aggregator object in scala and pass the aggregator to Spark Dataset's agg() method, Spark's will initialize TypedAggregateExpression with the nodeName field as aggregator.getClass.getSimpleName. However, getSimpleName is not safe in scala environment, depending on how user creates the aggregator object. For example, if the aggregator class full qualified name is "com.my.company.MyUtils$myAgg$2$", the getSimpleName will throw java.lang.InternalError "Malformed class name". This has been reported in scalatest https://github.com/scalatest/scalatest/pull/1044 and discussed in many scala upstream jiras such as SI-8110, SI-5425.

To fix this issue, we follow the solution in https://github.com/scalatest/scalatest/pull/1044 to add safer version of getSimpleName as a util method, and TypedAggregateExpression will invoke this util method rather than getClass.getSimpleName.

## How was this patch tested?
added unit test

Author: Fangshi Li <fli@linkedin.com>

Closes #21276 from fangshil/SPARK-24216.
2018-06-12 12:10:08 -07:00
Jonathan Kelly 9b6f24202f [MINOR][CORE] Log committer class used by HadoopMapRedCommitProtocol
## What changes were proposed in this pull request?

When HadoopMapRedCommitProtocol is used (e.g., when using saveAsTextFile() or
saveAsHadoopFile() with RDDs), it's not easy to determine which output committer
class was used, so this PR simply logs the class that was used, similarly to what
is done in SQLHadoopMapReduceCommitProtocol.

## How was this patch tested?

Built Spark then manually inspected logging when calling saveAsTextFile():

```scala
scala> sc.setLogLevel("INFO")
scala> sc.textFile("README.md").saveAsTextFile("/tmp/out")
...
18/05/29 10:06:20 INFO HadoopMapRedCommitProtocol: Using output committer class org.apache.hadoop.mapred.FileOutputCommitter
```

Author: Jonathan Kelly <jonathak@amazon.com>

Closes #21452 from ejono/master.
2018-06-11 16:41:15 -05:00
Ilan Filonenko 1a644afbac [SPARK-23984][K8S] Initial Python Bindings for PySpark on K8s
## What changes were proposed in this pull request?

Introducing Python Bindings for PySpark.

- [x] Running PySpark Jobs
- [x] Increased Default Memory Overhead value
- [ ] Dependency Management for virtualenv/conda

## How was this patch tested?

This patch was tested with

- [x] Unit Tests
- [x] Integration tests with [this addition](https://github.com/apache-spark-on-k8s/spark-integration/pull/46)
```
KubernetesSuite:
- Run SparkPi with no resources
- Run SparkPi with a very long application name.
- Run SparkPi with a master URL without a scheme.
- Run SparkPi with an argument.
- Run SparkPi with custom labels, annotations, and environment variables.
- Run SparkPi with a test secret mounted into the driver and executor pods
- Run extraJVMOptions check on driver
- Run SparkRemoteFileTest using a remote data file
- Run PySpark on simple pi.py example
- Run PySpark with Python2 to test a pyfiles example
- Run PySpark with Python3 to test a pyfiles example
Run completed in 4 minutes, 28 seconds.
Total number of tests run: 11
Suites: completed 2, aborted 0
Tests: succeeded 11, failed 0, canceled 0, ignored 0, pending 0
All tests passed.
```

Author: Ilan Filonenko <if56@cornell.edu>
Author: Ilan Filonenko <ifilondz@gmail.com>

Closes #21092 from ifilonenko/master.
2018-06-08 11:18:34 -07:00
xueyu a2166ecdda [SPARK-24455][CORE] fix typo in TaskSchedulerImpl comment
change runTasks to submitTasks  in the TaskSchedulerImpl.scala 's comment

Author: xueyu <xueyu@yidian-inc.com>
Author: Xue Yu <278006819@qq.com>

Closes #21485 from xueyumusic/fixtypo1.
2018-06-04 08:10:49 +07:00
Xingbo Jiang 8ef167a5f9 [SPARK-24340][CORE] Clean up non-shuffle disk block manager files following executor exits on a Standalone cluster
## What changes were proposed in this pull request?

Currently we only clean up the local directories on application removed. However, when executors die and restart repeatedly, many temp files are left untouched in the local directories, which is undesired behavior and could cause disk space used up gradually.

We can detect executor death in the Worker, and clean up the non-shuffle files (files not ended with ".index" or ".data") in the local directories, we should not touch the shuffle files since they are expected to be used by the external shuffle service.

Scope of this PR is limited to only implement the cleanup logic on a Standalone cluster, we defer to experts familiar with other cluster managers(YARN/Mesos/K8s) to determine whether it's worth to add similar support.

## How was this patch tested?

Add new test suite to cover.

Author: Xingbo Jiang <xingbo.jiang@databricks.com>

Closes #21390 from jiangxb1987/cleanupNonshuffleFiles.
2018-06-01 13:46:05 -07:00
hyukjinkwon 2c9c8629b7 [MINOR][YARN] Add YARN-specific credential providers in debug logging message
This PR adds a debugging log for YARN-specific credential providers which is loaded by service loader mechanism.

It took me a while to debug if it's actually loaded or not. I had to explicitly set the deprecated configuration and check if that's actually being loaded.

The change scope is manually tested. Logs are like:

```
Using the following builtin delegation token providers: hadoopfs, hive, hbase.
Using the following YARN-specific credential providers: yarn-test.
```

Author: hyukjinkwon <gurwls223@apache.org>

Closes #21466 from HyukjinKwon/minor-log.

Change-Id: I18e2fb8eeb3289b148f24c47bb3130a560a881cf
2018-06-01 08:44:57 +08:00
Tathagata Das 223df5d9d4 [SPARK-24397][PYSPARK] Added TaskContext.getLocalProperty(key) in Python
## What changes were proposed in this pull request?

This adds a new API `TaskContext.getLocalProperty(key)` to the Python TaskContext. It mirrors the Java TaskContext API of returning a string value if the key exists, or None if the key does not exist.

## How was this patch tested?
New test added.

Author: Tathagata Das <tathagata.das1565@gmail.com>

Closes #21437 from tdas/SPARK-24397.
2018-05-31 11:23:57 -07:00
Marcelo Vanzin 7a82e93b34 [SPARK-24414][UI] Calculate the correct number of tasks for a stage.
This change takes into account all non-pending tasks when calculating
the number of tasks to be shown. This also means that when the stage
is pending, the task table (or, in fact, most of the data in the stage
page) will not be rendered.

I also fixed the label when the known number of tasks is larger than
the recorded number of tasks (it was inverted).

Author: Marcelo Vanzin <vanzin@cloudera.com>

Closes #21457 from vanzin/SPARK-24414.
2018-05-31 10:05:20 -07:00
Sean Owen 698b9a0981 [WEBUI] Avoid possibility of script in query param keys
As discussed separately, this avoids the possibility of XSS on certain request param keys.

CC vanzin

Author: Sean Owen <srowen@gmail.com>

Closes #21464 from srowen/XSS2.
2018-05-31 09:34:39 -07:00
William Sheu 0053e153fa [SPARK-24337][CORE] Improve error messages for Spark conf values
## What changes were proposed in this pull request?

Improve the exception messages when retrieving Spark conf values to include the key name when the value is invalid.

## How was this patch tested?

Unit tests for all get* operations in SparkConf that require a specific value format

Author: William Sheu <william.sheu@databricks.com>

Closes #21454 from PenguinToast/SPARK-24337-spark-config-errors.
2018-05-30 22:37:27 -07:00
hyukjinkwon b142157dcc [SPARK-24384][PYTHON][SPARK SUBMIT] Add .py files correctly into PythonRunner in submit with client mode in spark-submit
## What changes were proposed in this pull request?

In client side before context initialization specifically,  .py file doesn't work in client side before context initialization when the application is a Python file. See below:

```
$ cat /home/spark/tmp.py
def testtest():
    return 1
```

This works:

```
$ cat app.py
import pyspark
pyspark.sql.SparkSession.builder.getOrCreate()
import tmp
print("************************%s" % tmp.testtest())

$ ./bin/spark-submit --master yarn --deploy-mode client --py-files /home/spark/tmp.py app.py
...
************************1
```

but this doesn't:

```
$ cat app.py
import pyspark
import tmp
pyspark.sql.SparkSession.builder.getOrCreate()
print("************************%s" % tmp.testtest())

$ ./bin/spark-submit --master yarn --deploy-mode client --py-files /home/spark/tmp.py app.py
Traceback (most recent call last):
  File "/home/spark/spark/app.py", line 2, in <module>
    import tmp
ImportError: No module named tmp
```

### How did it happen?

In client mode specifically, the paths are being added into PythonRunner as are:

628c7b5179/core/src/main/scala/org/apache/spark/deploy/SparkSubmit.scala (L430)

628c7b5179/core/src/main/scala/org/apache/spark/deploy/PythonRunner.scala (L49-L88)

The problem here is, .py file shouldn't be added as are since `PYTHONPATH` expects a directory or an archive like zip or egg.

### How does this PR fix?

We shouldn't simply just add its parent directory because other files in the parent directory could also be added into the `PYTHONPATH` in client mode before context initialization.

Therefore, we copy .py files into a temp directory for .py files and add it to `PYTHONPATH`.

## How was this patch tested?

Unit tests are added and manually tested in both standalond and yarn client modes with submit.

Author: hyukjinkwon <gurwls223@apache.org>

Closes #21426 from HyukjinKwon/SPARK-24384.
2018-05-30 10:33:34 -07:00
jerryshao 2ced6193b3 [SPARK-24377][SPARK SUBMIT] make --py-files work in non pyspark application
## What changes were proposed in this pull request?

For some Spark applications, though they're a java program, they require not only jar dependencies, but also python dependencies. One example is Livy remote SparkContext application, this application is actually an embedded REPL for Scala/Python/R, it will not only load in jar dependencies, but also python and R deps, so we should specify not only "--jars", but also "--py-files".

Currently for a Spark application, --py-files can only be worked for a pyspark application, so it will not be worked in the above case. So here propose to remove such restriction.

Also we tested that "spark.submit.pyFiles" only supports quite limited scenario (client mode with local deps), so here also expand the usage of "spark.submit.pyFiles" to be alternative of --py-files.

## How was this patch tested?

UT added.

Author: jerryshao <sshao@hortonworks.com>

Closes #21420 from jerryshao/SPARK-24377.
2018-05-29 10:48:48 +08:00
Marco Gaido fd315f5884 [MINOR] Add port SSL config in toString and scaladoc
## What changes were proposed in this pull request?

SPARK-17874 introduced a new configuration to set the port where SSL services bind to. We missed to update the scaladoc and the `toString` method, though. The PR adds it in the missing places

## How was this patch tested?

checked the `toString` output in the logs

Author: Marco Gaido <marcogaido91@gmail.com>

Closes #21429 from mgaido91/minor_ssl.
2018-05-25 12:49:06 -07:00
Xingbo Jiang e108f84f5c [MINOR][CORE] Cleanup unused vals in DAGScheduler.handleTaskCompletion
## What changes were proposed in this pull request?

Cleanup unused vals in `DAGScheduler.handleTaskCompletion` to reduce the code complexity slightly.

## How was this patch tested?

Existing test cases.

Author: Xingbo Jiang <xingbo.jiang@databricks.com>

Closes #21406 from jiangxb1987/handleTaskCompletion.
2018-05-24 11:42:25 +08:00
jinxing b7a036b75b [SPARK-24294] Throw SparkException when OOM in BroadcastExchangeExec
## What changes were proposed in this pull request?

When OutOfMemoryError thrown from BroadcastExchangeExec, scala.concurrent.Future will hit scala bug – https://github.com/scala/bug/issues/9554, and hang until future timeout:

We could wrap the OOM inside SparkException to resolve this issue.

## How was this patch tested?

Manually tested.

Author: jinxing <jinxing6042@126.com>

Closes #21342 from jinxing64/SPARK-24294.
2018-05-23 13:12:05 -07:00
Jake Charland a4470bc78c [SPARK-21673] Use the correct sandbox environment variable set by Mesos
## What changes were proposed in this pull request?
This change changes spark behavior to use the correct environment variable set by Mesos in the container on startup.

Author: Jake Charland <jakec@uber.com>

Closes #18894 from jakecharland/MesosSandbox.
2018-05-22 08:06:15 -05:00
Xianjin YE 82fb5bfa77 [SPARK-20087][CORE] Attach accumulators / metrics to 'TaskKilled' end reason
## What changes were proposed in this pull request?
The ultimate goal is for listeners to onTaskEnd to receive metrics when a task is killed intentionally, since the data is currently just thrown away. This is already done for ExceptionFailure, so this just copies the same approach.

## How was this patch tested?
Updated existing tests.

This is a rework of https://github.com/apache/spark/pull/17422, all credits should go to noodle-fb

Author: Xianjin YE <advancedxy@gmail.com>
Author: Charles Lewis <noodle@fb.com>

Closes #21165 from advancedxy/SPARK-20087.
2018-05-22 21:02:17 +08:00
Marco Gaido 84d31aa5d4 [SPARK-24209][SHS] Automatic retrieve proxyBase from Knox headers
## What changes were proposed in this pull request?

The PR retrieves the proxyBase automatically from the header `X-Forwarded-Context` (if available). This is the header used by Knox to inform the proxied service about the base path.

This provides 0-configuration support for Knox gateway (instead of having to properly set `spark.ui.proxyBase`) and it allows to access directly SHS when it is proxied by Knox. In the previous scenario, indeed, after setting `spark.ui.proxyBase`, direct access to SHS was not working fine (due to bad link generated).

## How was this patch tested?

added UT + manual tests

Author: Marco Gaido <marcogaido91@gmail.com>

Closes #21268 from mgaido91/SPARK-24209.
2018-05-21 18:11:05 -07:00
Imran Rashid 32447079e9 [SPARK-24309][CORE] AsyncEventQueue should stop on interrupt.
EventListeners can interrupt the event queue thread.  In particular,
when the EventLoggingListener writes to hdfs, hdfs can interrupt the
thread.  When there is an interrupt, the queue should be removed and stop
accepting any more events.  Before this change, the queue would continue
to take more events (till it was full), and then would not stop when the
application was complete because the PoisonPill couldn't be added.

Added a unit test which failed before this change.

Author: Imran Rashid <irashid@cloudera.com>

Closes #21356 from squito/SPARK-24309.
2018-05-21 16:26:39 -07:00
Wenchen Fan 03e90f65bf [SPARK-24250][SQL] support accessing SQLConf inside tasks
re-submit https://github.com/apache/spark/pull/21299 which broke build.

A few new commits are added to fix the SQLConf problem in `JsonSchemaInference.infer`, and prevent us to access `SQLConf` in DAGScheduler event loop thread.

## What changes were proposed in this pull request?

Previously in #20136 we decided to forbid tasks to access `SQLConf`, because it doesn't work and always give you the default conf value. In #21190 we fixed the check and all the places that violate it.

Currently the pattern of accessing configs at the executor side is: read the configs at the driver side, then access the variables holding the config values in the RDD closure, so that they will be serialized to the executor side. Something like
```
val someConf = conf.getXXX
child.execute().mapPartitions {
  if (someConf == ...) ...
  ...
}
```

However, this pattern is hard to apply if the config needs to be propagated via a long call stack. An example is `DataType.sameType`, and see how many changes were made in #21190 .

When it comes to code generation, it's even worse. I tried it locally and we need to change a ton of files to propagate configs to code generators.

This PR proposes to allow tasks to access `SQLConf`. The idea is, we can save all the SQL configs to job properties when an SQL execution is triggered. At executor side we rebuild the `SQLConf` from job properties.

## How was this patch tested?

a new test suite

Author: Wenchen Fan <wenchen@databricks.com>

Closes #21376 from cloud-fan/config.
2018-05-22 00:19:18 +08:00
Stavros 8eac621229 [SPARK-23857][MESOS] remove keytab check in mesos cluster mode at first submit time
## What changes were proposed in this pull request?
- Removes the check for the keytab when we are running in mesos cluster mode.
- Keeps the check for client mode since in cluster mode we eventually launch the driver within the cluster in client mode. In the latter case we want to have the check done when the container starts, the keytab should be checked if it exists within the container's local filesystem.

## How was this patch tested?

This was manually tested by running spark submit in mesos cluster mode.

Author: Stavros <st.kontopoulos@gmail.com>

Closes #20967 from skonto/fix_mesos_keytab_susbmit.
2018-05-20 18:15:04 -05:00
Wenchen Fan 000e25ae79 Revert "[SPARK-24250][SQL] support accessing SQLConf inside tasks"
This reverts commit dd37529a8d.
2018-05-20 16:13:42 +08:00
Wenchen Fan dd37529a8d [SPARK-24250][SQL] support accessing SQLConf inside tasks
## What changes were proposed in this pull request?

Previously in #20136 we decided to forbid tasks to access `SQLConf`, because it doesn't work and always give you the default conf value. In #21190 we fixed the check and all the places that violate it.

Currently the pattern of accessing configs at the executor side is: read the configs at the driver side, then access the variables holding the config values in the RDD closure, so that they will be serialized to the executor side. Something like
```
val someConf = conf.getXXX
child.execute().mapPartitions {
  if (someConf == ...) ...
  ...
}
```

However, this pattern is hard to apply if the config needs to be propagated via a long call stack. An example is `DataType.sameType`, and see how many changes were made in #21190 .

When it comes to code generation, it's even worse. I tried it locally and we need to change a ton of files to propagate configs to code generators.

This PR proposes to allow tasks to access `SQLConf`. The idea is, we can save all the SQL configs to job properties when an SQL execution is triggered. At executor side we rebuild the `SQLConf` from job properties.

## How was this patch tested?

a new test suite

Author: Wenchen Fan <wenchen@databricks.com>

Closes #21299 from cloud-fan/config.
2018-05-19 18:51:02 +08:00
gatorsmile 1c4553d67d Revert "[SPARK-24277][SQL] Code clean up in SQL module: HadoopMapReduceCommitProtocol"
This reverts commit 7b2dca5b12.
2018-05-18 12:51:09 -07:00
Marcelo Vanzin ed7ba7db8f [SPARK-23850][SQL] Add separate config for SQL options redaction.
The old code was relying on a core configuration and extended its
default value to include things that redact desired things in the
app's environment. Instead, add a SQL-specific option for which
options to redact, and apply both the core and SQL-specific rules
when redacting the options in the save command.

This is a little sub-optimal since it adds another config, but it
retains the current default behavior.

While there I also fixed a typo and a couple of minor config API
usage issues in the related redaction option that SQL already had.

Tested with existing unit tests, plus checking the env page on
a shell UI.

Author: Marcelo Vanzin <vanzin@cloudera.com>

Closes #21158 from vanzin/SPARK-23850.
2018-05-18 11:14:22 -07:00
Gengliang Wang 7b2dca5b12 [SPARK-24277][SQL] Code clean up in SQL module: HadoopMapReduceCommitProtocol
## What changes were proposed in this pull request?

In HadoopMapReduceCommitProtocol and FileFormatWriter, there are unnecessary settings in hadoop configuration.

Also clean up some code in SQL module.

## How was this patch tested?

Unit test

Author: Gengliang Wang <gengliang.wang@databricks.com>

Closes #21329 from gengliangwang/codeCleanWrite.
2018-05-18 15:32:29 +08:00
Wenchen Fan 6ec05826d7 [SPARK-24107][CORE][FOLLOWUP] ChunkedByteBuffer.writeFully method has not reset the limit value
## What changes were proposed in this pull request?

According to the discussion in https://github.com/apache/spark/pull/21175 , this PR proposes 2 improvements:
1. add comments to explain why we call `limit` to write out `ByteBuffer` with slices.
2. remove the `try ... finally`

## How was this patch tested?

existing tests

Author: Wenchen Fan <wenchen@databricks.com>

Closes #21327 from cloud-fan/minor.
2018-05-17 20:42:40 +08:00
Artem Rudoy 6c35865d94 [SPARK-22371][CORE] Return None instead of throwing an exception when an accumulator is garbage collected.
## What changes were proposed in this pull request?

There's a period of time when an accumulator has been garbage collected, but hasn't been removed from AccumulatorContext.originals by ContextCleaner. When an update is received for such accumulator it will throw an exception and kill the whole job. This can happen when a stage completes, but there're still running tasks from other attempts, speculation etc. Since AccumulatorContext.get() returns an option we can just return None in such case.

## How was this patch tested?

Unit test.

Author: Artem Rudoy <artem.rudoy@gmail.com>

Closes #21114 from artemrd/SPARK-22371.
2018-05-17 18:49:46 +08:00
Kent Yao 4a2b15f0af [SPARK-24241][SUBMIT] Do not fail fast when dynamic resource allocation enabled with 0 executor
## What changes were proposed in this pull request?
```
~/spark-2.3.0-bin-hadoop2.7$ bin/spark-sql --num-executors 0 --conf spark.dynamicAllocation.enabled=true
Java HotSpot(TM) 64-Bit Server VM warning: ignoring option PermSize=1024m; support was removed in 8.0
Java HotSpot(TM) 64-Bit Server VM warning: ignoring option MaxPermSize=1024m; support was removed in 8.0
Error: Number of executors must be a positive number
Run with --help for usage help or --verbose for debug output
```

Actually, we could start up with min executor number with 0 before if dynamically

## How was this patch tested?

ut added

Author: Kent Yao <yaooqinn@hotmail.com>

Closes #21290 from yaooqinn/SPARK-24241.
2018-05-15 16:04:17 +08:00
Kazuaki Ishizaki d3c426a5b0 [SPARK-10878][CORE] Fix race condition when multiple clients resolves artifacts at the same time
## What changes were proposed in this pull request?

When multiple clients attempt to resolve artifacts via the `--packages` parameter, they could run into race condition when they each attempt to modify the dummy `org.apache.spark-spark-submit-parent-default.xml` file created in the default ivy cache dir.
This PR changes the behavior to encode UUID in the dummy module descriptor so each client will operate on a different resolution file in the ivy cache dir. In addition, this patch changes the behavior of when and which resolution files are cleaned to prevent accumulation of resolution files in the default ivy cache dir.

Since this PR is a successor of #18801, close #18801. Many codes were ported from #18801. **Many efforts were put here. I think this PR should credit to Victsm .**

## How was this patch tested?

added UT into `SparkSubmitUtilsSuite`

Author: Kazuaki Ishizaki <ishizaki@jp.ibm.com>

Closes #21251 from kiszk/SPARK-10878.
2018-05-10 14:41:55 -07:00
“attilapiros” 3e2600538e [SPARK-19181][CORE] Fixing flaky "SparkListenerSuite.local metrics"
## What changes were proposed in this pull request?

Sometimes "SparkListenerSuite.local metrics" test fails because the average of executorDeserializeTime is too short. As squito suggested to avoid these situations in one of the task a reference introduced to an object implementing a custom Externalizable.readExternal which sleeps 1ms before returning.

## How was this patch tested?

With unit tests (and checking the effect of this change to the average with a much larger sleep time).

Author: “attilapiros” <piros.attila.zsolt@gmail.com>
Author: Attila Zsolt Piros <2017933+attilapiros@users.noreply.github.com>

Closes #21280 from attilapiros/SPARK-19181.
2018-05-10 14:26:38 -07:00
mcheah 6282fc64e3 [SPARK-24137][K8S] Mount local directories as empty dir volumes.
## What changes were proposed in this pull request?

Drastically improves performance and won't cause Spark applications to fail because they write too much data to the Docker image's specific file system. The file system's directories that back emptydir volumes are generally larger and more performant.

## How was this patch tested?

Has been in use via the prototype version of Kubernetes support, but lost in the transition to here.

Author: mcheah <mcheah@palantir.com>

Closes #21238 from mccheah/mount-local-dirs.
2018-05-10 11:36:41 -07:00
wuyi 9e3bb31368 [SPARK-24141][CORE] Fix bug in CoarseGrainedSchedulerBackend.killExecutors
## What changes were proposed in this pull request?

In method *CoarseGrainedSchedulerBackend.killExecutors()*, `numPendingExecutors` should add
`executorsToKill.size` rather than `knownExecutors.size` if we do not adjust target number of executors.

## How was this patch tested?

N/A

Author: wuyi <ngone_5451@163.com>

Closes #21209 from Ngone51/SPARK-24141.
2018-05-09 15:44:36 -07:00
Marcelo Vanzin 628c7b5179 [SPARKR] Match pyspark features in SparkR communication protocol. 2018-05-09 10:47:35 -07:00
Marcelo Vanzin cc613b552e [PYSPARK] Update py4j to version 0.10.7. 2018-05-09 10:47:35 -07:00
Marcelo Vanzin 05eb19b6e0 [SPARK-24188][CORE] Restore "/version" API endpoint.
It was missing the jax-rs annotation.

Author: Marcelo Vanzin <vanzin@cloudera.com>

Closes #21245 from vanzin/SPARK-24188.

Change-Id: Ib338e34b363d7c729cc92202df020dc51033b719
2018-05-08 14:32:04 +08:00
Marco Gaido f06528015d [SPARK-24160][FOLLOWUP] Fix compilation failure
## What changes were proposed in this pull request?

SPARK-24160 is causing a compilation failure (after SPARK-24143 was merged). This fixes the issue.

## How was this patch tested?

building successfully

Author: Marco Gaido <marcogaido91@gmail.com>

Closes #21256 from mgaido91/SPARK-24160_FOLLOWUP.
2018-05-07 15:42:10 +08:00
Josh Rosen d2aa859b4f [SPARK-24160] ShuffleBlockFetcherIterator should fail if it receives zero-size blocks
## What changes were proposed in this pull request?

This patch modifies `ShuffleBlockFetcherIterator` so that the receipt of zero-size blocks is treated as an error. This is done as a preventative measure to guard against a potential source of data loss bugs.

In the shuffle layer, we guarantee that zero-size blocks will never be requested (a block containing zero records is always 0 bytes in size and is marked as empty such that it will never be legitimately requested by executors). However, the existing code does not fully take advantage of this invariant in the shuffle-read path: the existing code did not explicitly check whether blocks are non-zero-size.

Additionally, our decompression and deserialization streams treat zero-size inputs as empty streams rather than errors (EOF might actually be treated as "end-of-stream" in certain layers (longstanding behavior dating to earliest versions of Spark) and decompressors like Snappy may be tolerant to zero-size inputs).

As a result, if some other bug causes legitimate buffers to be replaced with zero-sized buffers (due to corruption on either the send or receive sides) then this would translate into silent data loss rather than an explicit fail-fast error.

This patch addresses this problem by adding a `buf.size != 0` check. See code comments for pointers to tests which guarantee the invariants relied on here.

## How was this patch tested?

Existing tests (which required modifications, since some were creating empty buffers in mocks). I also added a test to make sure we fail on zero-size blocks.

To test that the zero-size blocks are indeed a potential corruption source, I manually ran a workload in `spark-shell` with a modified build which replaces all buffers with zero-size buffers in the receive path.

Author: Josh Rosen <joshrosen@databricks.com>

Closes #21219 from JoshRosen/SPARK-24160.
2018-05-07 14:34:03 +08:00
jinxing 889f6cc10c [SPARK-24143] filter empty blocks when convert mapstatus to (blockId, size) pair
## What changes were proposed in this pull request?

In current code(`MapOutputTracker.convertMapStatuses`), mapstatus are converted to (blockId, size) pair for all blocks – no matter the block is empty or not, which result in OOM when there are lots of consecutive empty blocks, especially when adaptive execution is enabled.

(blockId, size) pair is only used in `ShuffleBlockFetcherIterator` to control shuffle-read and only non-empty block request is sent. Can we just filter out the empty blocks in MapOutputTracker.convertMapStatuses and save memory?

## How was this patch tested?

not added yet.

Author: jinxing <jinxing6042@126.com>

Closes #21212 from jinxing64/SPARK-24143.
2018-05-07 14:16:27 +08:00
Thomas Graves d04806a23c [SPARK-24124] Spark history server should create spark.history.store.…
…path and set permissions properly

## What changes were proposed in this pull request?

Spark history server should create spark.history.store.path and set permissions properly. Note createdDirectories doesn't do anything if the directories are already created.  This does not stomp on the permissions if the user had manually created the directory before the history server could.

## How was this patch tested?

Manually tested in a 100 node cluster. Ensured directories created with proper permissions. Ensured restarted worked apps/temp directories worked as apps were read.

Author: Thomas Graves <tgraves@thirteenroutine.corp.gq1.yahoo.com>

Closes #21234 from tgravescs/SPARK-24124.
2018-05-04 13:29:47 -07:00
Wenchen Fan 4d5de4d303 [SPARK-23697][CORE] LegacyAccumulatorWrapper should define isZero correctly
## What changes were proposed in this pull request?

It's possible that Accumulators of Spark 1.x may no longer work with Spark 2.x. This is because `LegacyAccumulatorWrapper.isZero` may return wrong answer if `AccumulableParam` doesn't define equals/hashCode.

This PR fixes this by using reference equality check in `LegacyAccumulatorWrapper.isZero`.

## How was this patch tested?

a new test

Author: Wenchen Fan <wenchen@databricks.com>

Closes #21229 from cloud-fan/accumulator.
2018-05-04 19:20:15 +08:00
Imran Rashid 94641fe6cc [SPARK-23433][CORE] Late zombie task completions update all tasksets
Fetch failure lead to multiple tasksets which are active for a given
stage.  While there is only one "active" version of the taskset, the
earlier attempts can still have running tasks, which can complete
successfully.  So a task completion needs to update every taskset
so that it knows the partition is completed.  That way the final active
taskset does not try to submit another task for the same partition,
and so that it knows when it is completed and when it should be
marked as a "zombie".

Added a regression test.

Author: Imran Rashid <irashid@cloudera.com>

Closes #21131 from squito/SPARK-23433.
2018-05-03 10:59:18 -05:00
WangJinhai02 152eaf6ae6 [SPARK-24107][CORE] ChunkedByteBuffer.writeFully method has not reset the limit value
JIRA Issue: https://issues.apache.org/jira/browse/SPARK-24107?jql=text%20~%20%22ChunkedByteBuffer%22

ChunkedByteBuffer.writeFully method has not reset the limit value. When
chunks larger than bufferWriteChunkSize, such as 80 * 1024 * 1024 larger than
config.BUFFER_WRITE_CHUNK_SIZE(64 * 1024 * 1024),only while once, will lost 16 * 1024 * 1024 byte

Author: WangJinhai02 <jinhai.wang02@ele.me>

Closes #21175 from manbuyun/bugfix-ChunkedByteBuffer.
2018-05-02 22:40:14 +08:00
Devaraj K 007ae6878f [SPARK-24003][CORE] Add support to provide spark.executor.extraJavaOptions in terms of App Id and/or Executor Id's
## What changes were proposed in this pull request?

Added support to specify the 'spark.executor.extraJavaOptions' value in terms of the `{{APP_ID}}` and/or `{{EXECUTOR_ID}}`,  `{{APP_ID}}` will be replaced by Application Id and `{{EXECUTOR_ID}}` will be replaced by Executor Id while starting the executor.

## How was this patch tested?

I have verified this by checking the executor process command and gc logs. I verified the same in different deployment modes(Standalone, YARN, Mesos) client and cluster modes.

Author: Devaraj K <devaraj@apache.org>

Closes #21088 from devaraj-kavali/SPARK-24003.
2018-04-30 13:40:03 -07:00
jerryshao ffaf0f9fd4 [SPARK-24062][THRIFT SERVER] Fix SASL encryption cannot enabled issue in thrift server
## What changes were proposed in this pull request?

For the details of the exception please see [SPARK-24062](https://issues.apache.org/jira/browse/SPARK-24062).

The issue is:

Spark on Yarn stores SASL secret in current UGI's credentials, this credentials will be distributed to AM and executors, so that executors and drive share the same secret to communicate. But STS/Hive library code will refresh the current UGI by UGI's loginFromKeytab() after Spark application is started, this will create a new UGI in the current driver's context with empty tokens and secret keys, so secret key is lost in the current context's UGI, that's why Spark driver throws secret key not found exception.

In Spark 2.2 code, Spark also stores this secret key in SecurityManager's class variable, so even UGI is refreshed, the secret is still existed in the object, so STS with SASL can still be worked in Spark 2.2. But in Spark 2.3, we always search key from current UGI, which makes it fail to work in Spark 2.3.

To fix this issue, there're two possible solutions:

1. Fix in STS/Hive library, when a new UGI is refreshed, copy the secret key from original UGI to the new one. The difficulty is that some codes to refresh the UGI is existed in Hive library, which makes us hard to change the code.
2. Roll back the logics in SecurityManager to match Spark 2.2, so that this issue can be fixed.

2nd solution seems a simple one. So I will propose a PR with 2nd solution.

## How was this patch tested?

Verified in local cluster.

CC vanzin  tgravescs  please help to review. Thanks!

Author: jerryshao <sshao@hortonworks.com>

Closes #21138 from jerryshao/SPARK-24062.
2018-04-26 13:27:33 +08:00
Steve Loughran ce7ba2e98e [SPARK-23807][BUILD] Add Hadoop 3.1 profile with relevant POM fix ups
## What changes were proposed in this pull request?

1. Adds a `hadoop-3.1` profile build depending on the hadoop-3.1 artifacts.
1. In the hadoop-cloud module, adds an explicit hadoop-3.1 profile which switches from explicitly pulling in cloud connectors (hadoop-openstack, hadoop-aws, hadoop-azure) to depending on the hadoop-cloudstorage POM artifact, which pulls these in, has pre-excluded things like hadoop-common, and stays up to date with new connectors (hadoop-azuredatalake, hadoop-allyun). Goal: it becomes the Hadoop projects homework of keeping this clean, and the spark project doesn't need to handle new hadoop releases adding more dependencies.
1. the hadoop-cloud/hadoop-3.1 profile also declares support for jetty-ajax and jetty-util to ensure that these jars get into the distribution jar directory when needed by unshaded libraries.
1. Increases the curator and zookeeper versions to match those in hadoop-3, fixing spark core to build in sbt with the hadoop-3 dependencies.

## How was this patch tested?

* Everything this has been built and tested against both ASF Hadoop branch-3.1 and hadoop trunk.
* spark-shell was used to create connectors to all the stores and verify that file IO could take place.

The spark hive-1.2.1 JAR has problems here, as it's version check logic fails for Hadoop versions > 2.

This can be avoided with either of

* The hadoop JARs built to declare their version as Hadoop 2.11  `mvn install -DskipTests -DskipShade -Ddeclared.hadoop.version=2.11` . This is safe for local test runs, not for deployment (HDFS is very strict about cross-version deployment).
* A modified version of spark hive whose version check switch statement is happy with hadoop 3.

I've done both, with maven and SBT.

Three issues surfaced

1. A spark-core test failure —fixed in SPARK-23787.
1. SBT only: Zookeeper not being found in spark-core. Somehow curator 2.12.0 triggers some slightly different dependency resolution logic from previous versions, and Ivy was missing zookeeper.jar entirely. This patch adds the explicit declaration for all spark profiles, setting the ZK version = 3.4.9 for hadoop-3.1
1. Marking jetty-utils as provided in spark was stopping hadoop-azure from being able to instantiate the azure wasb:// client; it was using jetty-util-ajax, which could then not find a class in jetty-util.

Author: Steve Loughran <stevel@hortonworks.com>

Closes #20923 from steveloughran/cloud/SPARK-23807-hadoop-31.
2018-04-24 09:57:09 -07:00
Julien Cuquemelle 55c4ca88a3 [SPARK-22683][CORE] Add a executorAllocationRatio parameter to throttle the parallelism of the dynamic allocation
## What changes were proposed in this pull request?

By default, the dynamic allocation will request enough executors to maximize the
parallelism according to the number of tasks to process. While this minimizes the
latency of the job, with small tasks this setting can waste a lot of resources due to
executor allocation overhead, as some executor might not even do any work.
This setting allows to set a ratio that will be used to reduce the number of
target executors w.r.t. full parallelism.

The number of executors computed with this setting is still fenced by
`spark.dynamicAllocation.maxExecutors` and `spark.dynamicAllocation.minExecutors`

## How was this patch tested?
Units tests and runs on various actual workloads on a Yarn Cluster

Author: Julien Cuquemelle <j.cuquemelle@criteo.com>

Closes #19881 from jcuquemelle/AddTaskPerExecutorSlot.
2018-04-24 10:56:55 -05:00
wuyi c8f3ac69d1 [SPARK-23888][CORE] correct the comment of hasAttemptOnHost()
TaskSetManager.hasAttemptOnHost had a misleading comment.  The comment
said that it only checked for running tasks, but really it checked for
any tasks that might have run in the past as well.  This updates to line
up with the implementation.

Author: wuyi <ngone_5451@163.com>

Closes #20998 from Ngone51/SPARK-23888.
2018-04-23 15:35:50 -05:00
Marcelo Vanzin 32b4bcd6d3 [SPARK-24029][CORE] Set SO_REUSEADDR on listen sockets.
This allows sockets to be bound even if there are sockets
from a previous application that are still pending closure. It
avoids bind issues when, for example, re-starting the SHS.

Don't enable the option on Windows though. The following page
explains some odd behavior that this option can have there:
https://msdn.microsoft.com/en-us/library/windows/desktop/ms740621%28v=vs.85%29.aspx

I intentionally ignored server sockets that always bind to
ephemeral ports, since those don't benefit from this option.

Author: Marcelo Vanzin <vanzin@cloudera.com>

Closes #21110 from vanzin/SPARK-24029.
2018-04-21 23:14:58 +08:00
Gabor Somogyi e55953b0bf [SPARK-24022][TEST] Make SparkContextSuite not flaky
## What changes were proposed in this pull request?

SparkContextSuite.test("Cancelling stages/jobs with custom reasons.") could stay in an infinite loop because of the problem found and fixed in [SPARK-23775](https://issues.apache.org/jira/browse/SPARK-23775).

This PR solves this mentioned flakyness by removing shared variable usages when cancel happens in a loop and using wait and CountDownLatch for synhronization.

## How was this patch tested?

Existing unit test.

Author: Gabor Somogyi <gabor.g.somogyi@gmail.com>

Closes #21105 from gaborgsomogyi/SPARK-24022.
2018-04-19 15:06:27 -07:00
wuyi 0deaa52513 [SPARK-24021][CORE] fix bug in BlacklistTracker's updateBlacklistForFetchFailure
## What changes were proposed in this pull request?

There‘s a miswrite in BlacklistTracker's updateBlacklistForFetchFailure:
```
val blacklistedExecsOnNode =
    nodeToBlacklistedExecs.getOrElseUpdate(exec, HashSet[String]())
blacklistedExecsOnNode += exec
```
where first **exec** should be **host**.
## How was this patch tested?

adjust existed test.

Author: wuyi <ngone_5451@163.com>

Closes #21104 from Ngone51/SPARK-24021.
2018-04-19 09:00:33 -05:00
jinxing 3990daaf3b [SPARK-23948] Trigger mapstage's job listener in submitMissingTasks
## What changes were proposed in this pull request?

SparkContext submitted a map stage from `submitMapStage` to `DAGScheduler`,
`markMapStageJobAsFinished` is called only in (https://github.com/apache/spark/blob/master/core/src/main/scala/org/apache/spark/scheduler/DAGScheduler.scala#L933 and https://github.com/apache/spark/blob/master/core/src/main/scala/org/apache/spark/scheduler/DAGScheduler.scala#L1314);

But think about below scenario:
1. stage0 and stage1 are all `ShuffleMapStage` and stage1 depends on stage0;
2. We submit stage1 by `submitMapStage`;
3. When stage 1 running, `FetchFailed` happened, stage0 and stage1 got resubmitted as stage0_1 and stage1_1;
4. When stage0_1 running, speculated tasks in old stage1 come as succeeded, but stage1 is not inside `runningStages`. So even though all splits(including the speculated tasks) in stage1 succeeded, job listener in stage1 will not be called;
5. stage0_1 finished, stage1_1 starts running. When `submitMissingTasks`, there is no missing tasks. But in current code, job listener is not triggered.

We should call the job listener for map stage in `5`.

## How was this patch tested?

Not added yet.

Author: jinxing <jinxing6042@126.com>

Closes #21019 from jinxing64/SPARK-23948.
2018-04-17 08:55:01 -05:00
jinxing ed4101d29f [SPARK-22676] Avoid iterating all partition paths when spark.sql.hive.verifyPartitionPath=true
## What changes were proposed in this pull request?

In current code, it will scanning all partition paths when spark.sql.hive.verifyPartitionPath=true.
e.g. table like below:
```
CREATE TABLE `test`(
`id` int,
`age` int,
`name` string)
PARTITIONED BY (
`A` string,
`B` string)
load data local inpath '/tmp/data0' into table test partition(A='00', B='00')
load data local inpath '/tmp/data1' into table test partition(A='01', B='01')
load data local inpath '/tmp/data2' into table test partition(A='10', B='10')
load data local inpath '/tmp/data3' into table test partition(A='11', B='11')
```
If I query with SQL – "select * from test where A='00' and B='01'  ", current code will scan all partition paths including '/data/A=00/B=00', '/data/A=00/B=00', '/data/A=01/B=01', '/data/A=10/B=10', '/data/A=11/B=11'. It costs much time and memory cost.

This pr proposes to avoid iterating all partition paths. Add a config `spark.files.ignoreMissingFiles` and ignore the `file not found` when `getPartitions/compute`(for hive table scan). This is much like the logic brought by
`spark.sql.files.ignoreMissingFiles`(which is for datasource scan).

## How was this patch tested?
UT

Author: jinxing <jinxing6042@126.com>

Closes #19868 from jinxing64/SPARK-22676.
2018-04-17 21:52:33 +08:00
Yuming Wang 083cf22356 [SPARK-21033][CORE][FOLLOW-UP] Update Spillable
## What changes were proposed in this pull request?

Update
```scala
SparkEnv.get.conf.getLong("spark.shuffle.spill.numElementsForceSpillThreshold", Long.MaxValue)
```
to
```scala
SparkEnv.get.conf.get(SHUFFLE_SPILL_NUM_ELEMENTS_FORCE_SPILL_THRESHOLD)
```

 because of `SHUFFLE_SPILL_NUM_ELEMENTS_FORCE_SPILL_THRESHOLD`'s default value is `Integer.MAX_VALUE`:
c99fc9ad9b/core/src/main/scala/org/apache/spark/internal/config/package.scala (L503-L511)

## How was this patch tested?

N/A

Author: Yuming Wang <yumwang@ebay.com>

Closes #21077 from wangyum/SPARK-21033.
2018-04-16 23:50:50 +08:00
Fangshi Li 4b07036799 [SPARK-23815][CORE] Spark writer dynamic partition overwrite mode may fail to write output on multi level partition
## What changes were proposed in this pull request?

Spark introduced new writer mode to overwrite only related partitions in SPARK-20236. While we are using this feature in our production cluster, we found a bug when writing multi-level partitions on HDFS.

A simple test case to reproduce this issue:
val df = Seq(("1","2","3")).toDF("col1", "col2","col3")
df.write.partitionBy("col1","col2").mode("overwrite").save("/my/hdfs/location")

If HDFS location "/my/hdfs/location" does not exist, there will be no output.

This seems to be caused by the job commit change in SPARK-20236 in HadoopMapReduceCommitProtocol.

In the commit job process, the output has been written into staging dir /my/hdfs/location/.spark-staging.xxx/col1=1/col2=2, and then the code calls fs.rename to rename /my/hdfs/location/.spark-staging.xxx/col1=1/col2=2 to /my/hdfs/location/col1=1/col2=2. However, in our case the operation will fail on HDFS because /my/hdfs/location/col1=1 does not exists. HDFS rename can not create directory for more than one level.

This does not happen in the new unit test added with SPARK-20236 which uses local file system.

We are proposing a fix. When cleaning current partition dir /my/hdfs/location/col1=1/col2=2 before the rename op, if the delete op fails (because /my/hdfs/location/col1=1/col2=2 may not exist), we call mkdirs op to create the parent dir /my/hdfs/location/col1=1 (if the parent dir does not exist) so the following rename op can succeed.

Reference: in official HDFS document(https://hadoop.apache.org/docs/stable/hadoop-project-dist/hadoop-common/filesystem/filesystem.html), the rename command has precondition "dest must be root, or have a parent that exists"

## How was this patch tested?

We have tested this patch on our production cluster and it fixed the problem

Author: Fangshi Li <fli@linkedin.com>

Closes #20931 from fangshil/master.
2018-04-13 13:46:34 +08:00
Patrick Pisciuneri 682002b6da [SPARK-23867][SCHEDULER] use droppedCount in logWarning
## What changes were proposed in this pull request?

Get the count of dropped events for output in log message.

## How was this patch tested?

The fix is pretty trivial, but `./dev/run-tests` were run and were successful.

Please review http://spark.apache.org/contributing.html before opening a pull request.

vanzin cloud-fan

The contribution is my original work and I license the work to the project under the project’s open source license.

Author: Patrick Pisciuneri <Patrick.Pisciuneri@target.com>

Closes #20977 from phpisciuneri/fix-log-warning.
2018-04-13 09:45:27 +08:00
Marcelo Vanzin 3cb82047f2 [SPARK-22941][CORE] Do not exit JVM when submit fails with in-process launcher.
The current in-process launcher implementation just calls the SparkSubmit
object, which, in case of errors, will more often than not exit the JVM.
This is not desirable since this launcher is meant to be used inside other
applications, and that would kill the application.

The change turns SparkSubmit into a class, and abstracts aways some of
the functionality used to print error messages and abort the submission
process. The default implementation uses the logging system for messages,
and throws exceptions for errors. As part of that I also moved some code
that doesn't really belong in SparkSubmit to a better location.

The command line invocation of spark-submit now uses a special implementation
of the SparkSubmit class that overrides those behaviors to do what is expected
from the command line version (print to the terminal, exit the JVM, etc).

A lot of the changes are to replace calls to methods such as "printErrorAndExit"
with the new API.

As part of adding tests for this, I had to fix some small things in the
launcher option parser so that things like "--version" can work when
used in the launcher library.

There is still code that prints directly to the terminal, like all the
Ivy-related code in SparkSubmitUtils, and other areas where some re-factoring
would help, like the CommandLineUtils class, but I chose to leave those
alone to keep this change more focused.

Aside from existing and added unit tests, I ran command line tools with
a bunch of different arguments to make sure messages and errors behave
like before.

Author: Marcelo Vanzin <vanzin@cloudera.com>

Closes #20925 from vanzin/SPARK-22941.
2018-04-11 10:13:44 -05:00
Marcelo Vanzin 653fe02415 [SPARK-6951][CORE] Speed up parsing of event logs during listing.
This change introduces two optimizations to help speed up generation
of listing data when parsing events logs.

The first one allows the parser to be stopped when enough data to
create the listing entry has been read. This is currently the start
event plus environment info, to capture UI ACLs. If the end event is
needed, the code will skip to the end of the log to try to find that
information, instead of parsing the whole log file.

Unfortunately this works better with uncompressed logs. Skipping bytes
on compressed logs only saves the work of parsing lines and some events,
so not a lot of gains are observed.

The second optimization deals with in-progress logs. It works in two
ways: first, it completely avoids parsing the rest of the log for
these apps when enough listing data is read. This, unlike the above,
also speeds things up for compressed logs, since only the very beginning
of the log has to be read.

On top of that, the code that decides whether to re-parse logs to get
updated listing data will ignore in-progress applications until they've
completed.

Both optimizations can be disabled but are enabled by default.

I tested this on some fake event logs to see the effect. I created
500 logs of about 60M each (so ~30G uncompressed; each log was 1.7M
when compressed with zstd). Below, C = completed, IP = in-progress,
the size means the amount of data re-parsed at the end of logs
when necessary.

```
            none/C   none/IP   zstd/C   zstd/IP
On / 16k      2s       2s       22s       2s
On / 1m       3s       2s       24s       2s
Off          1.1m     1.1m      26s      24s
```

This was with 4 threads on a single local SSD. As expected from the
previous explanations, there are considerable gains for in-progress
logs, and for uncompressed logs, but not so much when looking at the
full compressed log.

As a side note, I removed the custom code to get the scan time by
creating a file on HDFS; since file mod times are not used to detect
changed logs anymore, local time is enough for the current use of
the SHS.

Author: Marcelo Vanzin <vanzin@cloudera.com>

Closes #20952 from vanzin/SPARK-6951.
2018-04-11 09:49:25 -05:00
Imran Rashid 10f45bb823 [SPARK-23816][CORE] Killed tasks should ignore FetchFailures.
SPARK-19276 ensured that FetchFailures do not get swallowed by other
layers of exception handling, but it also meant that a killed task could
look like a fetch failure.  This is particularly a problem with
speculative execution, where we expect to kill tasks as they are reading
shuffle data.  The fix is to ensure that we always check for killed
tasks first.

Added a new unit test which fails before the fix, ran it 1k times to
check for flakiness.  Full suite of tests on jenkins.

Author: Imran Rashid <irashid@cloudera.com>

Closes #20987 from squito/SPARK-23816.
2018-04-09 11:31:21 -07:00
Xingbo Jiang d81f29ecaf [SPARK-23881][CORE][TEST] Fix flaky test JobCancellationSuite."interruptible iterator of shuffle reader"
## What changes were proposed in this pull request?

The test case JobCancellationSuite."interruptible iterator of shuffle reader" has been flaky because `KillTask` event is handled asynchronously, so it can happen that the semaphore is released but the task is still running.
Actually we only have to check if the total number of processed elements is less than the input elements number, so we know the task get cancelled.

## How was this patch tested?

The new test case still fails without the purposed patch, and succeeded in current master.

Author: Xingbo Jiang <xingbo.jiang@databricks.com>

Closes #20993 from jiangxb1987/JobCancellationSuite.
2018-04-09 10:19:22 -07:00
Kazuaki Ishizaki 8d40a79a07 [SPARK-23893][CORE][SQL] Avoid possible integer overflow in multiplication
## What changes were proposed in this pull request?

This PR avoids possible overflow at an operation `long = (long)(int * int)`. The multiplication of large positive integer values may set one to MSB. This leads to a negative value in long while we expected a positive value (e.g. `0111_0000_0000_0000 * 0000_0000_0000_0010`).

This PR performs long cast before the multiplication to avoid this situation.

## How was this patch tested?

Existing UTs

Author: Kazuaki Ishizaki <ishizaki@jp.ibm.com>

Closes #21002 from kiszk/SPARK-23893.
2018-04-08 20:40:27 +02:00
Kazuaki Ishizaki 4807d381bb [SPARK-10399][CORE][SQL] Introduce multiple MemoryBlocks to choose several types of memory block
## What changes were proposed in this pull request?

This PR allows us to use one of several types of `MemoryBlock`, such as byte array, int array, long array, or `java.nio.DirectByteBuffer`. To use `java.nio.DirectByteBuffer` allows to have off heap memory which is automatically deallocated by JVM. `MemoryBlock`  class has primitive accessors like `Platform.getInt()`, `Platform.putint()`, or `Platform.copyMemory()`.

This PR uses `MemoryBlock` for `OffHeapColumnVector`, `UTF8String`, and other places. This PR can improve performance of operations involving memory accesses (e.g. `UTF8String.trim`) by 1.8x.

For now, this PR does not use `MemoryBlock` for `BufferHolder` based on cloud-fan's [suggestion](https://github.com/apache/spark/pull/11494#issuecomment-309694290).

Since this PR is a successor of #11494, close #11494. Many codes were ported from #11494. Many efforts were put here. **I think this PR should credit to yzotov.**

This PR can achieve **1.1-1.4x performance improvements** for  operations in `UTF8String` or `Murmur3_x86_32`. Other operations are almost comparable performances.

Without this PR
```
OpenJDK 64-Bit Server VM 1.8.0_121-8u121-b13-0ubuntu1.16.04.2-b13 on Linux 4.4.0-22-generic
Intel(R) Xeon(R) CPU E5-2667 v3  3.20GHz
OpenJDK 64-Bit Server VM 1.8.0_121-8u121-b13-0ubuntu1.16.04.2-b13 on Linux 4.4.0-22-generic
Intel(R) Xeon(R) CPU E5-2667 v3  3.20GHz
Hash byte arrays with length 268435487:  Best/Avg Time(ms)    Rate(M/s)   Per Row(ns)   Relative
------------------------------------------------------------------------------------------------
Murmur3_x86_32                                 526 /  536          0.0   131399881.5       1.0X

UTF8String benchmark:                    Best/Avg Time(ms)    Rate(M/s)   Per Row(ns)   Relative
------------------------------------------------------------------------------------------------
hashCode                                       525 /  552       1022.6           1.0       1.0X
substring                                      414 /  423       1298.0           0.8       1.3X
```

With this PR
```
OpenJDK 64-Bit Server VM 1.8.0_121-8u121-b13-0ubuntu1.16.04.2-b13 on Linux 4.4.0-22-generic
Intel(R) Xeon(R) CPU E5-2667 v3  3.20GHz
Hash byte arrays with length 268435487:  Best/Avg Time(ms)    Rate(M/s)   Per Row(ns)   Relative
------------------------------------------------------------------------------------------------
Murmur3_x86_32                                 474 /  488          0.0   118552232.0       1.0X

UTF8String benchmark:                    Best/Avg Time(ms)    Rate(M/s)   Per Row(ns)   Relative
------------------------------------------------------------------------------------------------
hashCode                                       476 /  480       1127.3           0.9       1.0X
substring                                      287 /  291       1869.9           0.5       1.7X
```

Benchmark program
```
test("benchmark Murmur3_x86_32") {
  val length = 8192 * 32768 + 31
  val seed = 42L
  val iters = 1 << 2
  val random = new Random(seed)
  val arrays = Array.fill[MemoryBlock](numArrays) {
    val bytes = new Array[Byte](length)
    random.nextBytes(bytes)
    new ByteArrayMemoryBlock(bytes, Platform.BYTE_ARRAY_OFFSET, length)
  }

  val benchmark = new Benchmark("Hash byte arrays with length " + length,
    iters * numArrays, minNumIters = 20)
  benchmark.addCase("HiveHasher") { _: Int =>
    var sum = 0L
    for (_ <- 0L until iters) {
      sum += HiveHasher.hashUnsafeBytesBlock(
        arrays(i), Platform.BYTE_ARRAY_OFFSET, length)
    }
  }
  benchmark.run()
}

test("benchmark UTF8String") {
  val N = 512 * 1024 * 1024
  val iters = 2
  val benchmark = new Benchmark("UTF8String benchmark", N, minNumIters = 20)
  val str0 = new java.io.StringWriter() { { for (i <- 0 until N) { write(" ") } } }.toString
  val s0 = UTF8String.fromString(str0)
  benchmark.addCase("hashCode") { _: Int =>
    var h: Int = 0
    for (_ <- 0L until iters) { h += s0.hashCode }
  }
  benchmark.addCase("substring") { _: Int =>
    var s: UTF8String = null
    for (_ <- 0L until iters) { s = s0.substring(N / 2 - 5, N / 2 + 5) }
  }
  benchmark.run()
}
```

I run [this benchmark program](https://gist.github.com/kiszk/94f75b506c93a663bbbc372ffe8f05de) using [the commit](ee5a79861c). I got the following results:

```
OpenJDK 64-Bit Server VM 1.8.0_151-8u151-b12-0ubuntu0.16.04.2-b12 on Linux 4.4.0-66-generic
Intel(R) Xeon(R) CPU E5-2667 v3  3.20GHz
Memory access benchmarks:                Best/Avg Time(ms)    Rate(M/s)   Per Row(ns)   Relative
------------------------------------------------------------------------------------------------
ByteArrayMemoryBlock get/putInt()              220 /  221        609.3           1.6       1.0X
Platform get/putInt(byte[])                    220 /  236        610.9           1.6       1.0X
Platform get/putInt(Object)                    492 /  494        272.8           3.7       0.4X
OnHeapMemoryBlock get/putLong()                322 /  323        416.5           2.4       0.7X
long[]                                         221 /  221        608.0           1.6       1.0X
Platform get/putLong(long[])                   321 /  321        418.7           2.4       0.7X
Platform get/putLong(Object)                   561 /  563        239.2           4.2       0.4X
```

I also run [this benchmark program](https://gist.github.com/kiszk/5fdb4e03733a5d110421177e289d1fb5) for comparing performance of `Platform.copyMemory()`.
```
OpenJDK 64-Bit Server VM 1.8.0_151-8u151-b12-0ubuntu0.16.04.2-b12 on Linux 4.4.0-66-generic
Intel(R) Xeon(R) CPU E5-2667 v3  3.20GHz
Platform copyMemory:                     Best/Avg Time(ms)    Rate(M/s)   Per Row(ns)   Relative
------------------------------------------------------------------------------------------------
Object to Object                              1961 / 1967          8.6         116.9       1.0X
System.arraycopy Object to Object             1917 / 1921          8.8         114.3       1.0X
byte array to byte array                      1961 / 1968          8.6         116.9       1.0X
System.arraycopy byte array to byte array      1909 / 1937          8.8         113.8       1.0X
int array to int array                        1921 / 1990          8.7         114.5       1.0X
double array to double array                  1918 / 1923          8.7         114.3       1.0X
Object to byte array                          1961 / 1967          8.6         116.9       1.0X
Object to short array                         1965 / 1972          8.5         117.1       1.0X
Object to int array                           1910 / 1915          8.8         113.9       1.0X
Object to float array                         1971 / 1978          8.5         117.5       1.0X
Object to double array                        1919 / 1944          8.7         114.4       1.0X
byte array to Object                          1959 / 1967          8.6         116.8       1.0X
int array to Object                           1961 / 1970          8.6         116.9       1.0X
double array to Object                        1917 / 1924          8.8         114.3       1.0X
```

These results show three facts:
1. According to the second/third or sixth/seventh results in the first experiment, if we use `Platform.get/putInt(Object)`, we achieve more than 2x worse performance than `Platform.get/putInt(byte[])` with concrete type (i.e. `byte[]`).
2. According to the second/third or fourth/fifth/sixth results in the first experiment, the fastest way to access an array element on Java heap is `array[]`. **Cons of `array[]` is that it is not possible to support unaligned-8byte access.**
3. According to the first/second/third or fourth/sixth/seventh results in the first experiment, `getInt()/putInt() or getLong()/putLong()` in subclasses of `MemoryBlock` can achieve comparable performance to `Platform.get/putInt()` or `Platform.get/putLong()` with concrete type (second or sixth result). There is no overhead regarding virtual call.
4. According to results in the second experiment, for `Platform.copy()`, to pass `Object` can achieve the same performance as to pass any type of primitive array as source or destination.
5. According to second/fourth results in the second experiment, `Platform.copy()` can achieve the same performance as `System.arrayCopy`. **It would be good to use `Platform.copy()` since `Platform.copy()` can take any types for src and dst.**

We are incrementally replace `Platform.get/putXXX` with `MemoryBlock.get/putXXX`. This is because we have two advantages.
1) Achieve better performance due to having a concrete type for an array.
2) Use simple OO design instead of passing `Object`
It is easy to use `MemoryBlock` in `InternalRow`, `BufferHolder`, `TaskMemoryManager`, and others that are already abstracted. It is not easy to use `MemoryBlock` in utility classes related to hashing or others.

Other candidates are
- UnsafeRow, UnsafeArrayData, UnsafeMapData, SpecificUnsafeRowJoiner
- UTF8StringBuffer
- BufferHolder
- TaskMemoryManager
- OnHeapColumnVector
- BytesToBytesMap
- CachedBatch
- classes for hash
- others.

## How was this patch tested?

Added `UnsafeMemoryAllocator`

Author: Kazuaki Ishizaki <ishizaki@jp.ibm.com>

Closes #19222 from kiszk/SPARK-10399.
2018-04-06 10:13:59 +08:00
Xingbo Jiang 7cf9fab334 [MINOR][CORE] Show block manager id when remove RDD/Broadcast fails.
## What changes were proposed in this pull request?

Address https://github.com/apache/spark/pull/20924#discussion_r177987175, show block manager id when remove RDD/Broadcast fails.

## How was this patch tested?

N/A

Author: Xingbo Jiang <xingbo.jiang@databricks.com>

Closes #20960 from jiangxb1987/bmid.
2018-04-03 21:26:49 +08:00
Marcelo Vanzin 441d0d0766 [SPARK-19964][CORE] Avoid reading from remote repos in SparkSubmitSuite.
These tests can fail with a timeout if the remote repos are not responding,
or slow. The tests don't need anything from those repos, so use an empty
ivy config file to avoid setting up the defaults.

The tests are passing reliably for me locally now, and failing more often
than not today without this change since http://dl.bintray.com/spark-packages/maven
doesn't seem to be loading from my machine.

Author: Marcelo Vanzin <vanzin@cloudera.com>

Closes #20916 from vanzin/SPARK-19964.
2018-04-03 09:31:47 +08:00
Xingbo Jiang 529f847105 [SPARK-23040][CORE][FOLLOW-UP] Avoid double wrap result Iterator.
## What changes were proposed in this pull request?

Address https://github.com/apache/spark/pull/20449#discussion_r172414393, If `resultIter` is already a `InterruptibleIterator`, don't double wrap it.

## How was this patch tested?
Existing tests.

Author: Xingbo Jiang <xingbo.jiang@databricks.com>

Closes #20920 from jiangxb1987/SPARK-23040.
2018-03-31 10:34:01 +08:00
Yuming Wang ae9172017c [SPARK-23640][CORE] Fix hadoop config may override spark config
## What changes were proposed in this pull request?

It may be get `spark.shuffle.service.port` from 9745ec3a61/core/src/main/scala/org/apache/spark/deploy/SparkHadoopUtil.scala (L459)

Therefore, the client configuration `spark.shuffle.service.port` does not working unless the configuration is `spark.hadoop.spark.shuffle.service.port`.

- This configuration is not working:
```
bin/spark-sql --master yarn --conf spark.shuffle.service.port=7338
```
- This configuration works:
```
bin/spark-sql --master yarn --conf spark.hadoop.spark.shuffle.service.port=7338
```

This PR fix this issue.

## How was this patch tested?

It's difficult to carry out unit testing. But I've tested it manually.

Author: Yuming Wang <yumwang@ebay.com>

Closes #20785 from wangyum/SPARK-23640.
2018-03-30 14:09:14 -07:00
Kent Yao a7755fd8ce [SPARK-23639][SQL] Obtain token before init metastore client in SparkSQL CLI
## What changes were proposed in this pull request?

In SparkSQLCLI, SessionState generates before SparkContext instantiating. When we use --proxy-user to impersonate, it's unable to initializing a metastore client to talk to the secured metastore for no kerberos ticket.

This PR use real user ugi to obtain token for owner before talking to kerberized metastore.

## How was this patch tested?

Manually verified with kerberized hive metasotre / hdfs.

Author: Kent Yao <yaooqinn@hotmail.com>

Closes #20784 from yaooqinn/SPARK-23639.
2018-03-29 10:46:28 -07:00
Sahil Takiar 491ec114fd [SPARK-23785][LAUNCHER] LauncherBackend doesn't check state of connection before setting state
## What changes were proposed in this pull request?

Changed `LauncherBackend` `set` method so that it checks if the connection is open or not before writing to it (uses `isConnected`).

## How was this patch tested?

None

Author: Sahil Takiar <stakiar@cloudera.com>

Closes #20893 from sahilTakiar/master.
2018-03-29 10:23:23 -07:00
Thomas Graves 641aec68e8 [SPARK-23806] Broadcast.unpersist can cause fatal exception when used…
… with dynamic allocation

## What changes were proposed in this pull request?

ignore errors when you are waiting for a broadcast.unpersist. This is handling it the same way as doing rdd.unpersist in https://issues.apache.org/jira/browse/SPARK-22618

## How was this patch tested?

Patch was tested manually against a couple jobs that exhibit this behavior, with the change the application no longer dies due to this and just prints the warning.

Please review http://spark.apache.org/contributing.html before opening a pull request.

Author: Thomas Graves <tgraves@unharmedunarmed.corp.ne1.yahoo.com>

Closes #20924 from tgravescs/SPARK-23806.
2018-03-29 16:37:46 +08:00
guoxiaolong ea2fdc0d28 [SPARK-23675][WEB-UI] Title add spark logo, use spark logo image
## What changes were proposed in this pull request?

Title add spark logo, use spark logo image. reference other big data system ui, so i think spark should add it.

spark fix before:
![spark_fix_before](https://user-images.githubusercontent.com/26266482/37387866-2d5add0e-2799-11e8-9165-250f2b59df3f.png)

spark fix after:
![spark_fix_after](https://user-images.githubusercontent.com/26266482/37387874-329e1876-2799-11e8-8bc5-c619fc1e680e.png)

reference kafka ui:
![kafka](https://user-images.githubusercontent.com/26266482/37387878-35ca89d0-2799-11e8-834e-1598ae7158e1.png)

reference storm ui:
![storm](https://user-images.githubusercontent.com/26266482/37387880-3854f12c-2799-11e8-8968-b428ba361995.png)

reference yarn ui:
![yarn](https://user-images.githubusercontent.com/26266482/37387881-3a72e130-2799-11e8-97bb-dea85f573e95.png)

reference nifi ui:
![nifi](https://user-images.githubusercontent.com/26266482/37387887-3cecfea0-2799-11e8-9a71-6c454d25840b.png)

reference flink ui:
![flink](https://user-images.githubusercontent.com/26266482/37387888-3f16b1ee-2799-11e8-9d37-8355f0100548.png)

## How was this patch tested?

manual tests

Please review http://spark.apache.org/contributing.html before opening a pull request.

Author: guoxiaolong <guo.xiaolong1@zte.com.cn>

Closes #20818 from guoxiaolongzte/SPARK-23675.
2018-03-28 19:49:32 -05:00
Marcelo Vanzin b30a7d28b3 [SPARK-23572][DOCS] Bring "security.md" up to date.
This change basically rewrites the security documentation so that it's
up to date with new features, more correct, and more complete.

Because security is such an important feature, I chose to move all the
relevant configuration documentation to the security page, instead of
having them peppered all over the place in the configuration page. This
allows an almost one-stop shop for security configuration in Spark. The
only exceptions are some YARN-specific minor features which I left in
the YARN page.

I also re-organized the page's topics, since they didn't make a lot of
sense. You had kerberos features described inside paragraphs talking
about UI access control, and other oddities. It should be easier now
to find information about specific Spark security features. I also
enabled TOCs for both the Security and YARN pages, since that makes it
easier to see what is covered.

I removed most of the comments from the SecurityManager javadoc since
they just replicated information in the security doc, with different
levels of out-of-dateness.

Author: Marcelo Vanzin <vanzin@cloudera.com>

Closes #20742 from vanzin/SPARK-23572.
2018-03-26 12:45:45 -07:00
Marcelo Vanzin eb48edf9ca [SPARK-23787][TESTS] Fix file download test in SparkSubmitSuite for Hadoop 2.9.
This particular test assumed that Hadoop libraries did not support
http as a file system. Hadoop 2.9 does, so the test failed. The test
now forces a non-existent implementation for the http fs, which
forces the expected error.

There were also a couple of other issues in the same test: SparkSubmit
arguments in the wrong order, and the wrong check later when asserting,
which was being masked by the previous issues.

Author: Marcelo Vanzin <vanzin@cloudera.com>

Closes #20895 from vanzin/SPARK-23787.
2018-03-26 14:01:04 +08:00
bag_of_tricks 8b56f16640 [SPARK-23759][UI] Unable to bind Spark UI to specific host name / IP
## What changes were proposed in this pull request?

Fixes SPARK-23759 by moving connector.start() after connector.setHost()

Problem was created due connector.setHost(hostName) call was after connector.start()

## How was this patch tested?

Patch was tested after build and deployment. This patch requires SPARK_LOCAL_IP environment variable to be set on spark-env.sh

Author: bag_of_tricks <falbani@hortonworks.com>

Closes #20883 from felixalbani/SPARK-23759.
2018-03-23 10:36:23 -07:00
arucard21 6ac4fba692 [SPARK-23769][CORE] Remove comments that unnecessarily disable Scalastyle check
## What changes were proposed in this pull request?

We re-enabled the Scalastyle checker on a line of code. It was previously disabled, but it does not violate any of the rules. So there's no reason to disable the Scalastyle checker here.

## How was this patch tested?

We tested this by running `build/mvn scalastyle:check` after removing the comments that disable the checker. This check passed with no errors or warnings for Spark Core
```
[INFO]
[INFO] ------------------------------------------------------------------------
[INFO] Building Spark Project Core 2.4.0-SNAPSHOT
[INFO] ------------------------------------------------------------------------
[INFO]
[INFO] --- scalastyle-maven-plugin:1.0.0:check (default-cli)  spark-core_2.11 ---
Saving to outputFile=<path to local dir>/spark/core/target/scalastyle-output.xml
Processed 485 file(s)
Found 0 errors
Found 0 warnings
Found 0 infos
```
We did not run all tests (with `dev/run-tests`) since this Scalastyle check seemed sufficient.

## Co-contributors:
chialun-yeh
Hrayo712
vpourquie

Author: arucard21 <arucard21@gmail.com>

Closes #20880 from arucard21/scalastyle_util.
2018-03-23 21:02:34 +09:00
Marcelo Vanzin 5fa4384711 [SPARK-23361][YARN] Allow AM to restart after initial tokens expire.
Currently, the Spark AM relies on the initial set of tokens created by
the submission client to be able to talk to HDFS and other services that
require delegation tokens. This means that after those tokens expire, a
new AM will fail to start (e.g. when there is an application failure and
re-attempts are enabled).

This PR makes it so that the first thing the AM does when the user provides
a principal and keytab is to create new delegation tokens for use. This
makes sure that the AM can be started irrespective of how old the original
token set is. It also allows all of the token management to be done by the
AM - there is no need for the submission client to set configuration values
to tell the AM when to renew tokens.

Note that even though in this case the AM will not be using the delegation
tokens created by the submission client, those tokens still need to be provided
to YARN, since they are used to do log aggregation.

To be able to re-use the code in the AMCredentialRenewal for the above
purposes, I refactored that class a bit so that it can fetch tokens into
a pre-defined UGI, insted of always logging in.

Another issue with re-attempts is that, after the fix that allows the AM
to restart correctly, new executors would get confused about when to
update credentials, because the credential updater used the update time
initially set up by the submission code. This could make the executor
fail to update credentials in time, since that value would be very out
of date in the situation described in the bug.

To fix that, I changed the YARN code to use the new RPC-based mechanism
for distributing tokens to executors. This allowed the old credential
updater code to be removed, and a lot of code in the renewer to be
simplified.

I also made two currently hardcoded values (the renewal time ratio, and
the retry wait) configurable; while this probably never needs to be set
by anyone in a production environment, it helps with testing; that's also
why they're not documented.

Tested on real cluster with a specially crafted application to test this
functionality: checked proper access to HDFS, Hive and HBase in cluster
mode with token renewal on and AM restarts. Tested things still work in
client mode too.

Author: Marcelo Vanzin <vanzin@cloudera.com>

Closes #20657 from vanzin/SPARK-23361.
2018-03-23 13:59:21 +08:00
Mihaly Toth 0604beaff2 [SPARK-23729][CORE] Respect URI fragment when resolving globs
Firstly, glob resolution will not result in swallowing the remote name part (that is preceded by the `#` sign) in case of `--files` or `--archives` options

Moreover in the special case of multiple resolutions when the remote naming does not make sense and error is returned.

Enhanced current test and wrote additional test for the error case

Author: Mihaly Toth <misutoth@gmail.com>

Closes #20853 from misutoth/glob-with-remote-name.
2018-03-21 17:06:22 -07:00
Ilan Filonenko f15906da15 [SPARK-22839][K8S] Remove the use of init-container for downloading remote dependencies
## What changes were proposed in this pull request?

Removal of the init-container for downloading remote dependencies. Built off of the work done by vanzin in an attempt to refactor driver/executor configuration elaborated in [this](https://issues.apache.org/jira/browse/SPARK-22839) ticket.

## How was this patch tested?

This patch was tested with unit and integration tests.

Author: Ilan Filonenko <if56@cornell.edu>

Closes #20669 from ifilonenko/remove-init-container.
2018-03-19 11:29:56 -07:00
zhoukang 745c8c0901 [SPARK-23708][CORE] Correct comment for function addShutDownHook in ShutdownHookManager
## What changes were proposed in this pull request?
Minor modification.Comment below is not right.
```
/**
   * Adds a shutdown hook with the given priority. Hooks with lower priority values run
   * first.
   *
   * param hook The code to run during shutdown.
   * return A handle that can be used to unregister the shutdown hook.
   */
  def addShutdownHook(priority: Int)(hook: () => Unit): AnyRef = {
    shutdownHooks.add(priority, hook)
  }
```

## How was this patch tested?

UT

Author: zhoukang <zhoukang199191@gmail.com>

Closes #20845 from caneGuy/zhoukang/fix-shutdowncomment.
2018-03-19 13:31:21 +08:00
Steve Loughran 8a1efe3076 [SPARK-23683][SQL] FileCommitProtocol.instantiate() hardening
## What changes were proposed in this pull request?

With SPARK-20236, `FileCommitProtocol.instantiate()` looks for a three argument constructor, passing in the `dynamicPartitionOverwrite` parameter. If there is no such constructor, it falls back to the classic two-arg one.

When `InsertIntoHadoopFsRelationCommand` passes down that `dynamicPartitionOverwrite` flag `to FileCommitProtocol.instantiate(`), it assumes that the instantiated protocol supports the specific requirements of dynamic partition overwrite. It does not notice when this does not hold, and so the output generated may be incorrect.

This patch changes  `FileCommitProtocol.instantiate()` so  when `dynamicPartitionOverwrite == true`, it requires the protocol implementation to have a 3-arg constructor. Classic two arg constructors are supported when it is false.

Also it adds some debug level logging for anyone trying to understand what's going on.

## How was this patch tested?

Unit tests verify that

* classes with only 2-arg constructor cannot be used with dynamic overwrite
* classes with only 2-arg constructor can be used without dynamic overwrite
* classes with 3 arg constructors can be used with both.
* the fallback to any two arg ctor takes place after the attempt to load the 3-arg ctor,
* passing in invalid class types fail as expected (regression tests on expected behavior)

Author: Steve Loughran <stevel@hortonworks.com>

Closes #20824 from steveloughran/stevel/SPARK-23683-protocol-instantiate.
2018-03-16 15:40:21 -07:00
Marco Gaido ca83526de5 [SPARK-23644][CORE][UI] Use absolute path for REST call in SHS
## What changes were proposed in this pull request?

SHS is using a relative path for the REST API call to get the list of the application is a relative path call. In case of the SHS being consumed through a proxy, it can be an issue if the path doesn't end with a "/".

Therefore, we should use an absolute path for the REST call as it is done for all the other resources.

## How was this patch tested?

manual tests
Before the change:
![screen shot 2018-03-10 at 4 22 02 pm](https://user-images.githubusercontent.com/8821783/37244190-8ccf9d40-2485-11e8-8fa9-345bc81472fc.png)

After the change:
![screen shot 2018-03-10 at 4 36 34 pm 1](https://user-images.githubusercontent.com/8821783/37244201-a1922810-2485-11e8-8856-eeab2bf5e180.png)

Author: Marco Gaido <marcogaido91@gmail.com>

Closes #20794 from mgaido91/SPARK-23644.
2018-03-16 15:12:26 +08:00
Ye Zhou 3675af7247 [SPARK-23608][CORE][WEBUI] Add synchronization in SHS between attachSparkUI and detachSparkUI functions to avoid concurrent modification issue to Jetty Handlers
Jetty handlers are dynamically attached/detached while SHS is running. But the attach and detach operations might be taking place at the same time due to the async in load/clear in Guava Cache.

## What changes were proposed in this pull request?
Add synchronization between attachSparkUI and detachSparkUI in SHS.

## How was this patch tested?
With this patch, the jetty handlers missing issue never happens again in our production cluster SHS.

Author: Ye Zhou <yezhou@linkedin.com>

Closes #20744 from zhouyejoe/SPARK-23608.
2018-03-15 17:15:53 -07:00
Marcelo Vanzin 18f8575e01 [SPARK-23671][CORE] Fix condition to enable the SHS thread pool.
Author: Marcelo Vanzin <vanzin@cloudera.com>

Closes #20814 from vanzin/SPARK-23671.
2018-03-15 17:12:01 -07:00
smallory 4f5bad615b [SPARK-23642][DOCS] AccumulatorV2 subclass isZero scaladoc fix
Added/corrected scaladoc for isZero on the DoubleAccumulator, CollectionAccumulator, and LongAccumulator subclasses of AccumulatorV2, particularly noting where there are requirements in addition to having a value of zero in order to return true.

## What changes were proposed in this pull request?

Three scaladoc comments are updated in AccumulatorV2.scala
No changes outside of comment blocks were made.

## How was this patch tested?

Running "sbt unidoc", fixing style errors found, and reviewing the resulting local scaladoc in firefox.

Author: smallory <s.mallory@gmail.com>

Closes #20790 from smallory/patch-1.
2018-03-15 11:58:54 +09:00
Marcelo Vanzin 2c3673680e [SPARK-23630][YARN] Allow user's hadoop conf customizations to take effect.
This change restores functionality that was inadvertently removed as part
of the fix for SPARK-22372.

Also modified an existing unit test to make sure the feature works as intended.

Author: Marcelo Vanzin <vanzin@cloudera.com>

Closes #20776 from vanzin/SPARK-23630.
2018-03-09 10:36:38 -08:00
Maxim Gekk fe22f32041 [SPARK-23620] Splitting thread dump lines by using the br tag
## What changes were proposed in this pull request?

I propose to replace `'\n'` by the `<br>` tag in generated html of thread dump page. The `<br>` tag will split thread lines in more reliable way. For now it could look like on
<img width="1265" alt="the screen shot" src="https://user-images.githubusercontent.com/1580697/37118202-bcd98fc0-2253-11e8-9e61-c2f946869ee0.png">
 if the html is proxied and `'\n'` is replaced by another whitespace. The changes allow to more easily read and copy stack traces.

## How was this patch tested?

I tested it manually by checking the thread dump page and its source.

Author: Maxim Gekk <maxim.gekk@databricks.com>

Closes #20762 from MaxGekk/br-thread-dump.
2018-03-08 10:50:09 +01:00