Commit graph

22668 commits

Author SHA1 Message Date
Sean Owen 1a7e747ce4 [SPARK-25047][ML] Can't assign SerializedLambda to scala.Function1 in deserialization of BucketedRandomProjectionLSHModel
## What changes were proposed in this pull request?

Convert two function fields in ML classes to simple functions to avoi…d odd SerializedLambda deserialization problem

## How was this patch tested?

Existing tests.

Closes #22032 from srowen/SPARK-25047.

Authored-by: Sean Owen <srowen@gmail.com>
Signed-off-by: Sean Owen <srowen@gmail.com>
2018-08-09 08:07:46 -05:00
Wenchen Fan b2950cef3c Revert "[SPARK-24648][SQL] SqlMetrics should be threadsafe"
This reverts commit 5264164a67.
2018-08-09 20:33:59 +08:00
Kazuaki Ishizaki 386fbd3aff [SPARK-23415][SQL][TEST] Make behavior of BufferHolderSparkSubmitSuite correct and stable
## What changes were proposed in this pull request?

This PR addresses two issues in `BufferHolderSparkSubmitSuite`.

1. While `BufferHolderSparkSubmitSuite` tried to allocate a large object several times, it actually allocated an object once and reused the object.
2. `BufferHolderSparkSubmitSuite` may fail due to timeout

To assign a small object before allocating a large object each time solved issue 1 by avoiding reuse.
To increasing heap size from 4g to 7g solved issue 2. It can also avoid OOM after fixing issue 1.

## How was this patch tested?

Updated existing `BufferHolderSparkSubmitSuite`

Closes #20636 from kiszk/SPARK-23415.

Authored-by: Kazuaki Ishizaki <ishizaki@jp.ibm.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2018-08-09 20:28:14 +08:00
Kazuaki Ishizaki 56e9e97073 [MINOR][DOC] Fix typo
## What changes were proposed in this pull request?

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

## How was this patch tested?

N/A

Closes #22040 from kiszk/spellcheck1.

Authored-by: Kazuaki Ishizaki <ishizaki@jp.ibm.com>
Signed-off-by: hyukjinkwon <gurwls223@apache.org>
2018-08-09 20:10:17 +08:00
Takuya UESHIN 519e03d82e [SPARK-25058][SQL] Use Block.isEmpty/nonEmpty to check whether the code is empty or not.
## What changes were proposed in this pull request?

We should use `Block.isEmpty/nonEmpty` instead of comparing with empty string to check whether the code is empty or not.

```
[error] [warn] /.../sql/core/src/main/scala/org/apache/spark/sql/execution/WholeStageCodegenExec.scala:278: org.apache.spark.sql.catalyst.expressions.codegen.Block and String are unrelated: they will most likely always compare unequal
[error] [warn]       if (ev.code != "" && required.contains(attributes(i))) {
[error] [warn]
[error] [warn] /.../sql/core/src/main/scala/org/apache/spark/sql/execution/joins/BroadcastHashJoinExec.scala:323: org.apache.spark.sql.catalyst.expressions.codegen.Block and String are unrelated: they will most likely never compare equal
[error] [warn]          |  ${buildVars.filter(_.code == "").map(v => s"${v.isNull} = true;").mkString("\n")}
[error] [warn]
```

## How was this patch tested?

Existing tests.

Closes #22041 from ueshin/issues/SPARK-25058/fix_comparison.

Authored-by: Takuya UESHIN <ueshin@databricks.com>
Signed-off-by: Takuya UESHIN <ueshin@databricks.com>
2018-08-09 14:06:28 +09:00
Liang-Chi Hsieh a40806d2bd [SPARK-23596][SQL] Test interpreted path on encoders test suites
## What changes were proposed in this pull request?

We have completed a significant subset of the object related Expressions to provide an interpreted fallback. This PR is going to modify the tests to also test the interpreted code paths.

One concern right now is that by testing the interpreted code paths too, we will double current test time or more. Otherwise, we can only choose to test the interpreted code paths for just few test suites such as encoder related.

## How was this patch tested?

Existing tests.

Closes #21535 from viirya/SPARK-23596.

Authored-by: Liang-Chi Hsieh <viirya@gmail.com>
Signed-off-by: hyukjinkwon <gurwls223@apache.org>
2018-08-09 12:07:57 +08:00
Takuya UESHIN f62fe435de [SPARK-25036][SQL][FOLLOW-UP] Avoid match may not be exhaustive in Scala-2.12.
## What changes were proposed in this pull request?

This is a follow-up pr of #22014.

We still have some more compilation errors in scala-2.12 with sbt:

