spark git commit: [SPARK-24244][SPARK-24368][SQL] Passing only required columns to the CSV parser

2018-05-24 Thread lixiao
Repository: spark
Updated Branches:
  refs/heads/master 3b20b34ab -> 64fad0b51


[SPARK-24244][SPARK-24368][SQL] Passing only required columns to the CSV parser

## What changes were proposed in this pull request?

uniVocity parser allows to specify only required column names or indexes for 
[parsing](https://www.univocity.com/pages/parsers-tutorial) like:

```
// Here we select only the columns by their indexes.
// The parser just skips the values in other columns
parserSettings.selectIndexes(4, 0, 1);
CsvParser parser = new CsvParser(parserSettings);
```
In this PR, I propose to extract indexes from required schema and pass them 
into the CSV parser. Benchmarks show the following improvements in parsing of 
1000 columns:

```
Select 100 columns out of 1000: x1.76
Select 1 column out of 1000: x2
```

**Note**: Comparing to current implementation, the changes can return different 
result for malformed rows in the `DROPMALFORMED` and `FAILFAST` modes if only 
subset of all columns is requested. To have previous behavior, set 
`spark.sql.csv.parser.columnPruning.enabled` to `false`.

## How was this patch tested?

It was tested by new test which selects 3 columns out of 15, by existing tests 
and by new benchmarks.

Author: Maxim Gekk 
Author: Maxim Gekk 

Closes #21415 from MaxGekk/csv-column-pruning2.


Project: http://git-wip-us.apache.org/repos/asf/spark/repo
Commit: http://git-wip-us.apache.org/repos/asf/spark/commit/64fad0b5
Tree: http://git-wip-us.apache.org/repos/asf/spark/tree/64fad0b5
Diff: http://git-wip-us.apache.org/repos/asf/spark/diff/64fad0b5

Branch: refs/heads/master
Commit: 64fad0b519cf35b8c0a0dec18dd3df9488a5ed25
Parents: 3b20b34
Author: Maxim Gekk 
Authored: Thu May 24 21:38:04 2018 -0700
Committer: Xiao Li 
Committed: Thu May 24 21:38:04 2018 -0700

--
 docs/sql-programming-guide.md   |  1 +
 .../org/apache/spark/sql/internal/SQLConf.scala |  9 
 .../org/apache/spark/sql/DataFrameReader.scala  |  1 +
 .../datasources/csv/CSVFileFormat.scala | 18 +---
 .../execution/datasources/csv/CSVOptions.scala  |  3 ++
 .../datasources/csv/UnivocityParser.scala   | 26 ++-
 .../datasources/csv/CSVBenchmarks.scala | 46 
 .../datasources/csv/CSVInferSchemaSuite.scala   | 22 +-
 .../execution/datasources/csv/CSVSuite.scala| 41 ++---
 .../datasources/csv/UnivocityParserSuite.scala  | 37 
 10 files changed, 152 insertions(+), 52 deletions(-)
--


http://git-wip-us.apache.org/repos/asf/spark/blob/64fad0b5/docs/sql-programming-guide.md
--
diff --git a/docs/sql-programming-guide.md b/docs/sql-programming-guide.md
index f1ed316..fc26562 100644
--- a/docs/sql-programming-guide.md
+++ b/docs/sql-programming-guide.md
@@ -1825,6 +1825,7 @@ working with timestamps in `pandas_udf`s to get the best 
performance, see
   - In version 2.3 and earlier, `to_utc_timestamp` and `from_utc_timestamp` 
respect the timezone in the input timestamp string, which breaks the assumption 
that the input timestamp is in a specific timezone. Therefore, these 2 
functions can return unexpected results. In version 2.4 and later, this problem 
has been fixed. `to_utc_timestamp` and `from_utc_timestamp` will return null if 
the input timestamp string contains timezone. As an example, 
`from_utc_timestamp('2000-10-10 00:00:00', 'GMT+1')` will return `2000-10-10 
01:00:00` in both Spark 2.3 and 2.4. However, `from_utc_timestamp('2000-10-10 
00:00:00+00:00', 'GMT+1')`, assuming a local timezone of GMT+8, will return 
`2000-10-10 09:00:00` in Spark 2.3 but `null` in 2.4. For people who don't care 
about this problem and want to retain the previous behaivor to keep their query 
unchanged, you can set `spark.sql.function.rejectTimezoneInString` to false. 
This option will be removed in Spark 3.0 and should only be used as a temporary 
w
 orkaround.
   - In version 2.3 and earlier, Spark converts Parquet Hive tables by default 
but ignores table properties like `TBLPROPERTIES (parquet.compression 'NONE')`. 
This happens for ORC Hive table properties like `TBLPROPERTIES (orc.compress 
'NONE')` in case of `spark.sql.hive.convertMetastoreOrc=true`, too. Since Spark 
2.4, Spark respects Parquet/ORC specific table properties while converting 
Parquet/ORC Hive tables. As an example, `CREATE TABLE t(id int) STORED AS 
PARQUET TBLPROPERTIES (parquet.compression 'NONE')` would generate Snappy 
parquet files during insertion in Spark 2.3, and in Spark 2.4, the result would 
be uncompressed parquet files.
   - Since Spark 2.0, Spark converts Parquet Hive tables by default for better 
performance. Since Spark 2.4, Spark converts ORC Hive tables by 

svn commit: r27097 - in /dev/spark/2.4.0-SNAPSHOT-2018_05_24_20_01-0fd68cb-docs: ./ _site/ _site/api/ _site/api/R/ _site/api/java/ _site/api/java/lib/ _site/api/java/org/ _site/api/java/org/apache/ _s

2018-05-24 Thread pwendell
Author: pwendell
Date: Fri May 25 03:15:45 2018
New Revision: 27097

Log:
Apache Spark 2.4.0-SNAPSHOT-2018_05_24_20_01-0fd68cb docs


[This commit notification would consist of 1464 parts, 
which exceeds the limit of 50 ones, so it was shortened to the summary.]

-
To unsubscribe, e-mail: commits-unsubscr...@spark.apache.org
For additional commands, e-mail: commits-h...@spark.apache.org



spark git commit: [SPARK-24367][SQL] Parquet: use JOB_SUMMARY_LEVEL instead of deprecated flag ENABLE_JOB_SUMMARY

2018-05-24 Thread gurwls223
Repository: spark
Updated Branches:
  refs/heads/master 0fd68cb72 -> 3b20b34ab


[SPARK-24367][SQL] Parquet: use JOB_SUMMARY_LEVEL instead of deprecated flag 
ENABLE_JOB_SUMMARY

## What changes were proposed in this pull request?

In current parquet version,the conf ENABLE_JOB_SUMMARY is deprecated.

When writing to Parquet files, the warning message
```WARN org.apache.parquet.hadoop.ParquetOutputFormat: Setting 
parquet.enable.summary-metadata is deprecated, please use 
parquet.summary.metadata.level```
keeps showing up.

>From 
>https://github.com/apache/parquet-mr/blame/master/parquet-hadoop/src/main/java/org/apache/parquet/hadoop/ParquetOutputFormat.java#L164
> we can see that we should use JOB_SUMMARY_LEVEL.

## How was this patch tested?

Unit test

Author: Gengliang Wang 

Closes #21411 from gengliangwang/summaryLevel.


Project: http://git-wip-us.apache.org/repos/asf/spark/repo
Commit: http://git-wip-us.apache.org/repos/asf/spark/commit/3b20b34a
Tree: http://git-wip-us.apache.org/repos/asf/spark/tree/3b20b34a
Diff: http://git-wip-us.apache.org/repos/asf/spark/diff/3b20b34a

Branch: refs/heads/master
Commit: 3b20b34ab72c92d9d20188ed430955e1a94eac9c
Parents: 0fd68cb
Author: Gengliang Wang 
Authored: Fri May 25 11:16:35 2018 +0800
Committer: hyukjinkwon 
Committed: Fri May 25 11:16:35 2018 +0800

--
 .../scala/org/apache/spark/sql/internal/SQLConf.scala |  2 +-
 .../execution/datasources/parquet/ParquetFileFormat.scala | 10 ++
 .../datasources/parquet/ParquetCommitterSuite.scala   |  7 ++-
 .../execution/datasources/parquet/ParquetIOSuite.scala|  2 +-
 .../parquet/ParquetPartitionDiscoverySuite.scala  |  2 +-
 .../execution/datasources/parquet/ParquetQuerySuite.scala |  2 +-
 .../spark/sql/sources/ParquetHadoopFsRelationSuite.scala  |  2 +-
 7 files changed, 17 insertions(+), 10 deletions(-)
--


http://git-wip-us.apache.org/repos/asf/spark/blob/3b20b34a/sql/catalyst/src/main/scala/org/apache/spark/sql/internal/SQLConf.scala
--
diff --git 
a/sql/catalyst/src/main/scala/org/apache/spark/sql/internal/SQLConf.scala 
b/sql/catalyst/src/main/scala/org/apache/spark/sql/internal/SQLConf.scala
index 15ba10f..93d356f 100644
--- a/sql/catalyst/src/main/scala/org/apache/spark/sql/internal/SQLConf.scala
+++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/internal/SQLConf.scala
@@ -395,7 +395,7 @@ object SQLConf {
 .doc("The output committer class used by Parquet. The specified class 
needs to be a " +
   "subclass of org.apache.hadoop.mapreduce.OutputCommitter. Typically, 
it's also a subclass " +
   "of org.apache.parquet.hadoop.ParquetOutputCommitter. If it is not, then 
metadata summaries" +
-  "will never be created, irrespective of the value of 
parquet.enable.summary-metadata")
+  "will never be created, irrespective of the value of 
parquet.summary.metadata.level")
 .internal()
 .stringConf
 .createWithDefault("org.apache.parquet.hadoop.ParquetOutputCommitter")

http://git-wip-us.apache.org/repos/asf/spark/blob/3b20b34a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetFileFormat.scala
--
diff --git 
a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetFileFormat.scala
 
b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetFileFormat.scala
index d1f9e11..60fc9ec 100644
--- 
a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetFileFormat.scala
+++ 
b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetFileFormat.scala
@@ -34,6 +34,7 @@ import org.apache.parquet.filter2.compat.FilterCompat
 import org.apache.parquet.filter2.predicate.FilterApi
 import 
org.apache.parquet.format.converter.ParquetMetadataConverter.SKIP_ROW_GROUPS
 import org.apache.parquet.hadoop._
+import org.apache.parquet.hadoop.ParquetOutputFormat.JobSummaryLevel
 import org.apache.parquet.hadoop.codec.CodecConfig
 import org.apache.parquet.hadoop.util.ContextUtil
 import org.apache.parquet.schema.MessageType
@@ -125,16 +126,17 @@ class ParquetFileFormat
 conf.set(ParquetOutputFormat.COMPRESSION, 
parquetOptions.compressionCodecClassName)
 
 // SPARK-15719: Disables writing Parquet summary files by default.
-if (conf.get(ParquetOutputFormat.ENABLE_JOB_SUMMARY) == null) {
-  conf.setBoolean(ParquetOutputFormat.ENABLE_JOB_SUMMARY, false)
+if (conf.get(ParquetOutputFormat.JOB_SUMMARY_LEVEL) == null
+  && conf.get(ParquetOutputFormat.ENABLE_JOB_SUMMARY) == null) {
+  conf.setEnum(ParquetOutputFormat.JOB_SUMMARY_LEVEL, JobSummaryLevel.NONE)
 }
 

spark git commit: [SPARK-24234][SS] Support multiple row writers in continuous processing shuffle reader.

2018-05-24 Thread tdas
Repository: spark
Updated Branches:
  refs/heads/master 53c06ddab -> 0fd68cb72


[SPARK-24234][SS] Support multiple row writers in continuous processing shuffle 
reader.

## What changes were proposed in this pull request?

https://docs.google.com/document/d/1IL4kJoKrZWeyIhklKUJqsW-yEN7V7aL05MmM65AYOfE/edit#heading=h.8t3ci57f7uii

Support multiple different row writers in continuous processing shuffle reader.

Note that having multiple read-side buffers ended up being the natural way to 
do this. Otherwise it's hard to express the constraint of sending an epoch 
marker only when all writers have sent one.

## How was this patch tested?

new unit tests

Author: Jose Torres 

Closes #21385 from jose-torres/multipleWrite.


Project: http://git-wip-us.apache.org/repos/asf/spark/repo
Commit: http://git-wip-us.apache.org/repos/asf/spark/commit/0fd68cb7
Tree: http://git-wip-us.apache.org/repos/asf/spark/tree/0fd68cb7
Diff: http://git-wip-us.apache.org/repos/asf/spark/diff/0fd68cb7

Branch: refs/heads/master
Commit: 0fd68cb7278e5fdf106e73b580ee7dd829006386
Parents: 53c06dd
Author: Jose Torres 
Authored: Thu May 24 17:08:52 2018 -0700
Committer: Tathagata Das 
Committed: Thu May 24 17:08:52 2018 -0700

--
 .../shuffle/ContinuousShuffleReadRDD.scala  |  21 ++-
 .../continuous/shuffle/UnsafeRowReceiver.scala  |  87 --
 .../shuffle/ContinuousShuffleReadSuite.scala| 163 +++
 3 files changed, 227 insertions(+), 44 deletions(-)
--


http://git-wip-us.apache.org/repos/asf/spark/blob/0fd68cb7/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/continuous/shuffle/ContinuousShuffleReadRDD.scala
--
diff --git 
a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/continuous/shuffle/ContinuousShuffleReadRDD.scala
 
b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/continuous/shuffle/ContinuousShuffleReadRDD.scala
index 270b1a5..801b28b 100644
--- 
a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/continuous/shuffle/ContinuousShuffleReadRDD.scala
+++ 
b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/continuous/shuffle/ContinuousShuffleReadRDD.scala
@@ -25,11 +25,16 @@ import org.apache.spark.sql.catalyst.expressions.UnsafeRow
 import org.apache.spark.sql.internal.SQLConf
 import org.apache.spark.util.NextIterator
 
-case class ContinuousShuffleReadPartition(index: Int, queueSize: Int) extends 
Partition {
+case class ContinuousShuffleReadPartition(
+  index: Int,
+  queueSize: Int,
+  numShuffleWriters: Int,
+  epochIntervalMs: Long)
+extends Partition {
   // Initialized only on the executor, and only once even as we call compute() 
multiple times.
   lazy val (reader: ContinuousShuffleReader, endpoint) = {
 val env = SparkEnv.get.rpcEnv
-val receiver = new UnsafeRowReceiver(queueSize, env)
+val receiver = new UnsafeRowReceiver(queueSize, numShuffleWriters, 
epochIntervalMs, env)
 val endpoint = 
env.setupEndpoint(s"UnsafeRowReceiver-${UUID.randomUUID()}", receiver)
 TaskContext.get().addTaskCompletionListener { ctx =>
   env.stop(endpoint)
@@ -42,16 +47,24 @@ case class ContinuousShuffleReadPartition(index: Int, 
queueSize: Int) extends Pa
  * RDD at the map side of each continuous processing shuffle task. Upstream 
tasks send their
  * shuffle output to the wrapped receivers in partitions of this RDD; each of 
the RDD's tasks
  * poll from their receiver until an epoch marker is sent.
+ *
+ * @param sc the RDD context
+ * @param numPartitions the number of read partitions for this RDD
+ * @param queueSize the size of the row buffers to use
+ * @param numShuffleWriters the number of continuous shuffle writers feeding 
into this RDD
+ * @param epochIntervalMs the checkpoint interval of the streaming query
  */
 class ContinuousShuffleReadRDD(
 sc: SparkContext,
 numPartitions: Int,
-queueSize: Int = 1024)
+queueSize: Int = 1024,
+numShuffleWriters: Int = 1,
+epochIntervalMs: Long = 1000)
   extends RDD[UnsafeRow](sc, Nil) {
 
   override protected def getPartitions: Array[Partition] = {
 (0 until numPartitions).map { partIndex =>
-  ContinuousShuffleReadPartition(partIndex, queueSize)
+  ContinuousShuffleReadPartition(partIndex, queueSize, numShuffleWriters, 
epochIntervalMs)
 }.toArray
   }
 

http://git-wip-us.apache.org/repos/asf/spark/blob/0fd68cb7/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/continuous/shuffle/UnsafeRowReceiver.scala
--
diff --git 

svn commit: r27096 - in /dev/spark/2.4.0-SNAPSHOT-2018_05_24_16_01-53c06dd-docs: ./ _site/ _site/api/ _site/api/R/ _site/api/java/ _site/api/java/lib/ _site/api/java/org/ _site/api/java/org/apache/ _s

2018-05-24 Thread pwendell
Author: pwendell
Date: Thu May 24 23:15:15 2018
New Revision: 27096

Log:
Apache Spark 2.4.0-SNAPSHOT-2018_05_24_16_01-53c06dd docs


[This commit notification would consist of 1464 parts, 
which exceeds the limit of 50 ones, so it was shortened to the summary.]

-
To unsubscribe, e-mail: commits-unsubscr...@spark.apache.org
For additional commands, e-mail: commits-h...@spark.apache.org



spark git commit: [SPARK-24332][SS][MESOS] Fix places reading 'spark.network.timeout' as milliseconds

2018-05-24 Thread zsxwing
Repository: spark
Updated Branches:
  refs/heads/master 0d8994344 -> 53c06ddab


[SPARK-24332][SS][MESOS] Fix places reading 'spark.network.timeout' as 
milliseconds

## What changes were proposed in this pull request?

This PR replaces `getTimeAsMs` with `getTimeAsSeconds` to fix the issue that 
reading "spark.network.timeout" using a wrong time unit when the user doesn't 
specify a time out.

## How was this patch tested?

Jenkins

Author: Shixiong Zhu 

Closes #21382 from zsxwing/fix-network-timeout-conf.


Project: http://git-wip-us.apache.org/repos/asf/spark/repo
Commit: http://git-wip-us.apache.org/repos/asf/spark/commit/53c06dda
Tree: http://git-wip-us.apache.org/repos/asf/spark/tree/53c06dda
Diff: http://git-wip-us.apache.org/repos/asf/spark/diff/53c06dda

Branch: refs/heads/master
Commit: 53c06ddabbdf689f8823807445849ad63173676f
Parents: 0d89943
Author: Shixiong Zhu 
Authored: Thu May 24 13:00:24 2018 -0700
Committer: Shixiong Zhu 
Committed: Thu May 24 13:00:24 2018 -0700

--
 .../org/apache/spark/sql/kafka010/KafkaMicroBatchReader.scala| 2 +-
 .../main/scala/org/apache/spark/sql/kafka010/KafkaRelation.scala | 4 +++-
 .../main/scala/org/apache/spark/sql/kafka010/KafkaSource.scala   | 2 +-
 .../scala/org/apache/spark/streaming/kafka010/KafkaRDD.scala | 2 +-
 .../cluster/mesos/MesosCoarseGrainedSchedulerBackend.scala   | 2 +-
 5 files changed, 7 insertions(+), 5 deletions(-)
--


http://git-wip-us.apache.org/repos/asf/spark/blob/53c06dda/external/kafka-0-10-sql/src/main/scala/org/apache/spark/sql/kafka010/KafkaMicroBatchReader.scala
--
diff --git 
a/external/kafka-0-10-sql/src/main/scala/org/apache/spark/sql/kafka010/KafkaMicroBatchReader.scala
 
b/external/kafka-0-10-sql/src/main/scala/org/apache/spark/sql/kafka010/KafkaMicroBatchReader.scala
index 64ba987..737da2e 100644
--- 
a/external/kafka-0-10-sql/src/main/scala/org/apache/spark/sql/kafka010/KafkaMicroBatchReader.scala
+++ 
b/external/kafka-0-10-sql/src/main/scala/org/apache/spark/sql/kafka010/KafkaMicroBatchReader.scala
@@ -68,7 +68,7 @@ private[kafka010] class KafkaMicroBatchReader(
 
   private val pollTimeoutMs = options.getLong(
 "kafkaConsumer.pollTimeoutMs",
-SparkEnv.get.conf.getTimeAsMs("spark.network.timeout", "120s"))
+SparkEnv.get.conf.getTimeAsSeconds("spark.network.timeout", "120s") * 
1000L)
 
   private val maxOffsetsPerTrigger =
 Option(options.get("maxOffsetsPerTrigger").orElse(null)).map(_.toLong)

http://git-wip-us.apache.org/repos/asf/spark/blob/53c06dda/external/kafka-0-10-sql/src/main/scala/org/apache/spark/sql/kafka010/KafkaRelation.scala
--
diff --git 
a/external/kafka-0-10-sql/src/main/scala/org/apache/spark/sql/kafka010/KafkaRelation.scala
 
b/external/kafka-0-10-sql/src/main/scala/org/apache/spark/sql/kafka010/KafkaRelation.scala
index 7103709..c31e6ed 100644
--- 
a/external/kafka-0-10-sql/src/main/scala/org/apache/spark/sql/kafka010/KafkaRelation.scala
+++ 
b/external/kafka-0-10-sql/src/main/scala/org/apache/spark/sql/kafka010/KafkaRelation.scala
@@ -48,7 +48,9 @@ private[kafka010] class KafkaRelation(
 
   private val pollTimeoutMs = sourceOptions.getOrElse(
 "kafkaConsumer.pollTimeoutMs",
-sqlContext.sparkContext.conf.getTimeAsMs("spark.network.timeout", 
"120s").toString
+(sqlContext.sparkContext.conf.getTimeAsSeconds(
+  "spark.network.timeout",
+  "120s") * 1000L).toString
   ).toLong
 
   override def schema: StructType = KafkaOffsetReader.kafkaSchema

http://git-wip-us.apache.org/repos/asf/spark/blob/53c06dda/external/kafka-0-10-sql/src/main/scala/org/apache/spark/sql/kafka010/KafkaSource.scala
--
diff --git 
a/external/kafka-0-10-sql/src/main/scala/org/apache/spark/sql/kafka010/KafkaSource.scala
 
b/external/kafka-0-10-sql/src/main/scala/org/apache/spark/sql/kafka010/KafkaSource.scala
index 1c7b3a2..101e649 100644
--- 
a/external/kafka-0-10-sql/src/main/scala/org/apache/spark/sql/kafka010/KafkaSource.scala
+++ 
b/external/kafka-0-10-sql/src/main/scala/org/apache/spark/sql/kafka010/KafkaSource.scala
@@ -84,7 +84,7 @@ private[kafka010] class KafkaSource(
 
   private val pollTimeoutMs = sourceOptions.getOrElse(
 "kafkaConsumer.pollTimeoutMs",
-sc.conf.getTimeAsMs("spark.network.timeout", "120s").toString
+(sc.conf.getTimeAsSeconds("spark.network.timeout", "120s") * 
1000L).toString
   ).toLong
 
   private val maxOffsetsPerTrigger =

http://git-wip-us.apache.org/repos/asf/spark/blob/53c06dda/external/kafka-0-10/src/main/scala/org/apache/spark/streaming/kafka010/KafkaRDD.scala
--
diff --git 

svn commit: r27092 - in /dev/spark/2.4.0-SNAPSHOT-2018_05_24_12_01-0d89943-docs: ./ _site/ _site/api/ _site/api/R/ _site/api/java/ _site/api/java/lib/ _site/api/java/org/ _site/api/java/org/apache/ _s

2018-05-24 Thread pwendell
Author: pwendell
Date: Thu May 24 19:16:12 2018
New Revision: 27092

Log:
Apache Spark 2.4.0-SNAPSHOT-2018_05_24_12_01-0d89943 docs


[This commit notification would consist of 1464 parts, 
which exceeds the limit of 50 ones, so it was shortened to the summary.]

-
To unsubscribe, e-mail: commits-unsubscr...@spark.apache.org
For additional commands, e-mail: commits-h...@spark.apache.org



svn commit: r27089 - in /dev/spark/2.3.2-SNAPSHOT-2018_05_24_10_01-d0f30e3-docs: ./ _site/ _site/api/ _site/api/R/ _site/api/java/ _site/api/java/lib/ _site/api/java/org/ _site/api/java/org/apache/ _s

2018-05-24 Thread pwendell
Author: pwendell
Date: Thu May 24 17:15:52 2018
New Revision: 27089

Log:
Apache Spark 2.3.2-SNAPSHOT-2018_05_24_10_01-d0f30e3 docs


[This commit notification would consist of 1443 parts, 
which exceeds the limit of 50 ones, so it was shortened to the summary.]

-
To unsubscribe, e-mail: commits-unsubscr...@spark.apache.org
For additional commands, e-mail: commits-h...@spark.apache.org



spark git commit: [SPARK-24378][SQL] Fix date_trunc function incorrect examples

2018-05-24 Thread gurwls223
Repository: spark
Updated Branches:
  refs/heads/branch-2.3 f48d62400 -> d0f30e3f3


[SPARK-24378][SQL] Fix date_trunc function incorrect examples

## What changes were proposed in this pull request?

Fix `date_trunc` function incorrect examples.

## How was this patch tested?

N/A

Author: Yuming Wang 

Closes #21423 from wangyum/SPARK-24378.


Project: http://git-wip-us.apache.org/repos/asf/spark/repo
Commit: http://git-wip-us.apache.org/repos/asf/spark/commit/d0f30e3f
Tree: http://git-wip-us.apache.org/repos/asf/spark/tree/d0f30e3f
Diff: http://git-wip-us.apache.org/repos/asf/spark/diff/d0f30e3f

Branch: refs/heads/branch-2.3
Commit: d0f30e3f36f50abfc18654e379fd03c1360c4fd6
Parents: f48d624
Author: Yuming Wang 
Authored: Thu May 24 23:38:50 2018 +0800
Committer: hyukjinkwon 
Committed: Thu May 24 23:42:20 2018 +0800

--
 .../catalyst/expressions/datetimeExpressions.scala  | 16 
 1 file changed, 8 insertions(+), 8 deletions(-)
--


http://git-wip-us.apache.org/repos/asf/spark/blob/d0f30e3f/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/datetimeExpressions.scala
--
diff --git 
a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/datetimeExpressions.scala
 
b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/datetimeExpressions.scala
index 424871f..7859cd8 100644
--- 
a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/datetimeExpressions.scala
+++ 
b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/datetimeExpressions.scala
@@ -1434,14 +1434,14 @@ case class TruncDate(date: Expression, format: 
Expression)
   """,
   examples = """
 Examples:
-  > SELECT _FUNC_('2015-03-05T09:32:05.359', 'YEAR');
-   2015-01-01T00:00:00
-  > SELECT _FUNC_('2015-03-05T09:32:05.359', 'MM');
-   2015-03-01T00:00:00
-  > SELECT _FUNC_('2015-03-05T09:32:05.359', 'DD');
-   2015-03-05T00:00:00
-  > SELECT _FUNC_('2015-03-05T09:32:05.359', 'HOUR');
-   2015-03-05T09:00:00
+  > SELECT _FUNC_('YEAR', '2015-03-05T09:32:05.359');
+   2015-01-01 00:00:00
+  > SELECT _FUNC_('MM', '2015-03-05T09:32:05.359');
+   2015-03-01 00:00:00
+  > SELECT _FUNC_('DD', '2015-03-05T09:32:05.359');
+   2015-03-05 00:00:00
+  > SELECT _FUNC_('HOUR', '2015-03-05T09:32:05.359');
+   2015-03-05 09:00:00
   """,
   since = "2.3.0")
 // scalastyle:on line.size.limit


-
To unsubscribe, e-mail: commits-unsubscr...@spark.apache.org
For additional commands, e-mail: commits-h...@spark.apache.org



spark git commit: [SPARK-24378][SQL] Fix date_trunc function incorrect examples

2018-05-24 Thread gurwls223
Repository: spark
Updated Branches:
  refs/heads/master 13bedc05c -> 0d8994344


[SPARK-24378][SQL] Fix date_trunc function incorrect examples

## What changes were proposed in this pull request?

Fix `date_trunc` function incorrect examples.

## How was this patch tested?

N/A

Author: Yuming Wang 

Closes #21423 from wangyum/SPARK-24378.


Project: http://git-wip-us.apache.org/repos/asf/spark/repo
Commit: http://git-wip-us.apache.org/repos/asf/spark/commit/0d899434
Tree: http://git-wip-us.apache.org/repos/asf/spark/tree/0d899434
Diff: http://git-wip-us.apache.org/repos/asf/spark/diff/0d899434

Branch: refs/heads/master
Commit: 0d89943449764e8a578edf4ceb6245158421eb96
Parents: 13bedc0
Author: Yuming Wang 
Authored: Thu May 24 23:38:50 2018 +0800
Committer: hyukjinkwon 
Committed: Thu May 24 23:38:50 2018 +0800

--
 .../catalyst/expressions/datetimeExpressions.scala  | 16 
 1 file changed, 8 insertions(+), 8 deletions(-)
--


http://git-wip-us.apache.org/repos/asf/spark/blob/0d899434/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/datetimeExpressions.scala
--
diff --git 
a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/datetimeExpressions.scala
 
b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/datetimeExpressions.scala
index e8d85f7..08838d2 100644
--- 
a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/datetimeExpressions.scala
+++ 
b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/datetimeExpressions.scala
@@ -1533,14 +1533,14 @@ case class TruncDate(date: Expression, format: 
Expression)
   """,
   examples = """
 Examples:
-  > SELECT _FUNC_('2015-03-05T09:32:05.359', 'YEAR');
-   2015-01-01T00:00:00
-  > SELECT _FUNC_('2015-03-05T09:32:05.359', 'MM');
-   2015-03-01T00:00:00
-  > SELECT _FUNC_('2015-03-05T09:32:05.359', 'DD');
-   2015-03-05T00:00:00
-  > SELECT _FUNC_('2015-03-05T09:32:05.359', 'HOUR');
-   2015-03-05T09:00:00
+  > SELECT _FUNC_('YEAR', '2015-03-05T09:32:05.359');
+   2015-01-01 00:00:00
+  > SELECT _FUNC_('MM', '2015-03-05T09:32:05.359');
+   2015-03-01 00:00:00
+  > SELECT _FUNC_('DD', '2015-03-05T09:32:05.359');
+   2015-03-05 00:00:00
+  > SELECT _FUNC_('HOUR', '2015-03-05T09:32:05.359');
+   2015-03-05 09:00:00
   """,
   since = "2.3.0")
 // scalastyle:on line.size.limit


-
To unsubscribe, e-mail: commits-unsubscr...@spark.apache.org
For additional commands, e-mail: commits-h...@spark.apache.org



svn commit: r27087 - in /dev/spark/2.4.0-SNAPSHOT-2018_05_24_08_02-13bedc0-docs: ./ _site/ _site/api/ _site/api/R/ _site/api/java/ _site/api/java/lib/ _site/api/java/org/ _site/api/java/org/apache/ _s

2018-05-24 Thread pwendell
Author: pwendell
Date: Thu May 24 15:21:11 2018
New Revision: 27087

Log:
Apache Spark 2.4.0-SNAPSHOT-2018_05_24_08_02-13bedc0 docs


[This commit notification would consist of 1464 parts, 
which exceeds the limit of 50 ones, so it was shortened to the summary.]

-
To unsubscribe, e-mail: commits-unsubscr...@spark.apache.org
For additional commands, e-mail: commits-h...@spark.apache.org



spark git commit: [SPARK-24329][SQL] Test for skipping multi-space lines

2018-05-24 Thread gurwls223
Repository: spark
Updated Branches:
  refs/heads/master 3469f5c98 -> 13bedc05c


[SPARK-24329][SQL] Test for skipping multi-space lines

## What changes were proposed in this pull request?

The PR is a continue of https://github.com/apache/spark/pull/21380 . It checks 
cases that are handled by the code:
https://github.com/apache/spark/blob/e3de6ab30d52890eb08578e55eb4a5d2b4e7aa35/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/csv/UnivocityParser.scala#L303-L304

Basically the code skips lines with one or many whitespaces, and lines with 
comments (see 
[filterCommentAndEmpty](https://github.com/apache/spark/blob/e3de6ab30d52890eb08578e55eb4a5d2b4e7aa35/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/csv/CSVUtils.scala#L47))

```scala
   iter.filter { line =>
  line.trim.nonEmpty && !line.startsWith(options.comment.toString)
}
```

Closes #21380

## How was this patch tested?

Added a test for the case described above.

Author: Maxim Gekk 
Author: Maxim Gekk 

Closes #21394 from MaxGekk/test-for-multi-space-lines.


Project: http://git-wip-us.apache.org/repos/asf/spark/repo
Commit: http://git-wip-us.apache.org/repos/asf/spark/commit/13bedc05
Tree: http://git-wip-us.apache.org/repos/asf/spark/tree/13bedc05
Diff: http://git-wip-us.apache.org/repos/asf/spark/diff/13bedc05

Branch: refs/heads/master
Commit: 13bedc05c28fcc6e739fb472bd2ee3035fa11648
Parents: 3469f5c
Author: Maxim Gekk 
Authored: Thu May 24 22:18:58 2018 +0800
Committer: hyukjinkwon 
Committed: Thu May 24 22:18:58 2018 +0800

--
 .../resources/test-data/comments-whitespaces.csv |  8 
 .../sql/execution/datasources/csv/CSVSuite.scala | 15 +++
 2 files changed, 23 insertions(+)
--


http://git-wip-us.apache.org/repos/asf/spark/blob/13bedc05/sql/core/src/test/resources/test-data/comments-whitespaces.csv
--
diff --git a/sql/core/src/test/resources/test-data/comments-whitespaces.csv 
b/sql/core/src/test/resources/test-data/comments-whitespaces.csv
new file mode 100644
index 000..2737978
--- /dev/null
+++ b/sql/core/src/test/resources/test-data/comments-whitespaces.csv
@@ -0,0 +1,8 @@
+# The file contains comments, whitespaces and empty lines
+colA
+# empty line
+
+# the line with a few whitespaces
+   
+# int value with leading and trailing whitespaces
+ "a" 

http://git-wip-us.apache.org/repos/asf/spark/blob/13bedc05/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/csv/CSVSuite.scala
--
diff --git 
a/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/csv/CSVSuite.scala
 
b/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/csv/CSVSuite.scala
index 07e6c74..2bac1a3 100644
--- 
a/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/csv/CSVSuite.scala
+++ 
b/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/csv/CSVSuite.scala
@@ -1368,4 +1368,19 @@ class CSVSuite extends QueryTest with SharedSQLContext 
with SQLTestUtils with Te
   checkAnswer(computed, expected)
 }
   }
+
+  test("SPARK-24329: skip lines with comments, and one or multiple 
whitespaces") {
+val schema = new StructType().add("colA", StringType)
+val ds = spark
+  .read
+  .schema(schema)
+  .option("multiLine", false)
+  .option("header", true)
+  .option("comment", "#")
+  .option("ignoreLeadingWhiteSpace", false)
+  .option("ignoreTrailingWhiteSpace", false)
+  .csv(testFile("test-data/comments-whitespaces.csv"))
+
+checkAnswer(ds, Seq(Row(""" "a" """)))
+  }
 }


-
To unsubscribe, e-mail: commits-unsubscr...@spark.apache.org
For additional commands, e-mail: commits-h...@spark.apache.org



spark git commit: [SPARK-24230][SQL] Fix SpecificParquetRecordReaderBase with dictionary filters.

2018-05-24 Thread wenchen
Repository: spark
Updated Branches:
  refs/heads/branch-2.3 068c4ae34 -> f48d62400


[SPARK-24230][SQL] Fix SpecificParquetRecordReaderBase with dictionary filters.

## What changes were proposed in this pull request?

I missed this commit when preparing #21070.

When Parquet is able to filter blocks with dictionary filtering, the expected 
total value count to be too high in Spark, leading to an error when there were 
fewer than expected row groups to process. Spark should get the row groups from 
Parquet to pick up new filter schemes in Parquet like dictionary filtering.

## How was this patch tested?

Using in production at Netflix. Added test case for dictionary-filtered blocks.

Author: Ryan Blue 

Closes #21295 from rdblue/SPARK-24230-fix-parquet-block-tracking.

(cherry picked from commit 3469f5c989e686866051382a3a28b2265619cab9)
Signed-off-by: Wenchen Fan 


Project: http://git-wip-us.apache.org/repos/asf/spark/repo
Commit: http://git-wip-us.apache.org/repos/asf/spark/commit/f48d6240
Tree: http://git-wip-us.apache.org/repos/asf/spark/tree/f48d6240
Diff: http://git-wip-us.apache.org/repos/asf/spark/diff/f48d6240

Branch: refs/heads/branch-2.3
Commit: f48d62400a757470c44191b9e5581c10236fe976
Parents: 068c4ae
Author: Ryan Blue 
Authored: Thu May 24 20:55:26 2018 +0800
Committer: Wenchen Fan 
Committed: Thu May 24 21:00:44 2018 +0800

--
 .../parquet/SpecificParquetRecordReaderBase.java|  6 --
 .../datasources/parquet/ParquetQuerySuite.scala | 12 
 2 files changed, 16 insertions(+), 2 deletions(-)
--


http://git-wip-us.apache.org/repos/asf/spark/blob/f48d6240/sql/core/src/main/java/org/apache/spark/sql/execution/datasources/parquet/SpecificParquetRecordReaderBase.java
--
diff --git 
a/sql/core/src/main/java/org/apache/spark/sql/execution/datasources/parquet/SpecificParquetRecordReaderBase.java
 
b/sql/core/src/main/java/org/apache/spark/sql/execution/datasources/parquet/SpecificParquetRecordReaderBase.java
index e65cd25..95fe130 100644
--- 
a/sql/core/src/main/java/org/apache/spark/sql/execution/datasources/parquet/SpecificParquetRecordReaderBase.java
+++ 
b/sql/core/src/main/java/org/apache/spark/sql/execution/datasources/parquet/SpecificParquetRecordReaderBase.java
@@ -147,7 +147,8 @@ public abstract class SpecificParquetRecordReaderBase 
extends RecordReader 
extends RecordReaderhttp://git-wip-us.apache.org/repos/asf/spark/blob/f48d6240/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetQuerySuite.scala
--
diff --git 
a/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetQuerySuite.scala
 
b/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetQuerySuite.scala
index 55b0f72..5b680bf 100644
--- 
a/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetQuerySuite.scala
+++ 
b/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetQuerySuite.scala
@@ -879,6 +879,18 @@ class ParquetQuerySuite extends QueryTest with ParquetTest 
with SharedSQLContext
   }
 }
   }
+
+  test("SPARK-24230: filter row group using dictionary") {
+withSQLConf(("parquet.filter.dictionary.enabled", "true")) {
+  // create a table with values from 0, 2, ..., 18 that will be 
dictionary-encoded
+  withParquetTable((0 until 100).map(i => ((i * 2) % 20, s"data-$i")), 
"t") {
+// search for a key that is not present so the dictionary filter 
eliminates all row groups
+// Fails without SPARK-24230:
+//   java.io.IOException: expecting more rows but reached last block. 
Read 0 out of 50
+checkAnswer(sql("SELECT _2 FROM t WHERE t._1 = 5"), Seq.empty)
+  }
+}
+  }
 }
 
 object TestingUDT {


-
To unsubscribe, e-mail: commits-unsubscr...@spark.apache.org
For additional commands, e-mail: commits-h...@spark.apache.org



spark git commit: [SPARK-24230][SQL] Fix SpecificParquetRecordReaderBase with dictionary filters.

2018-05-24 Thread wenchen
Repository: spark
Updated Branches:
  refs/heads/master 4a14dc0af -> 3469f5c98


[SPARK-24230][SQL] Fix SpecificParquetRecordReaderBase with dictionary filters.

## What changes were proposed in this pull request?

I missed this commit when preparing #21070.

When Parquet is able to filter blocks with dictionary filtering, the expected 
total value count to be too high in Spark, leading to an error when there were 
fewer than expected row groups to process. Spark should get the row groups from 
Parquet to pick up new filter schemes in Parquet like dictionary filtering.

## How was this patch tested?

Using in production at Netflix. Added test case for dictionary-filtered blocks.

Author: Ryan Blue 

Closes #21295 from rdblue/SPARK-24230-fix-parquet-block-tracking.


Project: http://git-wip-us.apache.org/repos/asf/spark/repo
Commit: http://git-wip-us.apache.org/repos/asf/spark/commit/3469f5c9
Tree: http://git-wip-us.apache.org/repos/asf/spark/tree/3469f5c9
Diff: http://git-wip-us.apache.org/repos/asf/spark/diff/3469f5c9

Branch: refs/heads/master
Commit: 3469f5c989e686866051382a3a28b2265619cab9
Parents: 4a14dc0
Author: Ryan Blue 
Authored: Thu May 24 20:55:26 2018 +0800
Committer: Wenchen Fan 
Committed: Thu May 24 20:55:26 2018 +0800

--
 .../parquet/SpecificParquetRecordReaderBase.java|  6 --
 .../datasources/parquet/ParquetQuerySuite.scala | 12 
 2 files changed, 16 insertions(+), 2 deletions(-)
--


http://git-wip-us.apache.org/repos/asf/spark/blob/3469f5c9/sql/core/src/main/java/org/apache/spark/sql/execution/datasources/parquet/SpecificParquetRecordReaderBase.java
--
diff --git 
a/sql/core/src/main/java/org/apache/spark/sql/execution/datasources/parquet/SpecificParquetRecordReaderBase.java
 
b/sql/core/src/main/java/org/apache/spark/sql/execution/datasources/parquet/SpecificParquetRecordReaderBase.java
index daedfd7..c975e52 100644
--- 
a/sql/core/src/main/java/org/apache/spark/sql/execution/datasources/parquet/SpecificParquetRecordReaderBase.java
+++ 
b/sql/core/src/main/java/org/apache/spark/sql/execution/datasources/parquet/SpecificParquetRecordReaderBase.java
@@ -146,7 +146,8 @@ public abstract class SpecificParquetRecordReaderBase 
extends RecordReader 
extends RecordReaderhttp://git-wip-us.apache.org/repos/asf/spark/blob/3469f5c9/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetQuerySuite.scala
--
diff --git 
a/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetQuerySuite.scala
 
b/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetQuerySuite.scala
index e1f094d..2b1227f 100644
--- 
a/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetQuerySuite.scala
+++ 
b/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetQuerySuite.scala
@@ -879,6 +879,18 @@ class ParquetQuerySuite extends QueryTest with ParquetTest 
with SharedSQLContext
   }
 }
   }
+
+  test("SPARK-24230: filter row group using dictionary") {
+withSQLConf(("parquet.filter.dictionary.enabled", "true")) {
+  // create a table with values from 0, 2, ..., 18 that will be 
dictionary-encoded
+  withParquetTable((0 until 100).map(i => ((i * 2) % 20, s"data-$i")), 
"t") {
+// search for a key that is not present so the dictionary filter 
eliminates all row groups
+// Fails without SPARK-24230:
+//   java.io.IOException: expecting more rows but reached last block. 
Read 0 out of 50
+checkAnswer(sql("SELECT _2 FROM t WHERE t._1 = 5"), Seq.empty)
+  }
+}
+  }
 }
 
 object TestingUDT {


-
To unsubscribe, e-mail: commits-unsubscr...@spark.apache.org
For additional commands, e-mail: commits-h...@spark.apache.org



svn commit: r27082 - in /dev/spark/2.3.2-SNAPSHOT-2018_05_24_02_01-068c4ae-docs: ./ _site/ _site/api/ _site/api/R/ _site/api/java/ _site/api/java/lib/ _site/api/java/org/ _site/api/java/org/apache/ _s

2018-05-24 Thread pwendell
Author: pwendell
Date: Thu May 24 09:17:13 2018
New Revision: 27082

Log:
Apache Spark 2.3.2-SNAPSHOT-2018_05_24_02_01-068c4ae docs


[This commit notification would consist of 1443 parts, 
which exceeds the limit of 50 ones, so it was shortened to the summary.]

-
To unsubscribe, e-mail: commits-unsubscr...@spark.apache.org
For additional commands, e-mail: commits-h...@spark.apache.org



svn commit: r27076 - in /dev/spark/2.4.0-SNAPSHOT-2018_05_24_00_01-4a14dc0-docs: ./ _site/ _site/api/ _site/api/R/ _site/api/java/ _site/api/java/lib/ _site/api/java/org/ _site/api/java/org/apache/ _s

2018-05-24 Thread pwendell
Author: pwendell
Date: Thu May 24 07:18:05 2018
New Revision: 27076

Log:
Apache Spark 2.4.0-SNAPSHOT-2018_05_24_00_01-4a14dc0 docs


[This commit notification would consist of 1464 parts, 
which exceeds the limit of 50 ones, so it was shortened to the summary.]

-
To unsubscribe, e-mail: commits-unsubscr...@spark.apache.org
For additional commands, e-mail: commits-h...@spark.apache.org



spark git commit: fix compilation caused by SPARK-24257

2018-05-24 Thread wenchen
Repository: spark
Updated Branches:
  refs/heads/branch-2.0 5fd080902 -> dccd8c754


fix compilation caused by SPARK-24257


Project: http://git-wip-us.apache.org/repos/asf/spark/repo
Commit: http://git-wip-us.apache.org/repos/asf/spark/commit/dccd8c75
Tree: http://git-wip-us.apache.org/repos/asf/spark/tree/dccd8c75
Diff: http://git-wip-us.apache.org/repos/asf/spark/diff/dccd8c75

Branch: refs/heads/branch-2.0
Commit: dccd8c7549390a6f46272182052dcf17e5e51319
Parents: 5fd0809
Author: Wenchen Fan 
Authored: Thu May 24 12:44:26 2018 +0800
Committer: Wenchen Fan 
Committed: Thu May 24 12:46:33 2018 +0800

--
 .../org/apache/spark/sql/execution/joins/HashedRelationSuite.scala | 2 +-
 1 file changed, 1 insertion(+), 1 deletion(-)
--


http://git-wip-us.apache.org/repos/asf/spark/blob/dccd8c75/sql/core/src/test/scala/org/apache/spark/sql/execution/joins/HashedRelationSuite.scala
--
diff --git 
a/sql/core/src/test/scala/org/apache/spark/sql/execution/joins/HashedRelationSuite.scala
 
b/sql/core/src/test/scala/org/apache/spark/sql/execution/joins/HashedRelationSuite.scala
index b575e55..f0288c8 100644
--- 
a/sql/core/src/test/scala/org/apache/spark/sql/execution/joins/HashedRelationSuite.scala
+++ 
b/sql/core/src/test/scala/org/apache/spark/sql/execution/joins/HashedRelationSuite.scala
@@ -256,7 +256,7 @@ class HashedRelationSuite extends SparkFunSuite with 
SharedSQLContext {
   test("SPARK-24257: insert big values into LongToUnsafeRowMap") {
 val taskMemoryManager = new TaskMemoryManager(
   new StaticMemoryManager(
-new SparkConf().set(MEMORY_OFFHEAP_ENABLED.key, "false"),
+new SparkConf().set("spark.memory.offHeap.enabled", "false"),
 Long.MaxValue,
 Long.MaxValue,
 1),


-
To unsubscribe, e-mail: commits-unsubscr...@spark.apache.org
For additional commands, e-mail: commits-h...@spark.apache.org



spark git commit: [SPARK-22269][BUILD] Run Java linter via SBT for Jenkins

2018-05-24 Thread gurwls223
Repository: spark
Updated Branches:
  refs/heads/master 8a545822d -> 4a14dc0af


[SPARK-22269][BUILD] Run Java linter via SBT for Jenkins

## What changes were proposed in this pull request?

This PR proposes to check Java lint via SBT for Jenkins. It uses the SBT 
wrapper for checkstyle.

I manually tested. If we build the codes once, running this script takes 2 mins 
at maximum in my local:

Test codes:

```
Checkstyle failed at following occurrences:
[error] Checkstyle error found in 
/.../spark/core/src/test/java/test/org/apache/spark/JavaAPISuite.java:82: Line 
is longer than 100 characters (found 103).
[error] 1 issue(s) found in Checkstyle report: 
/.../spark/core/target/checkstyle-test-report.xml
[error] Checkstyle error found in 
/.../spark/sql/hive/src/test/java/org/apache/spark/sql/hive/JavaDataFrameSuite.java:84:
 Line is longer than 100 characters (found 115).
[error] 1 issue(s) found in Checkstyle report: 
/.../spark/sql/hive/target/checkstyle-test-report.xml
...
```

Main codes:

```
Checkstyle failed at following occurrences:
[error] Checkstyle error found in 
/.../spark/sql/core/src/main/java/org/apache/spark/sql/sources/v2/reader/InputPartition.java:39:
 Line is longer than 100 characters (found 104).
[error] Checkstyle error found in 
/.../spark/sql/core/src/main/java/org/apache/spark/sql/sources/v2/reader/InputPartitionReader.java:26:
 Line is longer than 100 characters (found 110).
[error] Checkstyle error found in 
/.../spark/sql/core/src/main/java/org/apache/spark/sql/sources/v2/reader/InputPartitionReader.java:30:
 Line is longer than 100 characters (found 104).
...
```

## How was this patch tested?

Manually tested. Jenkins build should test this.

Author: hyukjinkwon 

Closes #21399 from HyukjinKwon/SPARK-22269.


Project: http://git-wip-us.apache.org/repos/asf/spark/repo
Commit: http://git-wip-us.apache.org/repos/asf/spark/commit/4a14dc0a
Tree: http://git-wip-us.apache.org/repos/asf/spark/tree/4a14dc0a
Diff: http://git-wip-us.apache.org/repos/asf/spark/diff/4a14dc0a

Branch: refs/heads/master
Commit: 4a14dc0aff9cac85390cab94bc183271fa95beef
Parents: 8a54582
Author: hyukjinkwon 
Authored: Thu May 24 14:19:32 2018 +0800
Committer: hyukjinkwon 
Committed: Thu May 24 14:19:32 2018 +0800

--
 dev/run-tests.py |  5 ++---
 dev/sbt-checkstyle   | 42 ++
 project/SparkBuild.scala | 14 +-
 project/plugins.sbt  |  8 
 4 files changed, 65 insertions(+), 4 deletions(-)
--


http://git-wip-us.apache.org/repos/asf/spark/blob/4a14dc0a/dev/run-tests.py
--
diff --git a/dev/run-tests.py b/dev/run-tests.py
index 164c1e2..5e8c859 100755
--- a/dev/run-tests.py
+++ b/dev/run-tests.py
@@ -204,7 +204,7 @@ def run_scala_style_checks():
 
 def run_java_style_checks():
 set_title_and_block("Running Java style checks", "BLOCK_JAVA_STYLE")
-run_cmd([os.path.join(SPARK_HOME, "dev", "lint-java")])
+run_cmd([os.path.join(SPARK_HOME, "dev", "sbt-checkstyle")])
 
 
 def run_python_style_checks():
@@ -574,8 +574,7 @@ def main():
 or f.endswith("checkstyle.xml")
 or f.endswith("checkstyle-suppressions.xml")
 for f in changed_files):
-# run_java_style_checks()
-pass
+run_java_style_checks()
 if not changed_files or any(f.endswith("lint-python")
 or f.endswith("tox.ini")
 or f.endswith(".py")

http://git-wip-us.apache.org/repos/asf/spark/blob/4a14dc0a/dev/sbt-checkstyle
--
diff --git a/dev/sbt-checkstyle b/dev/sbt-checkstyle
new file mode 100755
index 000..8821a7c
--- /dev/null
+++ b/dev/sbt-checkstyle
@@ -0,0 +1,42 @@
+#!/usr/bin/env bash
+
+#
+# Licensed to the Apache Software Foundation (ASF) under one or more
+# contributor license agreements.  See the NOTICE file distributed with
+# this work for additional information regarding copyright ownership.
+# The ASF licenses this file to You under the Apache License, Version 2.0
+# (the "License"); you may not use this file except in compliance with
+# the License.  You may obtain a copy of the License at
+#
+#http://www.apache.org/licenses/LICENSE-2.0
+#
+# Unless required by applicable law or agreed to in writing, software
+# distributed under the License is distributed on an "AS IS" BASIS,
+# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+# See the License for the specific language governing permissions and
+# limitations under the License.
+#
+
+# NOTE: echo "q" is needed because SBT prompts the user for input on 
encountering