Commit graph

20823 commits

Author SHA1 Message Date
Liang-Chi Hsieh fccb337f9d [SPARK-22538][ML] SQLTransformer should not unpersist possibly cached input dataset
## What changes were proposed in this pull request?

`SQLTransformer.transform` unpersists input dataset when dropping temporary view. We should not change input dataset's cache status.

## How was this patch tested?

Added test.

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

Closes #19772 from viirya/SPARK-22538.
2017-11-17 17:43:40 +01:00
Li Jin 7d039e0c0a [SPARK-22409] Introduce function type argument in pandas_udf
## What changes were proposed in this pull request?

* Add a "function type" argument to pandas_udf.
* Add a new public enum class `PandasUdfType` in pyspark.sql.functions
* Refactor udf related code from pyspark.sql.functions to pyspark.sql.udf
* Merge "PythonUdfType" and "PythonEvalType" into a single enum class "PythonEvalType"

Example:
```
from pyspark.sql.functions import pandas_udf, PandasUDFType

pandas_udf('double', PandasUDFType.SCALAR):
def plus_one(v):
    return v + 1
```

## Design doc
https://docs.google.com/document/d/1KlLaa-xJ3oz28xlEJqXyCAHU3dwFYkFs_ixcUXrJNTc/edit

## How was this patch tested?

Added PandasUDFTests

## TODO:
* [x] Implement proper enum type for `PandasUDFType`
* [x] Update documentation
* [x] Add more tests in PandasUDFTests

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

Closes #19630 from icexelloss/spark-22409-pandas-udf-type.
2017-11-17 16:43:08 +01:00
yucai d00b55d4b2 [SPARK-22540][SQL] Ensure HighlyCompressedMapStatus calculates correct avgSize
## What changes were proposed in this pull request?

Ensure HighlyCompressedMapStatus calculates correct avgSize

## How was this patch tested?

New unit test added.

Author: yucai <yucai.yu@intel.com>

Closes #19765 from yucai/avgsize.
2017-11-17 07:53:53 -06:00
Wenchen Fan b9dcbe5e1b [SPARK-22542][SQL] remove unused features in ColumnarBatch
## What changes were proposed in this pull request?

`ColumnarBatch` provides features to do fast filter and project in a columnar fashion, however this feature is never used by Spark, as Spark uses whole stage codegen and processes the data in a row fashion. This PR proposes to remove these unused features as we won't switch to columnar execution in the near future. Even we do, I think this part needs a proper redesign.

This is also a step to make `ColumnVector` public, as we don't wanna expose these features to users.

## How was this patch tested?

existing tests

Author: Wenchen Fan <wenchen@databricks.com>

Closes #19766 from cloud-fan/vector.
2017-11-16 18:23:00 -08:00
Kazuaki Ishizaki 7f2e62ee6b [SPARK-22501][SQL] Fix 64KB JVM bytecode limit problem with in
## What changes were proposed in this pull request?

This PR changes `In` code generation to place generated code for expression for expressions for arguments into separated methods if these size could be large.

## How was this patch tested?

Added new test cases into `PredicateSuite`

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

Closes #19733 from kiszk/SPARK-22501.
2017-11-16 18:24:49 +01:00
Marco Gaido 4e7f07e255 [SPARK-22494][SQL] Fix 64KB limit exception with Coalesce and AtleastNNonNulls
## What changes were proposed in this pull request?

Both `Coalesce` and `AtLeastNNonNulls` can cause the 64KB limit exception when used with a lot of arguments and/or complex expressions.
This PR splits their expressions in order to avoid the issue.

## How was this patch tested?

Added UTs

Author: Marco Gaido <marcogaido91@gmail.com>
Author: Marco Gaido <mgaido@hortonworks.com>

Closes #19720 from mgaido91/SPARK-22494.
2017-11-16 18:19:13 +01:00
Kazuaki Ishizaki ed885e7a65 [SPARK-22499][SQL] Fix 64KB JVM bytecode limit problem with least and greatest
## What changes were proposed in this pull request?

This PR changes `least` and `greatest` code generation to place generated code for expression for arguments into separated methods if these size could be large.
This PR resolved two cases:

* `least` with a lot of argument
* `greatest` with a lot of argument

## How was this patch tested?

Added a new test case into `ArithmeticExpressionsSuite`

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

Closes #19729 from kiszk/SPARK-22499.
2017-11-16 17:56:21 +01:00
Shixiong Zhu 03f2b7bff7 [SPARK-22535][PYSPARK] Sleep before killing the python worker in PythonRunner.MonitorThread
## What changes were proposed in this pull request?

`PythonRunner.MonitorThread` should give the task a little time to finish before forcibly killing the python worker. This will reduce the chance of the race condition a lot. I also improved the log a bit to find out the task to blame when it's stuck.

## How was this patch tested?

Jenkins

Author: Shixiong Zhu <zsxwing@gmail.com>

Closes #19762 from zsxwing/SPARK-22535.
2017-11-16 14:22:25 +09:00
ArtRand 1e82335413 [SPARK-21842][MESOS] Support Kerberos ticket renewal and creation in Mesos
## What changes were proposed in this pull request?
tl;dr: Add a class, `MesosHadoopDelegationTokenManager` that updates delegation tokens on a schedule on the behalf of Spark Drivers. Broadcast renewed credentials to the executors.

## The problem
We recently added Kerberos support to Mesos-based Spark jobs as well as Secrets support to the Mesos Dispatcher (SPARK-16742, SPARK-20812, respectively). However the delegation tokens have a defined expiration. This poses a problem for long running Spark jobs (e.g. Spark Streaming applications). YARN has a solution for this where a thread is scheduled to renew the tokens they reach 75% of their way to expiration. It then writes the tokens to HDFS for the executors to find (uses a monotonically increasing suffix).

## This solution
We replace the current method in `CoarseGrainedSchedulerBackend` which used to discard the token renewal time with a protected method `fetchHadoopDelegationTokens`. Now the individual cluster backends are responsible for overriding this method to fetch and manage token renewal. The delegation tokens themselves, are still part of the `CoarseGrainedSchedulerBackend` as before.
In the case of Mesos renewed Credentials are broadcasted to the executors. This maintains all transfer of Credentials within Spark (as opposed to Spark-to-HDFS). It also does not require any writing of Credentials to disk. It also does not require any GC of old files.

## How was this patch tested?
Manually against a Kerberized HDFS cluster.

Thank you for the reviews.

Author: ArtRand <arand@soe.ucsc.edu>

Closes #19272 from ArtRand/spark-21842-450-kerberos-ticket-renewal.
2017-11-15 15:53:05 -08:00
osatici 2014e7a789 [SPARK-22479][SQL] Exclude credentials from SaveintoDataSourceCommand.simpleString
## What changes were proposed in this pull request?