```
[error] [warn] /.../sql/core/src/main/scala/org/apache/spark/sql/DataFrameNaFunctions.scala:493: match may not be exhaustive.
[error] It would fail on the following input: (_, _)
[error] [warn]       val typeMatches = (targetType, f.dataType) match {
[error] [warn]
[error] [warn] /.../sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/MicroBatchExecution.scala:393: match may not be exhaustive.
[error] It would fail on the following input: (_, _)
[error] [warn]             prevBatchOff.get.toStreamProgress(sources).foreach {
[error] [warn]
[error] [warn] /.../sql/core/src/main/scala/org/apache/spark/sql/execution/aggregate/AggUtils.scala:173: match may not be exhaustive.
[error] It would fail on the following input: AggregateExpression(_, _, false, _)
[error] [warn]     val rewrittenDistinctFunctions = functionsWithDistinct.map {
[error] [warn]
[error] [warn] /.../sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/state/SymmetricHashJoinStateManager.scala:271: match may not be exhaustive.
[error] It would fail on the following input: (_, _)
[error] [warn]       keyWithIndexToValueMetrics.customMetrics.map {
[error] [warn]
[error] [warn] /.../sql/core/src/main/scala/org/apache/spark/sql/execution/command/tables.scala:959: match may not be exhaustive.
[error] It would fail on the following input: CatalogTableType(_)
[error] [warn]     val tableTypeString = metadata.tableType match {
[error] [warn]
[error] [warn] /.../sql/hive/src/main/scala/org/apache/spark/sql/hive/client/HiveClientImpl.scala:923: match may not be exhaustive.
[error] It would fail on the following input: CatalogTableType(_)
[error] [warn]     hiveTable.setTableType(table.tableType match {
[error] [warn]
```

## How was this patch tested?

Manually build with Scala-2.12.

Closes #22039 from ueshin/issues/SPARK-25036/fix_match.

Authored-by: Takuya UESHIN <ueshin@databricks.com>
Signed-off-by: Sean Owen <srowen@gmail.com>
2018-08-08 16:47:22 -05:00
Takuya UESHIN 6f6a420078 [SPARK-23911][SQL][FOLLOW-UP] Fix examples of aggregate function.
## What changes were proposed in this pull request?

This pr is a follow-up pr of #21982 and fixes the examples.

## How was this patch tested?

Existing tests.

Closes #22035 from ueshin/issues/SPARK-23911/fup1.

Authored-by: Takuya UESHIN <ueshin@databricks.com>
Signed-off-by: Takuya UESHIN <ueshin@databricks.com>
2018-08-09 00:01:03 +09:00
Kazuaki Ishizaki 960af63913 [SPARK-25036][SQL] avoid match may not be exhaustive in Scala-2.12
## What changes were proposed in this pull request?

The PR remove the following compilation error using scala-2.12 with sbt by adding a default case to `match`.

```
/home/ishizaki/Spark/PR/scala212/spark/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/statsEstimation/ValueInterval.scala:63: match may not be exhaustive.
[error] It would fail on the following inputs: (NumericValueInterval(_, _), _), (_, NumericValueInterval(_, _)), (_, _)
[error] [warn]   def isIntersected(r1: ValueInterval, r2: ValueInterval): Boolean = (r1, r2) match {
[error] [warn]
[error] [warn] /home/ishizaki/Spark/PR/scala212/spark/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/statsEstimation/ValueInterval.scala:79: match may not be exhaustive.
[error] It would fail on the following inputs: (NumericValueInterval(_, _), _), (_, NumericValueInterval(_, _)), (_, _)
[error] [warn]     (r1, r2) match {
[error] [warn]
[error] [warn] /home/ishizaki/Spark/PR/scala212/spark/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/aggregate/ApproxCountDistinctForIntervals.scala:67: match may not be exhaustive.
[error] It would fail on the following inputs: (ArrayType(_, _), _), (_, ArrayData()), (_, _)
[error] [warn]     (endpointsExpression.dataType, endpointsExpression.eval()) match {
[error] [warn]
[error] [warn] /home/ishizaki/Spark/PR/scala212/spark/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/codegen/CodeGenerator.scala:470: match may not be exhaustive.
[error] It would fail on the following inputs: NewFunctionSpec(_, None, Some(_)), NewFunctionSpec(_, Some(_), None)
[error] [warn]     newFunction match {
[error] [warn]
[error] [warn] [error] [warn] /home/ishizaki/Spark/PR/scala212/spark/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/ScalaReflection.scala:709: match may not be exhaustive.
[error] It would fail on the following input: Schema((x: org.apache.spark.sql.types.DataType forSome x not in org.apache.spark.sql.types.StructType), _)
[error] [warn]   def attributesFor[T: TypeTag]: Seq[Attribute] = schemaFor[T] match {
[error] [warn]
```

## How was this patch tested?

Existing UTs with Scala-2.11.
Manually build with Scala-2.12

Closes #22014 from kiszk/SPARK-25036b.

Authored-by: Kazuaki Ishizaki <ishizaki@jp.ibm.com>
Signed-off-by: hyukjinkwon <gurwls223@apache.org>
2018-08-08 14:46:00 +08:00
Kazuaki Ishizaki f08f6f4314 [SPARK-23935][SQL][FOLLOWUP] mapEntry throws org.codehaus.commons.compiler.CompileException
## What changes were proposed in this pull request?

This PR fixes an exception during the compilation of generated code of `mapEntry`. This error occurs since the current code uses `key` type to store a `value` when `key` and `value` types are primitive type.