Do not include jdbc properties which may contain credentials in logging a logical plan with `SaveIntoDataSourceCommand` in it.

## How was this patch tested?

building locally and trying to reproduce (per the steps in https://issues.apache.org/jira/browse/SPARK-22479):
```
== Parsed Logical Plan ==
SaveIntoDataSourceCommand org.apache.spark.sql.execution.datasources.jdbc.JdbcRelationProvider570127fa, Map(dbtable -> test20, driver -> org.postgresql.Driver, url -> *********(redacted), password -> *********(redacted)), ErrorIfExists
   +- Range (0, 100, step=1, splits=Some(8))

== Analyzed Logical Plan ==
SaveIntoDataSourceCommand org.apache.spark.sql.execution.datasources.jdbc.JdbcRelationProvider570127fa, Map(dbtable -> test20, driver -> org.postgresql.Driver, url -> *********(redacted), password -> *********(redacted)), ErrorIfExists
   +- Range (0, 100, step=1, splits=Some(8))

== Optimized Logical Plan ==
SaveIntoDataSourceCommand org.apache.spark.sql.execution.datasources.jdbc.JdbcRelationProvider570127fa, Map(dbtable -> test20, driver -> org.postgresql.Driver, url -> *********(redacted), password -> *********(redacted)), ErrorIfExists
   +- Range (0, 100, step=1, splits=Some(8))

== Physical Plan ==
Execute SaveIntoDataSourceCommand
   +- SaveIntoDataSourceCommand org.apache.spark.sql.execution.datasources.jdbc.JdbcRelationProvider570127fa, Map(dbtable -> test20, driver -> org.postgresql.Driver, url -> *********(redacted), password -> *********(redacted)), ErrorIfExists
         +- Range (0, 100, step=1, splits=Some(8))
```

Author: osatici <osatici@palantir.com>

Closes #19708 from onursatici/os/redact-jdbc-creds.
2017-11-15 14:08:51 -08:00
Marcelo Vanzin 39b3f10dda [SPARK-20649][CORE] Simplify REST API resource structure.
With the new UI store, the API resource classes have a lot less code,
since there's no need for complicated translations between the UI
types and the API types. So the code ended up with a bunch of files
with a single method declared in them.

This change re-structures the API code so that it uses less classes;
mainly, most sub-resources were removed, and the code to deal with
single-attempt and multi-attempt apps was simplified.

The only change was the addition of a method to return a single
attempt's information; that was missing in the old API, so trying
to retrieve "/v1/applications/appId/attemptId" would result in a
404 even if the attempt existed (and URIs under that one would
return valid data).

The streaming API resources also overtook the same treatment, even
though the data is not stored in the new UI store.

Author: Marcelo Vanzin <vanzin@cloudera.com>

Closes #19748 from vanzin/SPARK-20649.
2017-11-15 15:41:53 -06:00
liutang123 bc0848b4c1 [SPARK-22469][SQL] Accuracy problem in comparison with string and numeric
## What changes were proposed in this pull request?
This fixes a problem caused by #15880
`select '1.5' > 0.5; // Result is NULL in Spark but is true in Hive.
`
When compare string and numeric, cast them as double like Hive.

Author: liutang123 <liutang123@yeah.net>

Closes #19692 from liutang123/SPARK-22469.
2017-11-15 09:02:54 -08:00
Dongjoon Hyun aa88b8dbbb [SPARK-22490][DOC] Add PySpark doc for SparkSession.builder
## What changes were proposed in this pull request?

In PySpark API Document, [SparkSession.build](http://spark.apache.org/docs/2.2.0/api/python/pyspark.sql.html) is not documented and shows default value description.
```
SparkSession.builder = <pyspark.sql.session.Builder object ...
```

This PR adds the doc.

![screen](https://user-images.githubusercontent.com/9700541/32705514-1bdcafaa-c7ca-11e7-88bf-05566fea42de.png)

The following is the diff of the generated result.

```
$ diff old.html new.html
95a96,101
> <dl class="attribute">
> <dt id="pyspark.sql.SparkSession.builder">
> <code class="descname">builder</code><a class="headerlink" href="#pyspark.sql.SparkSession.builder" title="Permalink to this definition">¶</a></dt>
> <dd><p>A class attribute having a <a class="reference internal" href="#pyspark.sql.SparkSession.Builder" title="pyspark.sql.SparkSession.Builder"><code class="xref py py-class docutils literal"><span class="pre">Builder</span></code></a> to construct <a class="reference internal" href="#pyspark.sql.SparkSession" title="pyspark.sql.SparkSession"><code class="xref py py-class docutils literal"><span class="pre">SparkSession</span></code></a> instances</p>
> </dd></dl>
>
212,216d217
< <dt id="pyspark.sql.SparkSession.builder">
< <code class="descname">builder</code><em class="property"> = &lt;pyspark.sql.session.SparkSession.Builder object&gt;</em><a class="headerlink" href="#pyspark.sql.SparkSession.builder" title="Permalink to this definition">¶</a></dt>
< <dd></dd></dl>
<
< <dl class="attribute">
```

## How was this patch tested?

Manual.

```
cd python/docs
make html
open _build/html/pyspark.sql.html
```

Author: Dongjoon Hyun <dongjoon@apache.org>

Closes #19726 from dongjoon-hyun/SPARK-22490.
2017-11-15 08:59:29 -08:00
test 7f99a05e6f [SPARK-22422][ML] Add Adjusted R2 to RegressionMetrics
## What changes were proposed in this pull request?

I added adjusted R2 as a regression metric which was implemented in all major statistical analysis tools.

In practice, no one looks at R2 alone. The reason is R2 itself is misleading. If we add more parameters, R2 will not decrease but only increase (or stay the same). This leads to overfitting. Adjusted R2 addressed this issue by using number of parameters as "weight" for the sum of errors.

## How was this patch tested?

- Added a new unit test and passed.
- ./dev/run-tests all passed.

Author: test <joseph.peng@quetica.com>
Author: tengpeng <tengpeng@users.noreply.github.com>

Closes #19638 from tengpeng/master.
2017-11-15 10:13:01 -06:00
Bryan Cutler 8f0e88df03 [SPARK-20791][PYTHON][FOLLOWUP] Check for unicode column names in createDataFrame with Arrow
## What changes were proposed in this pull request?

If schema is passed as a list of unicode strings for column names, they should be re-encoded to 'utf-8' to be consistent.  This is similar to the #13097 but for creation of DataFrame using Arrow.

## How was this patch tested?

Added new test of using unicode names for schema.

Author: Bryan Cutler <cutlerb@gmail.com>

Closes #19738 from BryanCutler/arrow-createDataFrame-followup-unicode-SPARK-20791.
2017-11-15 23:35:13 +09:00
Wenchen Fan dce1610ae3 [SPARK-22514][SQL] move ColumnVector.Array and ColumnarBatch.Row to individual files
## What changes were proposed in this pull request?

Logically the `Array` doesn't belong to `ColumnVector`, and `Row` doesn't belong to `ColumnarBatch`. e.g. `ColumnVector` needs to return `Array` for `getArray`, and `Row` for `getStruct`. `Array` and `Row` can return each other with the `getArray`/`getStruct` methods.

This is also a step to make `ColumnVector` public, it's cleaner to have `Array` and `Row` as top-level classes.

This PR is just code moving around, with 2 renaming: `Array` -> `VectorBasedArray`, `Row` -> `VectorBasedRow`.

## How was this patch tested?

existing tests.

Author: Wenchen Fan <wenchen@databricks.com>

Closes #19740 from cloud-fan/vector.
2017-11-15 14:42:37 +01:00
WeichenXu 1e6f760593 [SPARK-12375][ML] VectorIndexerModel support handle unseen categories via handleInvalid
## What changes were proposed in this pull request?

Support skip/error/keep strategy, similar to `StringIndexer`.
Implemented via `try...catch`, so that it can avoid possible performance impact.

## How was this patch tested?

Unit test added.

Author: WeichenXu <weichen.xu@databricks.com>

Closes #19588 from WeichenXu123/handle_invalid_for_vector_indexer.
2017-11-14 16:58:18 -08:00
WeichenXu 774398045b [SPARK-21087][ML] CrossValidator, TrainValidationSplit expose sub models after fitting: Scala
## What changes were proposed in this pull request?

We add a parameter whether to collect the full model list when CrossValidator/TrainValidationSplit training (Default is NOT), avoid the change cause OOM)

- Add a method in CrossValidatorModel/TrainValidationSplitModel, allow user to get the model list

- CrossValidatorModelWriter add a “option”, allow user to control whether to persist the model list to disk (will persist by default).

- Note: when persisting the model list, use indices as the sub-model path

## How was this patch tested?

Test cases added.

Author: WeichenXu <weichen.xu@databricks.com>

Closes #19208 from WeichenXu123/expose-model-list.
2017-11-14 16:48:26 -08:00
Sean Owen b009722591 [SPARK-22511][BUILD] Update maven central repo address
## What changes were proposed in this pull request?

Use repo.maven.apache.org repo address; use latest ASF parent POM version 18

## How was this patch tested?

Existing tests; no functional change

Author: Sean Owen <sowen@cloudera.com>

Closes #19742 from srowen/SPARK-22511.
2017-11-14 17:58:07 -06:00
Devaraj K eaff295a23 [SPARK-22519][YARN] Remove unnecessary stagingDirPath null check in ApplicationMaster.cleanupStagingDir()
## What changes were proposed in this pull request?
Removed the unnecessary stagingDirPath null check in ApplicationMaster.cleanupStagingDir().

## How was this patch tested?
I verified with the existing test cases.

Author: Devaraj K <devaraj@apache.org>

Closes #19749 from devaraj-kavali/SPARK-22519.
2017-11-14 15:20:03 -08:00
Marcelo Vanzin 0ffa7c488f [SPARK-20652][SQL] Store SQL UI data in the new app status store.
This change replaces the SQLListener with a new implementation that
saves the data to the same store used by the SparkContext's status
store. For that, the types used by the old SQLListener had to be
updated a bit so that they're more serialization-friendly.

The interface for getting data from the store was abstracted into
a new class, SQLAppStatusStore (following the convention used in
core).

Another change is the way that the SQL UI hooks up into the core
UI or the SHS. The old "SparkHistoryListenerFactory" was replaced
with a new "AppStatePlugin" that more explicitly differentiates
between the two use cases: processing events, and showing the UI.
Both live apps and the SHS use this new API (previously, it was
restricted to the SHS).

Note on the above: this causes a slight change of behavior for
live apps; the SQL tab will only show up after the first execution
is started.

The metrics gathering code was re-worked a bit so that the types
used are less memory hungry and more serialization-friendly. This
reduces memory usage when using in-memory stores, and reduces load
times when using disk stores.

Tested with existing and added unit tests. Note one unit test was
disabled because it depends on SPARK-20653, which isn't in yet.

Author: Marcelo Vanzin <vanzin@cloudera.com>

Closes #19681 from vanzin/SPARK-20652.
2017-11-14 15:28:22 -06:00
Marcelo Vanzin 4741c07809 [SPARK-20648][CORE] Port JobsTab and StageTab to the new UI backend.
This change is a little larger because there's a whole lot of logic
behind these pages, all really tied to internal types and listeners,
and some of that logic had to be implemented in the new listener and
the needed data exposed through the API types.

- Added missing StageData and ExecutorStageSummary fields which are
  used by the UI. Some json golden files needed to be updated to account
  for new fields.

- Save RDD graph data in the store. This tries to re-use existing types as
  much as possible, so that the code doesn't need to be re-written. So it's
  probably not very optimal.

- Some old classes (e.g. JobProgressListener) still remain, since they're used
  in other parts of the code; they're not used by the UI anymore, though, and
  will be cleaned up in a separate change.

- Save information about active pools in the store. This data is not really used
  in the SHS, but it's not a lot of data so it's still recorded when replaying
  applications.

- Because the new store sorts things slightly differently from the previous
  code, some json golden files had some elements within them shuffled around.

- The retention unit test in UISeleniumSuite was disabled because the code
  to throw away old stages / tasks hasn't been added yet.

- The job description field in the API tries to follow the old behavior, which
  makes it be empty most of the time, even though there's information to fill it
  in. For stages, a new field was added to hold the description (which is basically
  the job description), so that the UI can be rendered in the old way.

- A new stage status ("SKIPPED") was added to account for the fact that the API
  couldn't represent that state before. Without this, the stage would show up as
  "PENDING" in the UI, which is now based on API types.

- The API used to expose "executorRunTime" as the value of the task's duration,
  which wasn't really correct (also because that value was easily available
  from the metrics object); this change fixes that by storing the correct duration,
  which also means a few expectation files needed to be updated to account for
  the new durations and sorting differences due to the changed values.

- Added changes to implement SPARK-20713 and SPARK-21922 in the new code.

Tested with existing unit tests (and by using the UI a lot).

Author: Marcelo Vanzin <vanzin@cloudera.com>

Closes #19698 from vanzin/SPARK-20648.
2017-11-14 10:34:32 -06:00
Zhenhua Wang 11b60af737 [SPARK-17074][SQL] Generate equi-height histogram in column statistics
## What changes were proposed in this pull request?

Equi-height histogram is effective in cardinality estimation, and more accurate than basic column stats (min, max, ndv, etc) especially in skew distribution. So we need to support it.

For equi-height histogram, all buckets (intervals) have the same height (frequency).
In this PR, we use a two-step method to generate an equi-height histogram:
1. use `ApproximatePercentile` to get percentiles `p(0), p(1/n), p(2/n) ... p((n-1)/n), p(1)`;
2. construct range values of buckets, e.g. `[p(0), p(1/n)], [p(1/n), p(2/n)] ... [p((n-1)/n), p(1)]`, and use `ApproxCountDistinctForIntervals` to count ndv in each bucket. Each bucket is of the form: `(lowerBound, higherBound, ndv)`.

## How was this patch tested?

Added new test cases and modified some existing test cases.

Author: Zhenhua Wang <wangzhenhua@huawei.com>
Author: Zhenhua Wang <wzh_zju@163.com>

Closes #19479 from wzhfy/generate_histogram.
2017-11-14 16:41:43 +01:00
hyukjinkwon 673c670465 [SPARK-17310][SQL] Add an option to disable record-level filter in Parquet-side
## What changes were proposed in this pull request?

There is a concern that Spark-side codegen row-by-row filtering might be faster than Parquet's one in general due to type-boxing and additional fuction calls which Spark's one tries to avoid.

So, this PR adds an option to disable/enable record-by-record filtering in Parquet side.

It sets the default to `false` to take the advantage of the improvement.

This was also discussed in https://github.com/apache/spark/pull/14671.
## How was this patch tested?

Manually benchmarks were performed. I generated a billion (1,000,000,000) records and tested equality comparison concatenated with `OR`. This filter combinations were made from 5 to 30.

It seem indeed Spark-filtering is faster in the test case and the gap increased as the filter tree becomes larger.

The details are as below:

**Code**

``` scala
test("Parquet-side filter vs Spark-side filter - record by record") {
  withTempPath { path =>
    val N = 1000 * 1000 * 1000
    val df = spark.range(N).toDF("a")
    df.write.parquet(path.getAbsolutePath)

    val benchmark = new Benchmark("Parquet-side vs Spark-side", N)
    Seq(5, 10, 20, 30).foreach { num =>
      val filterExpr = (0 to num).map(i => s"a = $i").mkString(" OR ")

      benchmark.addCase(s"Parquet-side filter - number of filters [$num]", 3) { _ =>
        withSQLConf(SQLConf.PARQUET_VECTORIZED_READER_ENABLED.key -> false.toString,
          SQLConf.PARQUET_RECORD_FILTER_ENABLED.key -> true.toString) {

          // We should strip Spark-side filter to compare correctly.
          stripSparkFilter(
            spark.read.parquet(path.getAbsolutePath).filter(filterExpr)).count()
        }
      }

      benchmark.addCase(s"Spark-side filter - number of filters [$num]", 3) { _ =>
        withSQLConf(SQLConf.PARQUET_VECTORIZED_READER_ENABLED.key -> false.toString,
          SQLConf.PARQUET_RECORD_FILTER_ENABLED.key -> false.toString) {

          spark.read.parquet(path.getAbsolutePath).filter(filterExpr).count()
        }
      }
    }

    benchmark.run()
  }
}
```

**Result**

```
Parquet-side vs Spark-side:              Best/Avg Time(ms)    Rate(M/s)   Per Row(ns)   Relative
------------------------------------------------------------------------------------------------
Parquet-side filter - number of filters [5]      4268 / 4367        234.3           4.3       0.8X
Spark-side filter - number of filters [5]      3709 / 3741        269.6           3.7       0.9X
Parquet-side filter - number of filters [10]      5673 / 5727        176.3           5.7       0.6X
Spark-side filter - number of filters [10]      3588 / 3632        278.7           3.6       0.9X
Parquet-side filter - number of filters [20]      8024 / 8440        124.6           8.0       0.4X
Spark-side filter - number of filters [20]      3912 / 3946        255.6           3.9       0.8X
Parquet-side filter - number of filters [30]    11936 / 12041         83.8          11.9       0.3X
Spark-side filter - number of filters [30]      3929 / 3978        254.5           3.9       0.8X
```

Author: hyukjinkwon <gurwls223@gmail.com>

Closes #15049 from HyukjinKwon/SPARK-17310.
2017-11-14 12:34:21 +01:00
WeichenXu d8741b2b0f [SPARK-21911][ML][FOLLOW-UP] Fix doc for parallel ML Tuning in PySpark
## What changes were proposed in this pull request?

Fix doc issue mentioned here: https://github.com/apache/spark/pull/19122#issuecomment-340111834

## How was this patch tested?

N/A

Author: WeichenXu <weichen.xu@databricks.com>

Closes #19641 from WeichenXu123/fix_doc.
2017-11-13 17:00:51 -08:00
hyukjinkwon c8b7f97b8a [SPARK-22377][BUILD] Use /usr/sbin/lsof if lsof does not exists in release-build.sh
## What changes were proposed in this pull request?

This PR proposes to use `/usr/sbin/lsof` if `lsof` is missing in the path to fix nightly snapshot jenkins jobs. Please refer https://github.com/apache/spark/pull/19359#issuecomment-340139557:

> Looks like some of the snapshot builds are having lsof issues:
>
> https://amplab.cs.berkeley.edu/jenkins/view/Spark%20Packaging/job/spark-branch-2.1-maven-snapshots/182/console
>
>https://amplab.cs.berkeley.edu/jenkins/view/Spark%20Packaging/job/spark-branch-2.2-maven-snapshots/134/console
>
>spark-build/dev/create-release/release-build.sh: line 344: lsof: command not found
>usage: kill [ -s signal | -p ] [ -a ] pid ...
>kill -l [ signal ]

Up to my knowledge,  the full path of `lsof` is required for non-root user in few OSs.

## How was this patch tested?

Manually tested as below:

```bash
#!/usr/bin/env bash

LSOF=lsof
if ! hash $LSOF 2>/dev/null; then
  echo "a"
  LSOF=/usr/sbin/lsof
fi

$LSOF -P | grep "a"
```

Author: hyukjinkwon <gurwls223@gmail.com>

Closes #19695 from HyukjinKwon/SPARK-22377.
2017-11-14 08:28:13 +09:00
Wenchen Fan f7534b37ee [SPARK-22487][SQL][FOLLOWUP] still keep spark.sql.hive.version
## What changes were proposed in this pull request?

a followup of https://github.com/apache/spark/pull/19712 , adds back the `spark.sql.hive.version`, so that if users try to read this config, they can still get a default value instead of null.

## How was this patch tested?

N/A

Author: Wenchen Fan <wenchen@databricks.com>

Closes #19719 from cloud-fan/minor.
2017-11-13 13:10:13 -08:00
Xianyang Liu 176ae4d53e [MINOR][CORE] Using bufferedInputStream for dataDeserializeStream
## What changes were proposed in this pull request?

Small fix. Using bufferedInputStream for dataDeserializeStream.

## How was this patch tested?

Existing UT.

Author: Xianyang Liu <xianyang.liu@intel.com>

Closes #19735 from ConeyLiu/smallfix.
2017-11-13 06:19:13 -06:00
Bryan Cutler 209b9361ac [SPARK-20791][PYSPARK] Use Arrow to create Spark DataFrame from Pandas
## What changes were proposed in this pull request?

This change uses Arrow to optimize the creation of a Spark DataFrame from a Pandas DataFrame. The input df is sliced according to the default parallelism. The optimization is enabled with the existing conf "spark.sql.execution.arrow.enabled" and is disabled by default.

## How was this patch tested?

Added new unit test to create DataFrame with and without the optimization enabled, then compare results.

Author: Bryan Cutler <cutlerb@gmail.com>
Author: Takuya UESHIN <ueshin@databricks.com>

Closes #19459 from BryanCutler/arrow-createDataFrame-from_pandas-SPARK-20791.
2017-11-13 13:16:01 +09:00
hyukjinkwon 3d90b2cb38 [SPARK-21693][R][ML] Reduce max iterations in Linear SVM test in R to speed up AppVeyor build
## What changes were proposed in this pull request?

This PR proposes to reduce max iteration in Linear SVM test in SparkR. This particular test elapses roughly 5 mins on my Mac and over 20 mins on Windows.

The root cause appears, it triggers 2500ish jobs by the default 100 max iterations. In Linux, `daemon.R` is forked but on Windows another process is launched, which is extremely slow.

So, given my observation, there are many processes (not forked) ran on Windows, which makes the differences of elapsed time.

After reducing the max iteration to 10, the total jobs in this single test is reduced to 550ish.

After reducing the max iteration to 5, the total jobs in this single test is reduced to 360ish.

## How was this patch tested?

Manually tested the elapsed times.

Author: hyukjinkwon <gurwls223@gmail.com>

Closes #19722 from HyukjinKwon/SPARK-21693-test.
2017-11-12 14:37:20 -08:00
Kazuaki Ishizaki 9bf696dbec [SPARK-21720][SQL] Fix 64KB JVM bytecode limit problem with AND or OR
## What changes were proposed in this pull request?

This PR changes `AND` or `OR` code generation to place condition and then expressions' generated code into separated methods if these size could be large. When the method is newly generated, variables for `isNull` and `value` are declared as an instance variable to pass these values (e.g. `isNull1409` and `value1409`) to the callers of the generated method.

This PR resolved two cases:

* large code size of left expression
* large code size of right expression

## How was this patch tested?

Added a new test case into `CodeGenerationSuite`

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

Closes #18972 from kiszk/SPARK-21720.
2017-11-12 22:44:47 +01:00
Paul Mackles b3f9dbf48e [SPARK-19606][MESOS] Support constraints in spark-dispatcher
## What changes were proposed in this pull request?

A discussed in SPARK-19606, the addition of a new config property named "spark.mesos.constraints.driver" for constraining drivers running on a Mesos cluster

## How was this patch tested?

Corresponding unit test added also tested locally on a Mesos cluster

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

Author: Paul Mackles <pmackles@adobe.com>

Closes #19543 from pmackles/SPARK-19606.
2017-11-12 11:21:23 -08:00
Wenchen Fan 21a7bfd5c3 [SPARK-10365][SQL] Support Parquet logical type TIMESTAMP_MICROS
## What changes were proposed in this pull request?

This PR makes Spark to be able to read Parquet TIMESTAMP_MICROS values, and add a new config to allow Spark to write timestamp values to parquet as TIMESTAMP_MICROS type.

## How was this patch tested?

new test

Author: Wenchen Fan <wenchen@databricks.com>

Closes #19702 from cloud-fan/parquet.
2017-11-11 22:40:26 +01:00
gatorsmile d6ee69e776 [SPARK-22488][SQL] Fix the view resolution issue in the SparkSession internal table() API
## What changes were proposed in this pull request?
The current internal `table()` API of `SparkSession` bypasses the Analyzer and directly calls `sessionState.catalog.lookupRelation` API. This skips the view resolution logics in our Analyzer rule `ResolveRelations`. This internal API is widely used by various DDL commands, public and internal APIs.

Users might get the strange error caused by view resolution when the default database is different.
```
Table or view not found: t1; line 1 pos 14
org.apache.spark.sql.AnalysisException: Table or view not found: t1; line 1 pos 14
	at org.apache.spark.sql.catalyst.analysis.package$AnalysisErrorAt.failAnalysis(package.scala:42)
```

This PR is to fix it by enforcing it to use `ResolveRelations` to resolve the table.

## How was this patch tested?
Added a test case and modified the existing test cases

Author: gatorsmile <gatorsmile@gmail.com>

Closes #19713 from gatorsmile/viewResolution.
2017-11-11 18:20:11 +01:00
Liang-Chi Hsieh 154351e6db [SPARK-22462][SQL] Make rdd-based actions in Dataset trackable in SQL UI
## What changes were proposed in this pull request?

For the few Dataset actions such as `foreach`, currently no SQL metrics are visible in the SQL tab of SparkUI. It is because it binds wrongly to Dataset's `QueryExecution`. As the actions directly evaluate on the RDD which has individual `QueryExecution`, to show correct SQL metrics on UI, we should bind to RDD's `QueryExecution`.

## How was this patch tested?

Manually test. Screenshot is attached in the PR.

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

Closes #19689 from viirya/SPARK-22462.
2017-11-11 12:34:30 +01:00
hyukjinkwon 223d83ee93 [SPARK-22476][R] Add dayofweek function to R
## What changes were proposed in this pull request?

This PR adds `dayofweek` to R API:

```r
data <- list(list(d = as.Date("2012-12-13")),
             list(d = as.Date("2013-12-14")),
             list(d = as.Date("2014-12-15")))
df <- createDataFrame(data)
collect(select(df, dayofweek(df$d)))
```

```
  dayofweek(d)
1            5
2            7
3            2
```

## How was this patch tested?

Manual tests and unit tests in `R/pkg/tests/fulltests/test_sparkSQL.R`

Author: hyukjinkwon <gurwls223@gmail.com>

Closes #19706 from HyukjinKwon/add-dayofweek.
2017-11-11 19:16:31 +09:00
Marco Gaido 3eb315d714 [SPARK-19759][ML] not using blas in ALSModel.predict for optimization
## What changes were proposed in this pull request?

In `ALS.predict` currently we are using `blas.sdot` function to perform a dot product on two `Seq`s. It turns out that this is not the most efficient way.

I used the following code to compare the implementations:

```
def time[R](block: => R): Unit = {
    val t0 = System.nanoTime()
    block
    val t1 = System.nanoTime()
    println("Elapsed time: " + (t1 - t0) + "ns")
}
val r = new scala.util.Random(100)
val input = (1 to 500000).map(_ => (1 to 100).map(_ => r.nextFloat).toSeq)
def f(a:Seq[Float], b:Seq[Float]): Float = {
    var r = 0.0f
    for(i <- 0 until a.length) {
        r+=a(i)*b(i)
    }
    r
}
import com.github.fommil.netlib.BLAS.{getInstance => blas}
val b = (1 to 100).map(_ => r.nextFloat).toSeq
time { input.foreach(a=>blas.sdot(100, a.toArray, 1, b.toArray, 1)) }
// on average it takes 2968718815 ns
time { input.foreach(a=>f(a,b)) }
// on average it takes 515510185 ns
```

Thus this PR proposes the old-style for loop implementation for performance reasons.

## How was this patch tested?

existing UTs

Author: Marco Gaido <mgaido@hortonworks.com>

Closes #19685 from mgaido91/SPARK-19759.
2017-11-11 04:10:54 -06:00
Rekha Joshi 808e886b96 [SPARK-21667][STREAMING] ConsoleSink should not fail streaming query with checkpointLocation option
## What changes were proposed in this pull request?
Fix to allow recovery on console , avoid checkpoint exception

## How was this patch tested?
existing tests
manual tests [ Replicating error and seeing no checkpoint error after fix]

Author: Rekha Joshi <rekhajoshm@gmail.com>
Author: rjoshi2 <rekhajoshm@gmail.com>

Closes #19407 from rekhajoshm/SPARK-21667.
2017-11-10 15:18:11 -08:00
Kazuaki Ishizaki f2da738c76 [SPARK-22284][SQL] Fix 64KB JVM bytecode limit problem in calculating hash for nested structs
## What changes were proposed in this pull request?

This PR avoids to generate a huge method for calculating a murmur3 hash for nested structs. This PR splits a huge method (e.g. `apply_4`) into multiple smaller methods.

Sample program
```
  val structOfString = new StructType().add("str", StringType)
  var inner = new StructType()
  for (_ <- 0 until 800) {
    inner = inner1.add("structOfString", structOfString)
  }
  var schema = new StructType()
  for (_ <- 0 until 50) {
    schema = schema.add("structOfStructOfStrings", inner)
  }
  GenerateMutableProjection.generate(Seq(Murmur3Hash(exprs, 42)))
```

Without this PR
```
/* 005 */ class SpecificMutableProjection extends org.apache.spark.sql.catalyst.expressions.codegen.BaseMutableProjection {
/* 006 */
/* 007 */   private Object[] references;
/* 008 */   private InternalRow mutableRow;
/* 009 */   private int value;
/* 010 */   private int value_0;
...
/* 034 */   public java.lang.Object apply(java.lang.Object _i) {
/* 035 */     InternalRow i = (InternalRow) _i;
/* 036 */
/* 037 */
/* 038 */
/* 039 */     value = 42;
/* 040 */     apply_0(i);
/* 041 */     apply_1(i);
/* 042 */     apply_2(i);
/* 043 */     apply_3(i);
/* 044 */     apply_4(i);
/* 045 */     nestedClassInstance.apply_5(i);
...
/* 089 */     nestedClassInstance8.apply_49(i);
/* 090 */     value_0 = value;
/* 091 */
/* 092 */     // copy all the results into MutableRow
/* 093 */     mutableRow.setInt(0, value_0);
/* 094 */     return mutableRow;
/* 095 */   }
/* 096 */
/* 097 */
/* 098 */   private void apply_4(InternalRow i) {
/* 099 */
/* 100 */     boolean isNull5 = i.isNullAt(4);
/* 101 */     InternalRow value5 = isNull5 ? null : (i.getStruct(4, 800));
/* 102 */     if (!isNull5) {
/* 103 */
/* 104 */       if (!value5.isNullAt(0)) {
/* 105 */
/* 106 */         final InternalRow element6400 = value5.getStruct(0, 1);
/* 107 */
/* 108 */         if (!element6400.isNullAt(0)) {
/* 109 */
/* 110 */           final UTF8String element6401 = element6400.getUTF8String(0);
/* 111 */           value = org.apache.spark.unsafe.hash.Murmur3_x86_32.hashUnsafeBytes(element6401.getBaseObject(), element6401.getBaseOffset(), element6401.numBytes(), value);
/* 112 */
/* 113 */         }
/* 114 */
/* 115 */
/* 116 */       }
/* 117 */
/* 118 */
/* 119 */       if (!value5.isNullAt(1)) {
/* 120 */
/* 121 */         final InternalRow element6402 = value5.getStruct(1, 1);
/* 122 */
/* 123 */         if (!element6402.isNullAt(0)) {
/* 124 */
/* 125 */           final UTF8String element6403 = element6402.getUTF8String(0);
/* 126 */           value = org.apache.spark.unsafe.hash.Murmur3_x86_32.hashUnsafeBytes(element6403.getBaseObject(), element6403.getBaseOffset(), element6403.numBytes(), value);
/* 127 */
/* 128 */         }
/* 128 */         }
/* 129 */
/* 130 */
/* 131 */       }
/* 132 */
/* 133 */
/* 134 */       if (!value5.isNullAt(2)) {
/* 135 */
/* 136 */         final InternalRow element6404 = value5.getStruct(2, 1);
/* 137 */
/* 138 */         if (!element6404.isNullAt(0)) {
/* 139 */
/* 140 */           final UTF8String element6405 = element6404.getUTF8String(0);
/* 141 */           value = org.apache.spark.unsafe.hash.Murmur3_x86_32.hashUnsafeBytes(element6405.getBaseObject(), element6405.getBaseOffset(), element6405.numBytes(), value);
/* 142 */
/* 143 */         }
/* 144 */
/* 145 */
/* 146 */       }
/* 147 */
...
/* 12074 */       if (!value5.isNullAt(798)) {
/* 12075 */
/* 12076 */         final InternalRow element7996 = value5.getStruct(798, 1);
/* 12077 */
/* 12078 */         if (!element7996.isNullAt(0)) {
/* 12079 */
/* 12080 */           final UTF8String element7997 = element7996.getUTF8String(0);
/* 12083 */         }
/* 12084 */
/* 12085 */
/* 12086 */       }
/* 12087 */
/* 12088 */
/* 12089 */       if (!value5.isNullAt(799)) {
/* 12090 */
/* 12091 */         final InternalRow element7998 = value5.getStruct(799, 1);
/* 12092 */
/* 12093 */         if (!element7998.isNullAt(0)) {
/* 12094 */
/* 12095 */           final UTF8String element7999 = element7998.getUTF8String(0);
/* 12096 */           value = org.apache.spark.unsafe.hash.Murmur3_x86_32.hashUnsafeBytes(element7999.getBaseObject(), element7999.getBaseOffset(), element7999.numBytes(), value);
/* 12097 */
/* 12098 */         }
/* 12099 */
/* 12100 */
/* 12101 */       }
/* 12102 */
/* 12103 */     }
/* 12104 */
/* 12105 */   }
/* 12106 */
/* 12106 */
/* 12107 */
/* 12108 */   private void apply_1(InternalRow i) {
...
```

With this PR
```
/* 005 */ class SpecificMutableProjection extends org.apache.spark.sql.catalyst.expressions.codegen.BaseMutableProjection {
/* 006 */
/* 007 */   private Object[] references;
/* 008 */   private InternalRow mutableRow;
/* 009 */   private int value;
/* 010 */   private int value_0;
/* 011 */
...
/* 034 */   public java.lang.Object apply(java.lang.Object _i) {
/* 035 */     InternalRow i = (InternalRow) _i;
/* 036 */
/* 037 */
/* 038 */
/* 039 */     value = 42;
/* 040 */     nestedClassInstance11.apply50_0(i);
/* 041 */     nestedClassInstance11.apply50_1(i);
...
/* 088 */     nestedClassInstance11.apply50_48(i);
/* 089 */     nestedClassInstance11.apply50_49(i);
/* 090 */     value_0 = value;
/* 091 */
/* 092 */     // copy all the results into MutableRow
/* 093 */     mutableRow.setInt(0, value_0);
/* 094 */     return mutableRow;
/* 095 */   }
/* 096 */
...
/* 37717 */   private void apply4_0(InternalRow value5, InternalRow i) {
/* 37718 */
/* 37719 */     if (!value5.isNullAt(0)) {
/* 37720 */
/* 37721 */       final InternalRow element6400 = value5.getStruct(0, 1);
/* 37722 */
/* 37723 */       if (!element6400.isNullAt(0)) {
/* 37724 */
/* 37725 */         final UTF8String element6401 = element6400.getUTF8String(0);
/* 37726 */         value = org.apache.spark.unsafe.hash.Murmur3_x86_32.hashUnsafeBytes(element6401.getBaseObject(), element6401.getBaseOffset(), element6401.numBytes(), value);
/* 37727 */
/* 37728 */       }
/* 37729 */
/* 37730 */
/* 37731 */     }
/* 37732 */
/* 37733 */     if (!value5.isNullAt(1)) {
/* 37734 */
/* 37735 */       final InternalRow element6402 = value5.getStruct(1, 1);
/* 37736 */
/* 37737 */       if (!element6402.isNullAt(0)) {
/* 37738 */
/* 37739 */         final UTF8String element6403 = element6402.getUTF8String(0);
/* 37740 */         value = org.apache.spark.unsafe.hash.Murmur3_x86_32.hashUnsafeBytes(element6403.getBaseObject(), element6403.getBaseOffset(), element6403.numBytes(), value);
/* 37741 */
/* 37742 */       }
/* 37743 */
/* 37744 */
/* 37745 */     }
/* 37746 */
/* 37747 */     if (!value5.isNullAt(2)) {
/* 37748 */
/* 37749 */       final InternalRow element6404 = value5.getStruct(2, 1);
/* 37750 */
/* 37751 */       if (!element6404.isNullAt(0)) {
/* 37752 */
/* 37753 */         final UTF8String element6405 = element6404.getUTF8String(0);
/* 37754 */         value = org.apache.spark.unsafe.hash.Murmur3_x86_32.hashUnsafeBytes(element6405.getBaseObject(), element6405.getBaseOffset(), element6405.numBytes(), value);
/* 37755 */
/* 37756 */       }
/* 37757 */
/* 37758 */
/* 37759 */     }
/* 37760 */
/* 37761 */   }
...
/* 218470 */
/* 218471 */     private void apply50_4(InternalRow i) {
/* 218472 */
/* 218473 */       boolean isNull5 = i.isNullAt(4);
/* 218474 */       InternalRow value5 = isNull5 ? null : (i.getStruct(4, 800));
/* 218475 */       if (!isNull5) {
/* 218476 */         apply4_0(value5, i);
/* 218477 */         apply4_1(value5, i);
/* 218478 */         apply4_2(value5, i);
...
/* 218742 */         nestedClassInstance.apply4_266(value5, i);
/* 218743 */       }
/* 218744 */
/* 218745 */     }
```

## How was this patch tested?

Added new test to `HashExpressionsSuite`

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

Closes #19563 from kiszk/SPARK-22284.
2017-11-10 21:17:49 +01:00
Shixiong Zhu 24ea781cd3 [SPARK-19644][SQL] Clean up Scala reflection garbage after creating Encoder
## What changes were proposed in this pull request?

Because of the memory leak issue in `scala.reflect.api.Types.TypeApi.<:<` (https://github.com/scala/bug/issues/8302), creating an encoder may leak memory.

This PR adds `cleanUpReflectionObjects` to clean up these leaking objects for methods calling `scala.reflect.api.Types.TypeApi.<:<`.

## How was this patch tested?

The updated unit tests.

Author: Shixiong Zhu <zsxwing@gmail.com>

Closes #19687 from zsxwing/SPARK-19644.
2017-11-10 11:27:28 -08:00
Santiago Saavedra 5ebdcd185f [SPARK-22294][DEPLOY] Reset spark.driver.bindAddress when starting a Checkpoint
## What changes were proposed in this pull request?

It seems that recovering from a checkpoint can replace the old
driver and executor IP addresses, as the workload can now be taking
place in a different cluster configuration. It follows that the
bindAddress for the master may also have changed. Thus we should not be
keeping the old one, and instead be added to the list of properties to
reset and recreate from the new environment.

## How was this patch tested?

This patch was tested via manual testing on AWS, using the experimental (not yet merged) Kubernetes scheduler, which uses bindAddress to bind to a Kubernetes service (and thus was how I first encountered the bug too), but it is not a code-path related to the scheduler and this may have slipped through when merging SPARK-4563.

Author: Santiago Saavedra <ssaavedra@openshine.com>

Closes #19427 from ssaavedra/fix-checkpointing-master.
2017-11-10 10:57:58 -08:00
Felix Cheung b70aa9e08b [SPARK-22344][SPARKR] clean up install dir if running test as source package
## What changes were proposed in this pull request?

remove spark if spark downloaded & installed

## How was this patch tested?

manually by building package
Jenkins, AppVeyor

Author: Felix Cheung <felixcheung_m@hotmail.com>

Closes #19657 from felixcheung/rinstalldir.
2017-11-10 10:22:42 -08:00
Marco Gaido 5b41cbf13b [SPARK-22473][TEST] Replace deprecated AsyncAssertions.Waiter and methods of java.sql.Date
## What changes were proposed in this pull request?

In `spark-sql` module tests there are deprecations warnings caused by the usage of deprecated methods of `java.sql.Date` and the usage of the deprecated `AsyncAssertions.Waiter` class.
This PR replace the deprecated methods of `java.sql.Date` with non-deprecated ones (using `Calendar` where needed). It replaces also the deprecated `org.scalatest.concurrent.AsyncAssertions.Waiter` with `org.scalatest.concurrent.Waiters._`.

## How was this patch tested?

existing UTs

Author: Marco Gaido <mgaido@hortonworks.com>

Closes #19696 from mgaido91/SPARK-22473.
2017-11-10 11:24:24 -06:00
Xianyang Liu 1c923d7d65 [SPARK-22450][CORE][MLLIB] safely register class for mllib
## What changes were proposed in this pull request?

There are still some algorithms based on mllib, such as KMeans. For now, many mllib common class (such as: Vector, DenseVector, SparseVector, Matrix, DenseMatrix, SparseMatrix) are not registered in Kryo. So there are some performance issues for those object serialization or deserialization.
Previously dicussed: https://github.com/apache/spark/pull/19586

## How was this patch tested?

New test case.

Author: Xianyang Liu <xianyang.liu@intel.com>

Closes #19661 from ConeyLiu/register_vector.
2017-11-10 12:43:29 +01:00
Pralabh Kumar 9b9827759a [SPARK-20199][ML] : Provided featureSubsetStrategy to GBTClassifier and GBTRegressor
## What changes were proposed in this pull request?

(Provided featureSubset Strategy to GBTClassifier
a) Moved featureSubsetStrategy to TreeEnsembleParams
b)  Changed GBTClassifier to pass featureSubsetStrategy
val firstTreeModel = firstTree.train(input, treeStrategy, featureSubsetStrategy))