```
     val mid0 = Literal.create(Map(1 -> 1.1, 2 -> 2.2), MapType(IntegerType, DoubleType))
     checkEvaluation(MapEntries(mid0), Seq(r(1, 1.1), r(2, 2.2)))
```

```
[info]   Code generation of map_entries(keys: [1,2], values: [1.1,2.2]) failed:
[info]   java.util.concurrent.ExecutionException: org.codehaus.commons.compiler.CompileException: File 'generated.java', Line 80, Column 20: failed to compile: org.codehaus.commons.compiler.CompileException: File 'generated.java', Line 80, Column 20: No applicable constructor/method found for actual parameters "int, double"; candidates are: "public void org.apache.spark.sql.catalyst.expressions.UnsafeRow.setInt(int, int)", "public void org.apache.spark.sql.catalyst.InternalRow.setInt(int, int)"
[info]   java.util.concurrent.ExecutionException: org.codehaus.commons.compiler.CompileException: File 'generated.java', Line 80, Column 20: failed to compile: org.codehaus.commons.compiler.CompileException: File 'generated.java', Line 80, Column 20: No applicable constructor/method found for actual parameters "int, double"; candidates are: "public void org.apache.spark.sql.catalyst.expressions.UnsafeRow.setInt(int, int)", "public void org.apache.spark.sql.catalyst.InternalRow.setInt(int, int)"
[info]   	at com.google.common.util.concurrent.AbstractFuture$Sync.getValue(AbstractFuture.java:306)
[info]   	at com.google.common.util.concurrent.AbstractFuture$Sync.get(AbstractFuture.java:293)
[info]   	at com.google.common.util.concurrent.AbstractFuture.get(AbstractFuture.java:116)
[info]   	at com.google.common.util.concurrent.Uninterruptibles.getUninterruptibly(Uninterruptibles.java:135)
[info]   	at com.google.common.cache.LocalCache$Segment.getAndRecordStats(LocalCache.java:2410)
[info]   	at com.google.common.cache.LocalCache$Segment.loadSync(LocalCache.java:2380)
[info]   	at com.google.common.cache.LocalCache$Segment.lockedGetOrLoad(LocalCache.java:2342)
[info]   	at com.google.common.cache.LocalCache$Segment.get(LocalCache.java:2257)
[info]   	at com.google.common.cache.LocalCache.get(LocalCache.java:4000)
[info]   	at com.google.common.cache.LocalCache.getOrLoad(LocalCache.java:4004)
[info]   	at com.google.common.cache.LocalCache$LocalLoadingCache.get(LocalCache.java:4874)
[info]   	at org.apache.spark.sql.catalyst.expressions.codegen.CodeGenerator$.compile(CodeGenerator.scala:1290)
...
```

## How was this patch tested?

Added a new test to `CollectionExpressionsSuite`

Closes #22033 from kiszk/SPARK-23935-followup.

Authored-by: Kazuaki Ishizaki <ishizaki@jp.ibm.com>
Signed-off-by: Takuya UESHIN <ueshin@databricks.com>
2018-08-08 14:38:55 +09:00
Takuya UESHIN c7a229d655 [SPARK-25010][SQL][FOLLOWUP] Shuffle should also produce different values for each execution in streaming query.
## What changes were proposed in this pull request?

This is a follow-up pr of #21980.

`Shuffle` can also be `ExpressionWithRandomSeed` to produce different values for each execution in streaming query.

## How was this patch tested?

Added a test.

Closes #22027 from ueshin/issues/SPARK-25010/random_seed.

Authored-by: Takuya UESHIN <ueshin@databricks.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2018-08-08 11:05:52 +08:00
Ryan Blue 5fef6e3513 [SPARK-24251][SQL] Add AppendData logical plan.
## What changes were proposed in this pull request?

This adds a new logical plan, AppendData, that was proposed in SPARK-23521: Standardize SQL logical plans.

* DataFrameWriter uses the new AppendData plan for DataSourceV2 appends
* AppendData is resolved if its output columns match the incoming data frame
* A new analyzer rule, ResolveOutputColumns, validates data before it is appended. This rule will add safe casts, rename columns, and checks nullability

## How was this patch tested?

Existing tests for v2 appends. Will add AppendData tests to validate logical plan analysis.

Closes #21305 from rdblue/SPARK-24251-add-append-data.

Lead-authored-by: Ryan Blue <blue@apache.org>
Co-authored-by: Ryan Blue <rdblue@users.noreply.github.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2018-08-08 09:55:52 +08:00
Xingbo Jiang d90f1336d8 [SPARK-25045][CORE] Make RDDBarrier.mapParititions similar to RDD.mapPartitions
## What changes were proposed in this pull request?

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

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

## How was this patch tested?

Existing test cases.

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

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

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

## How was this patch tested?

Existing tests.

Closes #22004 from srowen/SPARK-25029.

Authored-by: Sean Owen <srowen@gmail.com>
Signed-off-by: Sean Owen <srowen@gmail.com>
2018-08-07 17:30:37 -05:00
Neal Song f6356f9bc0 [SPARK-25046][SQL] Fix Alter View can excute sql like "ALTER VIEW ... AS INSERT INTO"
## What changes were proposed in this pull request?

Alter View  can excute sql  like "ALTER VIEW ... AS INSERT INTO" . We should throw ParseException(s"Operation not allowed: $message", ctx)  as Create View does.
 ```
override def visitCreateView(ctx: CreateViewContext): LogicalPlan = withOrigin(ctx) {
    if (ctx.identifierList != null) {
      operationNotAllowed("CREATE VIEW ... PARTITIONED ON", ctx)
    } else {
      // CREATE VIEW ... AS INSERT INTO is not allowed.
      ctx.query.queryNoWith match {
        case s: SingleInsertQueryContext if s.insertInto != null =>
          operationNotAllowed("CREATE VIEW ... AS INSERT INTO", ctx)
        case _: MultiInsertQueryContext =>
          operationNotAllowed("CREATE VIEW ... AS FROM ... [INSERT INTO ...]+", ctx)
        case _ => // OK
      }
```

```
override def visitAlterViewQuery(ctx: AlterViewQueryContext): LogicalPlan = withOrigin(ctx) {
    // ALTER VIEW ... AS INSERT INTO is not allowed.
    ctx.query.queryNoWith match {
      case s: SingleInsertQueryContext if s.insertInto != null =>
        operationNotAllowed("ALTER VIEW ... AS INSERT INTO", ctx)
      case _: MultiInsertQueryContext =>
        operationNotAllowed("ALTER VIEW ... AS FROM ... [INSERT INTO ...]+", ctx)
      case _ => // OK
    }
    AlterViewAsCommand(
      name = visitTableIdentifier(ctx.tableIdentifier),
      originalText = source(ctx.query),
      query = plan(ctx.query))
  }
```

## How was this patch tested?

UT has been added in SparkSqlParserSuite

Closes #22028 from sddyljsx/SPARK-25046.

Lead-authored-by: Neal Song <neal_song@126.com>
Co-authored-by: neal <neal_song@126.com>
Signed-off-by: Xiao Li <gatorsmile@gmail.com>
2018-08-07 14:51:41 -07:00
invkrh 8c13cb2ae4 [SPARK-25031][SQL] Fix MapType schema print
## What changes were proposed in this pull request?

The PR fix the bug in `buildFormattedString` function in `MapType`, which makes the printed schema misleading.

## How was this patch tested?

Added UT

Closes #22006 from invkrh/fix-map-schema-print.

Authored-by: invkrh <invkrh@gmail.com>
Signed-off-by: Xiao Li <gatorsmile@gmail.com>
2018-08-07 11:04:37 -07:00
Marco Gaido cb6cb31363 [SPARK-23937][SQL] Add map_filter SQL function
## What changes were proposed in this pull request?

The PR adds the high order function `map_filter`, which filters the entries of a map and returns a new map which contains only the entries which satisfied the filter function.

## How was this patch tested?

added UTs

Closes #21986 from mgaido91/SPARK-23937.

Authored-by: Marco Gaido <marcogaido91@gmail.com>
Signed-off-by: Takuya UESHIN <ueshin@databricks.com>
2018-08-08 02:12:19 +09:00
Kazuaki Ishizaki 298e80f5c7 [SPARK-25041][BUILD] upgrade genJavaDoc-plugin from 0.10 to 0.11
## What changes were proposed in this pull request?