## How was this patch tested?
a) Tested GradientBoostedTreeClassifierExample by adding .setFeatureSubsetStrategy with GBTClassifier

b)Added test cases in GBTClassifierSuite and GBTRegressorSuite

Author: Pralabh Kumar <pralabhkumar@gmail.com>

Closes #18118 from pralabhkumar/develop.
2017-11-10 13:17:25 +02:00
Kent Yao 28ab5bf597 [SPARK-22487][SQL][HIVE] Remove the unused HIVE_EXECUTION_VERSION property
## What changes were proposed in this pull request?

At the beginning https://github.com/apache/spark/pull/2843 added `spark.sql.hive.version` to reveal underlying hive version for jdbc connections. For some time afterwards, it was used as a version identifier for the execution hive client.

Actually there is no hive client for executions in spark now and there are no usages of HIVE_EXECUTION_VERSION found in whole spark project. HIVE_EXECUTION_VERSION is set by `spark.sql.hive.version`, which is still set internally in some places or by users, this may confuse developers and users with HIVE_METASTORE_VERSION(spark.sql.hive.metastore.version).

It might better to be removed.

## How was this patch tested?

modify some existing ut

cc cloud-fan gatorsmile

Author: Kent Yao <yaooqinn@hotmail.com>

Closes #19712 from yaooqinn/SPARK-22487.
2017-11-10 12:01:02 +01:00
Wenchen Fan 0025ddeb1d [SPARK-22472][SQL] add null check for top-level primitive values
## What changes were proposed in this pull request?

One powerful feature of `Dataset` is, we can easily map SQL rows to Scala/Java objects and do runtime null check automatically.

For example, let's say we have a parquet file with schema `<a: int, b: string>`, and we have a `case class Data(a: Int, b: String)`. Users can easily read this parquet file into `Data` objects, and Spark will throw NPE if column `a` has null values.

However the null checking is left behind for top-level primitive values. For example, let's say we have a parquet file with schema `<a: Int>`, and we read it into Scala `Int`. If column `a` has null values, we will get some weird results.
```
scala> val ds = spark.read.parquet(...).as[Int]

scala> ds.show()
+----+
|v   |
+----+
|null|
|1   |
+----+

scala> ds.collect
res0: Array[Long] = Array(0, 1)

scala> ds.map(_ * 2).show
+-----+
|value|
+-----+
|-2   |
|2    |
+-----+
```

This is because internally Spark use some special default values for primitive types, but never expect users to see/operate these default value directly.

This PR adds null check for top-level primitive values

## How was this patch tested?

new test

Author: Wenchen Fan <wenchen@databricks.com>

Closes #19707 from cloud-fan/bug.
2017-11-09 21:56:20 -08:00
Nathan Kronenfeld b57ed2245c [SPARK-22308][TEST-MAVEN] Support alternative unit testing styles in external applications
Continuation of PR#19528 (https://github.com/apache/spark/pull/19529#issuecomment-340252119)