This PR fixes a build error with sbt using Scala-2.12. Since [`genJavaDoc-plugin`] (https://mvnrepository.com/artifact/com.typesafe.genjavadoc/genjavadoc-plugin) 0.10 is not prepared for Scala-2.12.6, the recent version of `genJavaDoc-plugin` is necessary.

The version 0.11 of `genJavaDoc-plugin` is also prepared for Scala-2.11.12.

[genJavaDoc-0.10](https://index.scala-lang.org/lightbend/genjavadoc/genjavadoc-plugin/0.10)
[genJavaDoc-0.11](https://index.scala-lang.org/lightbend/genjavadoc/genjavadoc-plugin/0.11)

## How was this patch tested?

Manually tested for Scala-2.12.

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

Closes #22020 from kiszk/SPARK-25041.
2018-08-07 11:58:44 -05:00
Wenchen Fan 1a29fec8e2 [SPARK-24979][SQL] add AnalysisHelper#resolveOperatorsUp
## What changes were proposed in this pull request?

This is a followup of https://github.com/apache/spark/pull/21822

Similar to `TreeNode`, `AnalysisHelper` should also provide 3 versions of transformations: `resolveOperatorsUp`, `resolveOperatorsDown` and `resolveOperators`.

This PR adds the missing `resolveOperatorsUp`, and also fixes some code style which is missed in #21822

## How was this patch tested?

existing tests.

Author: Wenchen Fan <wenchen@databricks.com>

Closes #21932 from cloud-fan/follow.
2018-08-07 08:45:20 -07:00
Marco Gaido 6a143e3ebf [SPARK-23928][TESTS][FOLLOWUP] Set seed to avoid flakiness
## What changes were proposed in this pull request?

The tests for shuffle can be flaky (eg. https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/94355/testReport/). This happens because we have not set the seed for `Random`.

## How was this patch tested?

running 10000 times the UT (validated that with a different seed eg. 12345 the test fails).

Closes #22023 from mgaido91/SPARK-23928_followup.

Authored-by: Marco Gaido <marcogaido91@gmail.com>
Signed-off-by: hyukjinkwon <gurwls223@apache.org>
2018-08-07 22:23:59 +08:00
Sunitha Kambhampati b4bf8be549 [SPARK-19602][SQL] Support column resolution of fully qualified column name ( 3 part name)
## What changes were proposed in this pull request?
The design details is attached to the JIRA issue [here](https://drive.google.com/file/d/1zKm3aNZ3DpsqIuoMvRsf0kkDkXsAasxH/view)

High level overview of the changes are:
- Enhance the qualifier to be more than one string
- Add support to store the qualifier. Enhance the lookupRelation to keep the qualifier appropriately.
- Enhance the table matching column resolution algorithm to account for qualifier being more than a string.
- Enhance the table matching algorithm in UnresolvedStar.expand
- Ensure that we continue to support select t1.i1 from db1.t1

## How was this patch tested?
- New tests are added.
- Several test scenarios were added in a separate  [test pr 17067](https://github.com/apache/spark/pull/17067).  The tests that were not supported earlier are marked with TODO markers and those are now supported with the code changes here.
- Existing unit tests ( hive, catalyst and sql) were run successfully.

Closes #17185 from skambha/colResolution.

Authored-by: Sunitha Kambhampati <skambha@us.ibm.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2018-08-07 21:11:08 +08:00
Gengliang Wang 819c4de45a [SPARK-24772][SQL] Avro: support logical date type
## What changes were proposed in this pull request?

Support Avro logical date type:
https://avro.apache.org/docs/1.8.2/spec.html#Date

## How was this patch tested?

Unit test

Closes #21984 from gengliangwang/avro_date.

Authored-by: Gengliang Wang <gengliang.wang@databricks.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2018-08-07 17:24:25 +08:00
Maxim Gekk 131ca146ed [SPARK-24005][CORE] Remove usage of Scala’s parallel collection
## What changes were proposed in this pull request?

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

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

## How was this patch tested?

A test was added to `ThreadUtilsSuite`

Closes #21913 from MaxGekk/par-map.

Authored-by: Maxim Gekk <maxim.gekk@databricks.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2018-08-07 17:14:30 +08:00
Marco Gaido 88e0c7bbd5 [SPARK-24341][SQL] Support only IN subqueries with the same number of items per row
## What changes were proposed in this pull request?

Using struct types in subqueries with the `IN` clause can generate invalid plans in `RewritePredicateSubquery`. Indeed, we are not handling clearly the cases when the outer value is a struct or the output of the inner subquery is a struct.

The PR aims to make Spark's behavior the same as the one of the other RDBMS - namely Oracle and Postgres behavior were checked. So we consider valid only queries having the same number of fields in the outer value and in the subquery. This means that:

 - `(a, b) IN (select c, d from ...)` is a valid query;
 - `(a, b) IN (select (c, d) from ...)` throws an AnalysisException, as in the subquery we have only one field of type struct while in the outer value we have 2 fields;
 - `a IN (select (c, d) from ...)` - where `a` is a struct - is a valid query.

## How was this patch tested?

Added UT

Closes #21403 from mgaido91/SPARK-24313.

Authored-by: Marco Gaido <marcogaido91@gmail.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2018-08-07 15:43:41 +08:00
Xingbo Jiang 388f5a0635 [SPARK-24817][CORE] Implement BarrierTaskContext.barrier()
## What changes were proposed in this pull request?

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

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

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

Closes #21898 from jiangxb1987/taskcontext.barrier.

Authored-by: Xingbo Jiang <xingbo.jiang@databricks.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2018-08-07 15:06:32 +08:00
Liang-Chi Hsieh 43763629f1 [SPARK-25010][SQL] Rand/Randn should produce different values for each execution in streaming query
## What changes were proposed in this pull request?

Like Uuid in SPARK-24896, Rand and Randn expressions now produce the same results for each execution in streaming query. It doesn't make too much sense for streaming queries. We should make them produce different results as Uuid.

In this change, similar to Uuid, we assign new random seeds to Rand/Randn when returning optimized plan from `IncrementalExecution`.

Note: Different to Uuid, Rand/Randn can be created with initial seed. Because we replace this initial seed at `IncrementalExecution`, it doesn't use the initial seed anymore. For now it seems to me not a big issue for streaming query. But need to confirm with others. cc zsxwing cloud-fan

## How was this patch tested?

Added test.

Closes #21980 from viirya/SPARK-25010.

Authored-by: Liang-Chi Hsieh <viirya@gmail.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2018-08-07 14:28:14 +08:00
Kazuaki Ishizaki 4446a0b0d9 [SPARK-23914][SQL][FOLLOW-UP] refactor ArrayUnion
## What changes were proposed in this pull request?

This PR refactors `ArrayUnion` based on [this suggestion](https://github.com/apache/spark/pull/21103#discussion_r205668821).
1. Generate optimized code for all of the primitive types except `boolean`
1. Generate code using `ArrayBuilder` or `ArrayBuffer`
1. Leave only a generic path in the interpreted path

## How was this patch tested?

Existing tests

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

Closes #21937 from kiszk/SPARK-23914-follow.
2018-08-07 12:07:56 +09:00
DB Tsai 51bee7aca1 [SPARK-25018][INFRA] Use Co-authored-by and Signed-off-by git trailer in merge_spark_pr.py
## What changes were proposed in this pull request?

In [Linux community](https://git.wiki.kernel.org/index.php/CommitMessageConventions), `Co-authored-by` and `Signed-off-by` git trailer have been used for awhile.

Until recently, Github adopted `Co-authored-by` to include the work of co-authors in the profile contributions graph and the repository's statistics. It's a convention for recognizing multiple authors, and can encourage people to collaborate in OSS communities.

Git provides a command line tools to read the metadata to know who commits the code to upstream, but it's not as easy as having `Signed-off-by` as part of the message so developers can find who is the relevant committers who can help with certain part of the codebase easier.

For a single author PR, I purpose to use `Authored-by` and `Signed-off-by`, so the message will look like

```
Authored-by: Author's name <authorexample.com>
Signed-off-by: Committer's name <committerexample.com>
```

For a multi-author PR, I purpose to use `Lead-authored-by:` and `Co-authored-by:` for the lead author and co-authors. The message will look like

```
Lead-authored-by: Lead Author's name <leadauthorexample.com>
Co-authored-by: CoAuthor's name <coauthorexample.com>
Signed-off-by: Committer's name <committerexample.com>
```

It's also useful to include `Reviewed-by:` to give credits to the people who participate on the code reviewing. We can add this in the next iteration.

Closes #21991 from dbtsai/script.

Lead-authored-by: DB Tsai <d_tsai@apple.com>
Co-authored-by: Liang-Chi Hsieh <viirya@gmail.com>
Co-authored-by: Brian Lindblom <blindblom@apple.com>
Co-authored-by: hyukjinkwon <gurwls223@apache.org>
Signed-off-by: hyukjinkwon <gurwls223@apache.org>
2018-08-07 10:31:11 +08:00
Arun Mahadevan 18b6ec1471 [SPARK-24748][SS] Support for reporting custom metrics via StreamingQuery Progress
## What changes were proposed in this pull request?

Currently the Structured Streaming sources and sinks does not have a way to report custom metrics. Providing an option to report custom metrics and making it available via Streaming Query progress can enable sources and sinks to report custom progress information (E.g. the lag metrics for Kafka source).

Similar metrics can be reported for Sinks as well, but would like to get initial feedback before proceeding further.

## How was this patch tested?

New and existing unit tests.

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

Closes #21721 from arunmahadevan/SPARK-24748.

Authored-by: Arun Mahadevan <arunm@apache.org>
Signed-off-by: hyukjinkwon <gurwls223@apache.org>
2018-08-07 10:28:26 +08:00
Jungtaek Lim 6afe6f32ca [SPARK-24637][SS] Add metrics regarding state and watermark to dropwizard metrics
## What changes were proposed in this pull request?

The patch adds metrics regarding state and watermark to dropwizard metrics, so that watermark and state rows/size can be tracked via time-series manner.

## How was this patch tested?

Manually tested with CSV metric sink.

Closes #21622 from HeartSaVioR/SPARK-24637.

Authored-by: Jungtaek Lim <kabhwan@gmail.com>
Signed-off-by: hyukjinkwon <gurwls223@apache.org>
2018-08-07 10:12:22 +08:00
deshanxiao 1076e4f002 [MINOR][DOCS] Fix grammatical error in SortShuffleManager
## What changes were proposed in this pull request?

Fix a grammatical error in the comment of SortShuffleManager.

## How was this patch tested?

N/A

Closes #21956 from deshanxiao/master.

Authored-by: deshanxiao <42019462+deshanxiao@users.noreply.github.com>
Signed-off-by: hyukjinkwon <gurwls223@apache.org>
2018-08-07 09:36:37 +08:00
Marco Gaido 0f3fa2f289 [SPARK-24996][SQL] Use DSL in DeclarativeAggregate
## What changes were proposed in this pull request?

The PR refactors the aggregate expressions which were not using DSL in order to simplify them.

## How was this patch tested?

NA

Author: Marco Gaido <marcogaido91@gmail.com>

Closes #21970 from mgaido91/SPARK-24996.
2018-08-06 19:46:51 -04:00
Kazuaki Ishizaki 408a3ff2c4 [SPARK-25036][SQL] Should compare ExprValue.isNull with LiteralTrue/LiteralFalse
## What changes were proposed in this pull request?

This PR fixes a comparison of `ExprValue.isNull` with `String`. `ExprValue.isNull` should be compared with `LiteralTrue` or `LiteralFalse`.

This causes the following compilation error using scala-2.12 with sbt. In addition, this code may also generate incorrect code in Spark 2.3.

```
/home/ishizaki/Spark/PR/scala212/spark/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/stringExpressions.scala:94: org.apache.spark.sql.catalyst.expressions.codegen.ExprValue and String are unrelated: they will most likely always compare unequal
[error] [warn]         if (eval.isNull != "true") {
[error] [warn]
[error] [warn] /home/ishizaki/Spark/PR/scala212/spark/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/stringExpressions.scala:126: org.apache.spark.sql.catalyst.expressions.codegen.ExprValue and String are unrelated: they will most likely never compare equal
[error] [warn]              if (eval.isNull == "true") {
[error] [warn]
[error] [warn] /home/ishizaki/Spark/PR/scala212/spark/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/stringExpressions.scala:133: org.apache.spark.sql.catalyst.expressions.codegen.ExprValue and String are unrelated: they will most likely never compare equal
[error] [warn]             if (eval.isNull == "true") {
[error] [warn]
[error] [warn] /home/ishizaki/Spark/PR/scala212/spark/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/codegen/GenerateUnsafeProjection.scala:90: org.apache.spark.sql.catalyst.expressions.codegen.ExprValue and String are unrelated: they will most likely never compare equal
[error] [warn]       if (inputs.map(_.isNull).forall(_ == "false")) {
[error] [warn]
```

## How was this patch tested?

Existing UTs

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

Closes #22012 from kiszk/SPARK-25036a.
2018-08-06 19:43:21 -04:00
Jungtaek Lim 87ca7396c7
[SPARK-24161][SS] Enable debug package feature on structured streaming
## What changes were proposed in this pull request?

Currently, debug package has a implicit class "DebugQuery" which matches Dataset to provide debug features on Dataset class. It doesn't work with structured streaming: it requires query is already started, and the information can be retrieved from StreamingQuery, not Dataset. I guess that's why "explain" had to be placed to StreamingQuery whereas it already exists on Dataset.

This patch adds a new implicit class "DebugStreamQuery" which matches StreamingQuery to provide similar debug features on StreamingQuery class.

## How was this patch tested?

Added relevant unit tests.

Author: Jungtaek Lim <kabhwan@gmail.com>

Closes #21222 from HeartSaVioR/SPARK-24161.
2018-08-06 15:23:47 -07:00
Marco Gaido 3c96937c7b [SPARK-24948][SHS] Delegate check access permissions to the file system
## What changes were proposed in this pull request?

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

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

## How was this patch tested?

modified UTs

Author: Marco Gaido <marcogaido91@gmail.com>

Closes #21895 from mgaido91/SPARK-24948.
2018-08-06 14:29:05 -07:00
Dongjoon Hyun 278984d5a5 [SPARK-25019][BUILD] Fix orc dependency to use the same exclusion rules
## What changes were proposed in this pull request?

During upgrading Apache ORC to 1.5.2 ([SPARK-24576](https://issues.apache.org/jira/browse/SPARK-24576)), `sql/core` module overrides the exclusion rules of parent pom file and it causes published `spark-sql_2.1X` artifacts have incomplete exclusion rules ([SPARK-25019](https://issues.apache.org/jira/browse/SPARK-25019)). This PR fixes it by moving the newly added exclusion rule to the parent pom. This also fixes the sbt build hack introduced at that time.

## How was this patch tested?

Pass the existing dependency check and the tests.

Author: Dongjoon Hyun <dongjoon@apache.org>

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

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

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

Closes #21953 from hthuynh2/SPARK_24992.
2018-08-06 13:58:28 -05:00
Kazuaki Ishizaki 1a5e460762 [SPARK-23913][SQL] Add array_intersect function
## What changes were proposed in this pull request?

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

This function returns returns an array of the elements in the intersection of array1 and array2.

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

## How was this patch tested?

Added UTs

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

Closes #21102 from kiszk/SPARK-23913.
2018-08-06 23:27:57 +09:00
Hieu Huynh 35700bb7f2 [SPARK-24981][CORE] ShutdownHook timeout causes job to fail when succeeded when SparkContext stop() not called by user program
**Description**
The issue is described in [SPARK-24981](https://issues.apache.org/jira/browse/SPARK-24981).

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

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

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

Closes #21936 from hthuynh2/SPARK_24981.
2018-08-06 09:01:51 -05:00
Dilip Biswal c1760da5dd [SPARK-25025][SQL] Remove the default value of isAll in INTERSECT/EXCEPT
## What changes were proposed in this pull request?

Having the default value of isAll in the logical plan nodes INTERSECT/EXCEPT could introduce bugs when the callers are not aware of it. This PR removes the default value and makes caller explicitly specify them.

## How was this patch tested?
This is a refactoring change. Existing tests test the functionality already.

Author: Dilip Biswal <dbiswal@us.ibm.com>

Closes #22000 from dilipbiswal/SPARK-25025.
2018-08-06 06:56:36 -04:00
John Zhuge d063e3a478 [SPARK-24940][SQL] Use IntegerLiteral in ResolveCoalesceHints
## What changes were proposed in this pull request?

Follow up to fix an unmerged review comment.

## How was this patch tested?

Unit test ResolveHintsSuite.

Author: John Zhuge <jzhuge@apache.org>

Closes #21998 from jzhuge/SPARK-24940.
2018-08-06 06:41:55 -04:00
Maxim Gekk 64ad7b841d [SPARK-23772][FOLLOW-UP][SQL] Provide an option to ignore column of all null values or empty array during JSON schema inference
## What changes were proposed in this pull request?

The `dropFieldIfAllNull` parameter of the `json` method wasn't set as an option. This PR fixes that.

## How was this patch tested?

I added a test to `sql/test.py`

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

Closes #22002 from MaxGekk/drop-field-if-all-null.
2018-08-06 16:46:55 +08:00
Wenchen Fan ac527b5205 [SPARK-24991][SQL] use InternalRow in DataSourceWriter
## What changes were proposed in this pull request?

A follow up of #21118

Since we use `InternalRow` in the read API of data source v2, we should do the same thing for the write API.

## How was this patch tested?

existing tests.

Author: Wenchen Fan <wenchen@databricks.com>

Closes #21948 from cloud-fan/row-write.
2018-08-06 15:52:01 +08:00
Takuya UESHIN 327bb30075 [SPARK-23911][SQL] Add aggregate function.
## What changes were proposed in this pull request?

This pr adds `aggregate` function which applies a binary operator to an initial state and all elements in the array, and reduces this to a single state. The final state is converted into the final result by applying a finish function.

```sql
> SELECT aggregate(array(1, 2, 3), (acc, x) -> acc + x);
 6
> SELECT aggregate(array(1, 2, 3), (acc, x) -> acc + x, acc -> acc * 10);
 60
```

## How was this patch tested?

Added tests.

Author: Takuya UESHIN <ueshin@databricks.com>

Closes #21982 from ueshin/issues/SPARK-23911/aggregate.
2018-08-05 08:58:35 +09:00
Sean Owen 5f9633dc97 [SPARK-25015][BUILD] Update Hadoop 2.7 to 2.7.7
## What changes were proposed in this pull request?

Update Hadoop 2.7 to 2.7.7 to pull in bug and security fixes.

## How was this patch tested?

Existing tests.

Author: Sean Owen <srowen@gmail.com>

Closes #21987 from srowen/SPARK-25015.
2018-08-04 14:59:13 -05:00
Yuval Itzchakov b7fdf8eb20 [SPARK-24987][SS] - Fix Kafka consumer leak when no new offsets for TopicPartition
## What changes were proposed in this pull request?

This small fix adds a `consumer.release()` call to `KafkaSourceRDD` in the case where we've retrieved offsets from Kafka, but the `fromOffset` is equal to the `lastOffset`, meaning there is no new data to read for a particular topic partition. Up until now, we'd just return an empty iterator without closing the consumer which would cause a FD leak.

If accepted, this pull request should be merged into master as well.

## How was this patch tested?

Haven't ran any specific tests, would love help on how to test methods running inside `RDD.compute`.

Author: Yuval Itzchakov <yuval.itzchakov@clicktale.com>

Closes #21997 from YuvalItzchakov/master.
2018-08-04 14:44:10 -05:00
hyukjinkwon 55e3ae6930 [SPARK-25001][BUILD] Fix miscellaneous build warnings
## What changes were proposed in this pull request?

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

**common**:

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

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

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

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

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

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

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

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

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

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

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

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

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

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

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

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

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

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

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

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

**launcher**:

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

**core**:

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

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

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

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

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

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

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

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

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

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

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

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

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

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

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

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

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

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

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

**sql**:

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

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

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

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

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

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

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

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

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

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

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

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

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

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

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

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

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

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

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

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

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

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

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

**mllib**:

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

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

## How was this patch tested?

Existing tests should cover this.

Author: hyukjinkwon <gurwls223@apache.org>

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

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

## How was this patch tested?

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

Passes scalastyle checks from dev/run-tests

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

Author: Nihar Sheth <niharrsheth@gmail.com>

Closes #21885 from NiharS/usableCores.
2018-08-04 10:27:34 -05:00
Wenchen Fan 684c719cc0 [SPARK-23915][SQL][FOLLOWUP] Add array_except function
## What changes were proposed in this pull request?

simplify the codegen:
1. only do real codegen if the type can be specialized by the hash set
2. change the null handling. Before: track the nullElementIndex, and create a new ArrayData to insert the null in the middle. After: track the nullElementIndex, put a null placeholder in the ArrayBuilder, at the end create ArrayData from ArrayBuilder directly.

## How was this patch tested?

existing tests.

Author: Wenchen Fan <wenchen@databricks.com>

Closes #21966 from cloud-fan/minor2.
2018-08-04 16:35:14 +09:00
Takuya UESHIN 0ecc132d6b [SPARK-23909][SQL] Add filter function.
## What changes were proposed in this pull request?

This pr adds `filter` function which filters the input array using the given predicate.

```sql
> SELECT filter(array(1, 2, 3), x -> x % 2 == 1);
 array(1, 3)
```

## How was this patch tested?

Added tests.

Author: Takuya UESHIN <ueshin@databricks.com>

Closes #21965 from ueshin/issues/SPARK-23909/filter.
2018-08-04 16:08:53 +09:00