The problem with the maven build in the previous PR was the new tests.... the creation of a spark session outside the tests meant there was more than one spark session around at a time.
I was using the spark session outside the tests so that the tests could share data; I've changed it so that each test creates the data anew.

Author: Nathan Kronenfeld <nicole.oresme@gmail.com>
Author: Nathan Kronenfeld <nkronenfeld@uncharted.software>

Closes #19705 from nkronenfeld/alternative-style-tests-2.
2017-11-09 19:11:30 -08:00
Paul Mackles f5fe63f7b8 [SPARK-22287][MESOS] SPARK_DAEMON_MEMORY not honored by MesosClusterD…
…ispatcher

## What changes were proposed in this pull request?

Allow JVM max heap size to be controlled for MesosClusterDispatcher via SPARK_DAEMON_MEMORY environment variable.

## How was this patch tested?

Tested on local Mesos cluster

Author: Paul Mackles <pmackles@adobe.com>

Closes #19515 from pmackles/SPARK-22287.
2017-11-09 16:42:33 -08:00
Dongjoon Hyun 64c989495a [SPARK-22485][BUILD] Use exclude[Problem] instead excludePackage in MiMa
## What changes were proposed in this pull request?

`excludePackage` is deprecated like the [following](https://github.com/lightbend/migration-manager/blob/master/core/src/main/scala/com/typesafe/tools/mima/core/Filters.scala#L33-L36) and shows deprecation warnings now. This PR uses `exclude[Problem](packageName + ".*")` instead.

```scala
deprecated("Replace with ProblemFilters.exclude[Problem](\"my.package.*\")", "0.1.15")
def excludePackage(packageName: String): ProblemFilter = {
  exclude[Problem](packageName + ".*")
}
```

## How was this patch tested?

Pass the Jenkins MiMa.

Author: Dongjoon Hyun <dongjoon@apache.org>

Closes #19710 from dongjoon-hyun/SPARK-22485.
2017-11-09 16:40:19 -08:00