Commit graph

20821 commits

Author SHA1 Message Date
liuxian 3d43a9f939 [SPARK-22349] In on-heap mode, when allocating memory from pool,we should fill memory with MEMORY_DEBUG_FILL_CLEAN_VALUE
## What changes were proposed in this pull request?
In on-heap mode, when allocating memory from pool,we should fill memory with `MEMORY_DEBUG_FILL_CLEAN_VALUE`

## How was this patch tested?
added unit tests

Author: liuxian <liu.xian3@zte.com.cn>

Closes #19572 from 10110346/MEMORY_DEBUG.
2017-10-25 21:34:00 +05:30
Liang-Chi Hsieh 1051ebec70 [SPARK-20783][SQL][FOLLOW-UP] Create ColumnVector to abstract existing compressed column
## What changes were proposed in this pull request?

Removed one unused method.

## How was this patch tested?

Existing tests.

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

Closes #19508 from viirya/SPARK-20783-followup.
2017-10-25 16:31:58 +01:00
Sean Owen 6c6950839d [SPARK-22322][CORE] Update FutureAction for compatibility with Scala 2.12 Future
## What changes were proposed in this pull request?

Scala 2.12's `Future` defines two new methods to implement, `transform` and `transformWith`. These can be implemented naturally in Spark's `FutureAction` extension and subclasses, but, only in terms of the new methods that don't exist in Scala 2.11. To support both at the same time, reflection is used to implement these.

## How was this patch tested?

Existing tests.

Author: Sean Owen <sowen@cloudera.com>

Closes #19561 from srowen/SPARK-22322.
2017-10-25 12:51:20 +01:00
Ruben Berenguel Montoro 427359f077 [SPARK-13947][SQL] The error message from using an invalid column reference is not clear
## What changes were proposed in this pull request?

 Rewritten error message for clarity. Added extra information in case of attribute name collision, hinting the user to double-check referencing two different tables

## How was this patch tested?

No functional changes, only final message has changed. It has been tested manually against the situation proposed in the JIRA ticket. Automated tests in repository pass.

This PR is original work from me and I license this work to the Spark project

Author: Ruben Berenguel Montoro <ruben@mostlymaths.net>
Author: Ruben Berenguel Montoro <ruben@dreamattic.com>
Author: Ruben Berenguel <ruben@mostlymaths.net>

Closes #17100 from rberenguel/SPARK-13947-error-message.
2017-10-24 23:02:11 -07:00
Yuming Wang 524abb996a [SPARK-21101][SQL] Catch IllegalStateException when CREATE TEMPORARY FUNCTION
## What changes were proposed in this pull request?

It must `override` [`public StructObjectInspector initialize(ObjectInspector[] argOIs)`](https://github.com/apache/hive/blob/release-2.0.0/ql/src/java/org/apache/hadoop/hive/ql/udf/generic/GenericUDTF.java#L70) when create a UDTF.

If you `override` [`public StructObjectInspector initialize(StructObjectInspector argOIs)`](https://github.com/apache/hive/blob/release-2.0.0/ql/src/java/org/apache/hadoop/hive/ql/udf/generic/GenericUDTF.java#L49), `IllegalStateException` will throw. per: [HIVE-12377](https://issues.apache.org/jira/browse/HIVE-12377).

This PR catch `IllegalStateException` and point user to `override` `public StructObjectInspector initialize(ObjectInspector[] argOIs)`.

## How was this patch tested?

unit tests

Source code and binary jar: [SPARK-21101.zip](https://github.com/apache/spark/files/1123763/SPARK-21101.zip)
These two source code copy from : https://github.com/apache/hive/blob/release-2.0.0/ql/src/java/org/apache/hadoop/hive/ql/udf/generic/GenericUDTFStack.java

Author: Yuming Wang <wgyumg@gmail.com>

Closes #18527 from wangyum/SPARK-21101.
2017-10-24 22:59:46 -07:00
Liang-Chi Hsieh bc1e76632d [SPARK-22348][SQL] The table cache providing ColumnarBatch should also do partition batch pruning
## What changes were proposed in this pull request?

We enable table cache `InMemoryTableScanExec` to provide `ColumnarBatch` now. But the cached batches are retrieved without pruning. In this case, we still need to do partition batch pruning.

## How was this patch tested?

Existing tests.

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

Closes #19569 from viirya/SPARK-22348.
2017-10-25 06:33:44 +01:00
Marco Gaido 3f5ba968c5 [SPARK-22301][SQL] Add rule to Optimizer for In with not-nullable value and empty list
## What changes were proposed in this pull request?

For performance reason, we should resolve in operation on an empty list as false in the optimizations phase, ad discussed in #19522.

## How was this patch tested?
Added UT

cc gatorsmile

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

Closes #19523 from mgaido91/SPARK-22301.
2017-10-24 09:11:52 -07:00
Sean Owen 8beeaed66b [SPARK-21936][SQL][FOLLOW-UP] backward compatibility test framework for HiveExternalCatalog
## What changes were proposed in this pull request?

Adjust Spark download in test to use Apache mirrors and respect its load balancer, and use Spark 2.1.2. This follows on a recent PMC list thread about removing the cloudfront download rather than update it further.

## How was this patch tested?

Existing tests.

Author: Sean Owen <sowen@cloudera.com>

Closes #19564 from srowen/SPARK-21936.2.
2017-10-24 13:56:10 +01:00
Kazuaki Ishizaki c30d5cfc71 [SPARK-20822][SQL] Generate code to directly get value from ColumnVector for table cache
## What changes were proposed in this pull request?

This PR generates the Java code to directly get a value for a column in `ColumnVector` without using an iterator (e.g. at lines 54-69 in the generated code example) for table cache (e.g. `dataframe.cache`). This PR improves runtime performance by eliminating data copy from column-oriented storage to `InternalRow` in a `SpecificColumnarIterator` iterator for primitive type. Another PR will support primitive type array.

Benchmark result: **1.2x**
```
OpenJDK 64-Bit Server VM 1.8.0_121-8u121-b13-0ubuntu1.16.04.2-b13 on Linux 4.4.0-22-generic
Intel(R) Xeon(R) CPU E5-2667 v3  3.20GHz
Int Sum with IntDelta cache:             Best/Avg Time(ms)    Rate(M/s)   Per Row(ns)   Relative
------------------------------------------------------------------------------------------------
InternalRow codegen                            731 /  812         43.0          23.2       1.0X
ColumnVector codegen                           616 /  772         51.0          19.6       1.2X
```
Benchmark program
```
  intSumBenchmark(sqlContext, 1024 * 1024 * 30)
  def intSumBenchmark(sqlContext: SQLContext, values: Int): Unit = {
    import sqlContext.implicits._
    val benchmarkPT = new Benchmark("Int Sum with IntDelta cache", values, 20)
    Seq(("InternalRow", "false"), ("ColumnVector", "true")).foreach {
      case (str, value) =>
        withSQLConf(sqlContext, SQLConf. COLUMN_VECTOR_CODEGEN.key -> value) { // tentatively added for benchmarking
          val dfPassThrough = sqlContext.sparkContext.parallelize(0 to values - 1, 1).toDF().cache()
          dfPassThrough.count()       // force to create df.cache()
          benchmarkPT.addCase(s"$str codegen") { iter =>
            dfPassThrough.agg(sum("value")).collect
          }
          dfPassThrough.unpersist(true)
        }
    }
    benchmarkPT.run()
  }
```

Motivating example
```
val dsInt = spark.range(3).cache
dsInt.count // force to build cache
dsInt.filter(_ > 0).collect
```
Generated code
```
/* 001 */ public Object generate(Object[] references) {
/* 002 */   return new GeneratedIterator(references);
/* 003 */ }
/* 004 */
/* 005 */ final class GeneratedIterator extends org.apache.spark.sql.execution.BufferedRowIterator {
/* 006 */   private Object[] references;
/* 007 */   private scala.collection.Iterator[] inputs;
/* 008 */   private scala.collection.Iterator inmemorytablescan_input;
/* 009 */   private org.apache.spark.sql.execution.metric.SQLMetric inmemorytablescan_numOutputRows;
/* 010 */   private org.apache.spark.sql.execution.metric.SQLMetric inmemorytablescan_scanTime;
/* 011 */   private long inmemorytablescan_scanTime1;
/* 012 */   private org.apache.spark.sql.execution.vectorized.ColumnarBatch inmemorytablescan_batch;
/* 013 */   private int inmemorytablescan_batchIdx;
/* 014 */   private org.apache.spark.sql.execution.vectorized.OnHeapColumnVector inmemorytablescan_colInstance0;
/* 015 */   private UnsafeRow inmemorytablescan_result;
/* 016 */   private org.apache.spark.sql.catalyst.expressions.codegen.BufferHolder inmemorytablescan_holder;
/* 017 */   private org.apache.spark.sql.catalyst.expressions.codegen.UnsafeRowWriter inmemorytablescan_rowWriter;
/* 018 */   private org.apache.spark.sql.execution.metric.SQLMetric filter_numOutputRows;
/* 019 */   private UnsafeRow filter_result;
/* 020 */   private org.apache.spark.sql.catalyst.expressions.codegen.BufferHolder filter_holder;
/* 021 */   private org.apache.spark.sql.catalyst.expressions.codegen.UnsafeRowWriter filter_rowWriter;
/* 022 */
/* 023 */   public GeneratedIterator(Object[] references) {
/* 024 */     this.references = references;
/* 025 */   }
/* 026 */
/* 027 */   public void init(int index, scala.collection.Iterator[] inputs) {
/* 028 */     partitionIndex = index;
/* 029 */     this.inputs = inputs;
/* 030 */     inmemorytablescan_input = inputs[0];
/* 031 */     inmemorytablescan_numOutputRows = (org.apache.spark.sql.execution.metric.SQLMetric) references[0];
/* 032 */     inmemorytablescan_scanTime = (org.apache.spark.sql.execution.metric.SQLMetric) references[1];
/* 033 */     inmemorytablescan_scanTime1 = 0;
/* 034 */     inmemorytablescan_batch = null;
/* 035 */     inmemorytablescan_batchIdx = 0;
/* 036 */     inmemorytablescan_colInstance0 = null;
/* 037 */     inmemorytablescan_result = new UnsafeRow(1);
/* 038 */     inmemorytablescan_holder = new org.apache.spark.sql.catalyst.expressions.codegen.BufferHolder(inmemorytablescan_result, 0);
/* 039 */     inmemorytablescan_rowWriter = new org.apache.spark.sql.catalyst.expressions.codegen.UnsafeRowWriter(inmemorytablescan_holder, 1);
/* 040 */     filter_numOutputRows = (org.apache.spark.sql.execution.metric.SQLMetric) references[2];
/* 041 */     filter_result = new UnsafeRow(1);
/* 042 */     filter_holder = new org.apache.spark.sql.catalyst.expressions.codegen.BufferHolder(filter_result, 0);
/* 043 */     filter_rowWriter = new org.apache.spark.sql.catalyst.expressions.codegen.UnsafeRowWriter(filter_holder, 1);
/* 044 */
/* 045 */   }
/* 046 */
/* 047 */   protected void processNext() throws java.io.IOException {
/* 048 */     if (inmemorytablescan_batch == null) {
/* 049 */       inmemorytablescan_nextBatch();
/* 050 */     }
/* 051 */     while (inmemorytablescan_batch != null) {
/* 052 */       int inmemorytablescan_numRows = inmemorytablescan_batch.numRows();
/* 053 */       int inmemorytablescan_localEnd = inmemorytablescan_numRows - inmemorytablescan_batchIdx;
/* 054 */       for (int inmemorytablescan_localIdx = 0; inmemorytablescan_localIdx < inmemorytablescan_localEnd; inmemorytablescan_localIdx++) {
/* 055 */         int inmemorytablescan_rowIdx = inmemorytablescan_batchIdx + inmemorytablescan_localIdx;
/* 056 */         int inmemorytablescan_value = inmemorytablescan_colInstance0.getInt(inmemorytablescan_rowIdx);
/* 057 */
/* 058 */         boolean filter_isNull = false;
/* 059 */
/* 060 */         boolean filter_value = false;
/* 061 */         filter_value = inmemorytablescan_value > 1;
/* 062 */         if (!filter_value) continue;
/* 063 */
/* 064 */         filter_numOutputRows.add(1);
/* 065 */
/* 066 */         filter_rowWriter.write(0, inmemorytablescan_value);
/* 067 */         append(filter_result);
/* 068 */         if (shouldStop()) { inmemorytablescan_batchIdx = inmemorytablescan_rowIdx + 1; return; }
/* 069 */       }
/* 070 */       inmemorytablescan_batchIdx = inmemorytablescan_numRows;
/* 071 */       inmemorytablescan_batch = null;
/* 072 */       inmemorytablescan_nextBatch();
/* 073 */     }
/* 074 */     inmemorytablescan_scanTime.add(inmemorytablescan_scanTime1 / (1000 * 1000));
/* 075 */     inmemorytablescan_scanTime1 = 0;
/* 076 */   }
/* 077 */
/* 078 */   private void inmemorytablescan_nextBatch() throws java.io.IOException {
/* 079 */     long getBatchStart = System.nanoTime();
/* 080 */     if (inmemorytablescan_input.hasNext()) {
/* 081 */       org.apache.spark.sql.execution.columnar.CachedBatch inmemorytablescan_cachedBatch = (org.apache.spark.sql.execution.columnar.CachedBatch)inmemorytablescan_input.next();
/* 082 */       inmemorytablescan_batch = org.apache.spark.sql.execution.columnar.InMemoryRelation$.MODULE$.createColumn(inmemorytablescan_cachedBatch);
/* 083 */
/* 084 */       inmemorytablescan_numOutputRows.add(inmemorytablescan_batch.numRows());
/* 085 */       inmemorytablescan_batchIdx = 0;
/* 086 */       inmemorytablescan_colInstance0 = (org.apache.spark.sql.execution.vectorized.OnHeapColumnVector) inmemorytablescan_batch.column(0); org.apache.spark.sql.execution.columnar.ColumnAccessor$.MODULE$.decompress(inmemorytablescan_cachedBatch.buffers()[0], (org.apache.spark.sql.execution.vectorized.WritableColumnVector) inmemorytablescan_colInstance0, org.apache.spark.sql.types.DataTypes.IntegerType, inmemorytablescan_cachedBatch.numRows());
/* 087 */
/* 088 */     }
/* 089 */     inmemorytablescan_scanTime1 += System.nanoTime() - getBatchStart;
/* 090 */   }
/* 091 */ }
```

## How was this patch tested?

Add test cases into `DataFrameTungstenSuite` and `WholeStageCodegenSuite`

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

Closes #18747 from kiszk/SPARK-20822a.
2017-10-24 08:46:22 +01:00
hyukjinkwon d9798c834f [SPARK-22313][PYTHON] Mark/print deprecation warnings as DeprecationWarning for deprecated APIs
## What changes were proposed in this pull request?

This PR proposes to mark the existing warnings as `DeprecationWarning` and print out warnings for deprecated functions.

This could be actually useful for Spark app developers. I use (old) PyCharm and this IDE can detect this specific `DeprecationWarning` in some cases:

**Before**

<img src="https://user-images.githubusercontent.com/6477701/31762664-df68d9f8-b4f6-11e7-8773-f0468f70a2cc.png" height="45" />

**After**

<img src="https://user-images.githubusercontent.com/6477701/31762662-de4d6868-b4f6-11e7-98dc-3c8446a0c28a.png" height="70" />

For console usage, `DeprecationWarning` is usually disabled (see https://docs.python.org/2/library/warnings.html#warning-categories and https://docs.python.org/3/library/warnings.html#warning-categories):

```
>>> import warnings
>>> filter(lambda f: f[2] == DeprecationWarning, warnings.filters)
[('ignore', <_sre.SRE_Pattern object at 0x10ba58c00>, <type 'exceptions.DeprecationWarning'>, <_sre.SRE_Pattern object at 0x10bb04138>, 0), ('ignore', None, <type 'exceptions.DeprecationWarning'>, None, 0)]
```

so, it won't actually mess up the terminal much unless it is intended.

If this is intendedly enabled, it'd should as below:

```
>>> import warnings
>>> warnings.simplefilter('always', DeprecationWarning)
>>>
>>> from pyspark.sql import functions
>>> functions.approxCountDistinct("a")
.../spark/python/pyspark/sql/functions.py:232: DeprecationWarning: Deprecated in 2.1, use approx_count_distinct instead.
  "Deprecated in 2.1, use approx_count_distinct instead.", DeprecationWarning)
...
```

These instances were found by:

```
cd python/pyspark
grep -r "Deprecated" .
grep -r "deprecated" .
grep -r "deprecate" .
```

## How was this patch tested?

Manually tested.

Author: hyukjinkwon <gurwls223@gmail.com>

Closes #19535 from HyukjinKwon/deprecated-warning.
2017-10-24 12:44:47 +09:00
Dongjoon Hyun 884d4f95f7 [SPARK-21912][SQL][FOLLOW-UP] ORC/Parquet table should not create invalid column names
## What changes were proposed in this pull request?

During [SPARK-21912](https://issues.apache.org/jira/browse/SPARK-21912), we skipped testing 'ADD COLUMNS' on ORC tables due to ORC limitation. Since [SPARK-21929](https://issues.apache.org/jira/browse/SPARK-21929) is resolved now, we can test both `ORC` and `PARQUET` completely.

## How was this patch tested?

Pass the updated test case.

Author: Dongjoon Hyun <dongjoon@apache.org>

Closes #19562 from dongjoon-hyun/SPARK-21912-2.
2017-10-23 17:21:49 -07:00
Zhenhua Wang f6290aea24 [SPARK-22285][SQL] Change implementation of ApproxCountDistinctForIntervals to TypedImperativeAggregate
## What changes were proposed in this pull request?

The current implementation of `ApproxCountDistinctForIntervals` is `ImperativeAggregate`. The number of `aggBufferAttributes` is the number of total words in the hllppHelper array. Each hllppHelper has 52 words by default relativeSD.

Since this aggregate function is used in equi-height histogram generation, and the number of buckets in histogram is usually hundreds, the number of `aggBufferAttributes` can easily reach tens of thousands or even more.

This leads to a huge method in codegen and causes error:
```
org.codehaus.janino.JaninoRuntimeException: Code of method "apply(Lorg/apache/spark/sql/catalyst/InternalRow;)Lorg/apache/spark/sql/catalyst/expressions/UnsafeRow;" of class "org.apache.spark.sql.catalyst.expressions.GeneratedClass$SpecificUnsafeProjection" grows beyond 64 KB.
```
Besides, huge generated methods also result in performance regression.

In this PR, we change its implementation to `TypedImperativeAggregate`. After the fix, `ApproxCountDistinctForIntervals` can deal with more than thousands endpoints without throwing codegen error, and improve performance from `20 sec` to `2 sec` in a test case of 500 endpoints.

## How was this patch tested?

Test by an added test case and existing tests.

Author: Zhenhua Wang <wangzhenhua@huawei.com>

Closes #19506 from wzhfy/change_forIntervals_typedAgg.
2017-10-23 23:02:36 +01:00
Kohki Nishio 5a5b6b7851 [SPARK-22303][SQL] Handle Oracle specific jdbc types in OracleDialect
TIMESTAMP (-101), BINARY_DOUBLE (101) and BINARY_FLOAT (100) are handled in OracleDialect

## What changes were proposed in this pull request?

When a oracle table contains columns whose type is BINARY_FLOAT or BINARY_DOUBLE, spark sql fails to load a table with SQLException

```
java.sql.SQLException: Unsupported type 101
 at org.apache.spark.sql.execution.datasources.jdbc.JdbcUtils$.org$apache$spark$sql$execution$datasources$jdbc$JdbcUtils$$getCatalystType(JdbcUtils.scala:235)
 at org.apache.spark.sql.execution.datasources.jdbc.JdbcUtils$$anonfun$8.apply(JdbcUtils.scala:292)
 at org.apache.spark.sql.execution.datasources.jdbc.JdbcUtils$$anonfun$8.apply(JdbcUtils.scala:292)
 at scala.Option.getOrElse(Option.scala:121)
 at org.apache.spark.sql.execution.datasources.jdbc.JdbcUtils$.getSchema(JdbcUtils.scala:291)
 at org.apache.spark.sql.execution.datasources.jdbc.JDBCRDD$.resolveTable(JDBCRDD.scala:64)
 at org.apache.spark.sql.execution.datasources.jdbc.JDBCRelation.<init>(JDBCRelation.scala:113)
 at org.apache.spark.sql.execution.datasources.jdbc.JdbcRelationProvider.createRelation(JdbcRelationProvider.scala:47)
 at org.apache.spark.sql.execution.datasources.DataSource.resolveRelation(DataSource.scala:306)
 at org.apache.spark.sql.DataFrameReader.load(DataFrameReader.scala:178)
 at org.apache.spark.sql.DataFrameReader.load(DataFrameReader.scala:146)
```

## How was this patch tested?

I updated a UT which covers type conversion test for types (-101, 100, 101), on top of that I tested this change against actual table with those columns and it was able to read and write to the table.

Author: Kohki Nishio <taroplus@me.com>

Closes #19548 from taroplus/oracle_sql_types_101.
2017-10-23 09:55:46 -07:00
Steven Rand 57accf6e39 [SPARK-22319][CORE] call loginUserFromKeytab before accessing hdfs
In `SparkSubmit`, call `loginUserFromKeytab` before attempting to make RPC calls to the NameNode.

I manually tested this patch by:

1. Confirming that my Spark application failed to launch with the error reported in https://issues.apache.org/jira/browse/SPARK-22319.
2. Applying this patch and confirming that the app no longer fails to launch, even when I have not manually run `kinit` on the host.

Presumably we also want integration tests for secure clusters so that we catch this sort of thing. I'm happy to take a shot at this if it's feasible and someone can point me in the right direction.

Author: Steven Rand <srand@palantir.com>

Closes #19540 from sjrand/SPARK-22319.

Change-Id: Ic306bfe7181107fbcf92f61d75856afcb5b6f761
2017-10-23 09:43:45 +08:00
Dongjoon Hyun ca2a780e7c [SPARK-21929][SQL] Support ALTER TABLE table_name ADD COLUMNS(..) for ORC data source
## What changes were proposed in this pull request?

When [SPARK-19261](https://issues.apache.org/jira/browse/SPARK-19261) implements `ALTER TABLE ADD COLUMNS`, ORC data source is omitted due to SPARK-14387, SPARK-16628, and SPARK-18355. Now, those issues are fixed and Spark 2.3 is [using Spark schema to read ORC table instead of ORC file schema](e6e36004af). This PR enables `ALTER TABLE ADD COLUMNS` for ORC data source.

## How was this patch tested?

Pass the updated and added test cases.

Author: Dongjoon Hyun <dongjoon@apache.org>

Closes #19545 from dongjoon-hyun/SPARK-21929.
2017-10-21 18:01:45 -07:00
hyukjinkwon ff8de99a1c [SPARK-22302][INFRA] Remove manual backports for subprocess and print explicit message for < Python 2.7
## What changes were proposed in this pull request?

Seems there was a mistake - missing import for `subprocess.call`, while refactoring this script a long ago, which should be used for backports of some missing functions in `subprocess`, specifically in < Python 2.7.

Reproduction is:

```
cd dev && python2.6
```

```
>>> from sparktestsupport import shellutils
>>> shellutils.subprocess_check_call("ls")
Traceback (most recent call last):
  File "<stdin>", line 1, in <module>
  File "sparktestsupport/shellutils.py", line 46, in subprocess_check_call
    retcode = call(*popenargs, **kwargs)
NameError: global name 'call' is not defined
```

For Jenkins logs, please see https://amplab.cs.berkeley.edu/jenkins/job/NewSparkPullRequestBuilder/3950/console

Since we dropped the Python 2.6.x support, looks better we remove those workarounds and print out explicit error messages in order to reduce the efforts to find out the root causes for such cases, for example, `https://github.com/apache/spark/pull/19513#issuecomment-337406734`.

## How was this patch tested?

Manually tested:

```
./dev/run-tests
```

```
Python versions prior to 2.7 are not supported.
```

```
./dev/run-tests-jenkins
```

```
Python versions prior to 2.7 are not supported.
```

Author: hyukjinkwon <gurwls223@gmail.com>

Closes #19524 from HyukjinKwon/SPARK-22302.
2017-10-22 02:22:35 +09:00
gatorsmile a763607e4f [SPARK-21055][SQL][FOLLOW-UP] replace grouping__id with grouping_id()
## What changes were proposed in this pull request?
Simplifies the test cases that were added in the PR https://github.com/apache/spark/pull/18270.

## How was this patch tested?
N/A

Author: gatorsmile <gatorsmile@gmail.com>

Closes #19546 from gatorsmile/backportSPARK-21055.
2017-10-21 10:07:31 -07:00
gatorsmile d8cada8d1d [SPARK-20331][SQL][FOLLOW-UP] Add a SQLConf for enhanced Hive partition pruning predicate pushdown
## What changes were proposed in this pull request?
This is a follow-up PR of https://github.com/apache/spark/pull/17633.

This PR is to add a conf `spark.sql.hive.advancedPartitionPredicatePushdown.enabled`, which can be used to turn the enhancement off.

## How was this patch tested?
Add a test case

Author: gatorsmile <gatorsmile@gmail.com>

Closes #19547 from gatorsmile/Spark20331FollowUp.
2017-10-21 10:05:45 -07:00
Zhenhua Wang d9f286d261 [SPARK-22326][SQL] Remove unnecessary hashCode and equals methods
## What changes were proposed in this pull request?

Plan equality should be computed by `canonicalized`, so we can remove unnecessary `hashCode` and `equals` methods.

## How was this patch tested?

Existing tests.

Author: Zhenhua Wang <wangzhenhua@huawei.com>

Closes #19539 from wzhfy/remove_equals.
2017-10-20 20:58:55 -07:00
Takuya UESHIN b8624b06e5 [SPARK-20396][SQL][PYSPARK][FOLLOW-UP] groupby().apply() with pandas udf
## What changes were proposed in this pull request?

This is a follow-up of #18732.
This pr modifies `GroupedData.apply()` method to convert pandas udf to grouped udf implicitly.

## How was this patch tested?

Exisiting tests.

Author: Takuya UESHIN <ueshin@databricks.com>

Closes #19517 from ueshin/issues/SPARK-20396/fup2.
2017-10-20 12:44:30 -07:00
Marcelo Vanzin 568763bafb [INFRA] Close stale PRs.
Closes #19541
Closes #19542
2017-10-20 12:32:45 -07:00
CenYuhai 16c9cc68c5 [SPARK-21055][SQL] replace grouping__id with grouping_id()
## What changes were proposed in this pull request?
spark does not support grouping__id, it has grouping_id() instead.
But it is not convenient for hive user to change to spark-sql
so this pr is to replace grouping__id with grouping_id()
hive user need not to alter their scripts

## How was this patch tested?

test with SQLQuerySuite.scala

Author: CenYuhai <yuhai.cen@ele.me>

Closes #18270 from cenyuhai/SPARK-21055.
2017-10-20 09:27:39 -07:00
guoxiaolong e2fea8cd60 [CORE][DOC] Add event log conf.
## What changes were proposed in this pull request?

Event Log Server has a total of five configuration parameters, and now the description of the other two configuration parameters on the doc, user-friendly access and use.

## How was this patch tested?

manual tests

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

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

Closes #19242 from guoxiaolongzte/addEventLogConf.
2017-10-20 09:43:46 +01:00
Zheng RuiFeng 673876b7ea [SPARK-22309][ML] Remove unused param in LDAModel.getTopicDistributionMethod
## What changes were proposed in this pull request?
Remove unused param in `LDAModel.getTopicDistributionMethod`

## How was this patch tested?
existing tests

Author: Zheng RuiFeng <ruifengz@foxmail.com>

Closes #19530 from zhengruifeng/lda_bc.
2017-10-20 08:28:05 +01:00
Eric Perry b84f61cd79 [SQL] Mark strategies with override for clarity.
## What changes were proposed in this pull request?

This is a very trivial PR, simply marking `strategies` in `SparkPlanner` with the `override` keyword for clarity since it is overriding `strategies` in `QueryPlanner` two levels up in the class hierarchy. I was reading through the code to learn a bit and got stuck on this fact for a little while, so I figured this may be helpful so that another developer new to the project doesn't get stuck where I was.

I did not make a JIRA ticket for this because it is so trivial, but I'm happy to do so to adhere to the contribution guidelines if required.

## How was this patch tested?

(Please explain how this patch was tested. E.g. unit tests, integration tests, manual tests)
(If this patch involves UI changes, please attach a screenshot; otherwise, remove this)

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

Author: Eric Perry <eric@ericjperry.com>

Closes #19537 from ericjperry/override-strategies.
2017-10-19 23:57:41 -07:00
Wenchen Fan b034f2565f [SPARK-22026][SQL] data source v2 write path
## What changes were proposed in this pull request?

A working prototype for data source v2 write path.

The writing framework is similar to the reading framework. i.e. `WriteSupport` -> `DataSourceV2Writer` -> `DataWriterFactory` -> `DataWriter`.

Similar to the `FileCommitPotocol`, the writing API has job and task level commit/abort to support the transaction.

## How was this patch tested?

new tests

Author: Wenchen Fan <wenchen@databricks.com>

Closes #19269 from cloud-fan/data-source-v2-write.
2017-10-19 20:24:51 -07:00
Andrew Ash 7fae7995ba [SPARK-22268][BUILD] Fix lint-java
## What changes were proposed in this pull request?

Fix java style issues

## How was this patch tested?

Run `./dev/lint-java` locally since it's not run on Jenkins

Author: Andrew Ash <andrew@andrewash.com>

Closes #19486 from ash211/aash/fix-lint-java.
2017-10-20 09:40:00 +09:00
krishna-pandey 5a07aca4d4 [SPARK-22188][CORE] Adding security headers for preventing XSS, MitM and MIME sniffing
## What changes were proposed in this pull request?

The HTTP Strict-Transport-Security response header (often abbreviated as HSTS) is a security feature that lets a web site tell browsers that it should only be communicated with using HTTPS, instead of using HTTP.

Note: The Strict-Transport-Security header is ignored by the browser when your site is accessed using HTTP; this is because an attacker may intercept HTTP connections and inject the header or remove it. When your site is accessed over HTTPS with no certificate errors, the browser knows your site is HTTPS capable and will honor the Strict-Transport-Security header.

The HTTP X-XSS-Protection response header is a feature of Internet Explorer, Chrome and Safari that stops pages from loading when they detect reflected cross-site scripting (XSS) attacks.

The HTTP X-Content-Type-Options response header is used to protect against MIME sniffing vulnerabilities.

## How was this patch tested?
Checked on my system locally.

<img width="750" alt="screen shot 2017-10-03 at 6 49 20 pm" src="https://user-images.githubusercontent.com/6433184/31127234-eadf7c0c-a86b-11e7-8e5d-f6ea3f97b210.png">

Author: krishna-pandey <krish.pandey21@gmail.com>
Author: Krishna Pandey <krish.pandey21@gmail.com>

Closes #19419 from krishna-pandey/SPARK-22188.
2017-10-19 08:33:14 +01:00
Marcelo Vanzin dc2714da50 [SPARK-22290][CORE] Avoid creating Hive delegation tokens when not necessary.
Hive delegation tokens are only needed when the Spark driver has no access
to the kerberos TGT. That happens only in two situations:

- when using a proxy user
- when using cluster mode without a keytab

This change modifies the Hive provider so that it only generates delegation
tokens in those situations, and tweaks the YARN AM so that it makes the proper
user visible to the Hive code when running with keytabs, so that the TGT
can be used instead of a delegation token.

The effect of this change is that now it's possible to initialize multiple,
non-concurrent SparkContext instances in the same JVM. Before, the second
invocation would fail to fetch a new Hive delegation token, which then could
make the second (or third or...) application fail once the token expired.
With this change, the TGT will be used to authenticate to the HMS instead.

This change also avoids polluting the current logged in user's credentials
when launching applications. The credentials are copied only when running
applications as a proxy user. This makes it possible to implement SPARK-11035
later, where multiple threads might be launching applications, and each app
should have its own set of credentials.

Tested by verifying HDFS and Hive access in following scenarios:
- client and cluster mode
- client and cluster mode with proxy user
- client and cluster mode with principal / keytab
- long-running cluster app with principal / keytab
- pyspark app that creates (and stops) multiple SparkContext instances
  through its lifetime

Author: Marcelo Vanzin <vanzin@cloudera.com>

Closes #19509 from vanzin/SPARK-22290.
2017-10-19 14:56:48 +08:00
Dongjoon Hyun 6f1d0dea1c [SPARK-22300][BUILD] Update ORC to 1.4.1
## What changes were proposed in this pull request?

Apache ORC 1.4.1 is released yesterday.
- https://orc.apache.org/news/2017/10/16/ORC-1.4.1/

Like ORC-233 (Allow `orc.include.columns` to be empty), there are several important fixes.
This PR updates Apache ORC dependency to use the latest one, 1.4.1.

## How was this patch tested?

Pass the Jenkins.

Author: Dongjoon Hyun <dongjoon@apache.org>

Closes #19521 from dongjoon-hyun/SPARK-22300.
2017-10-19 13:30:55 +08:00
Valeriy Avanesov 52facb0062 [SPARK-14371][MLLIB] OnlineLDAOptimizer should not collect stats for each doc in mini-batch to driver
Hi,

# What changes were proposed in this pull request?

as it was proposed by jkbradley , ```gammat``` are not collected to the driver anymore.

# How was this patch tested?
existing test suite.

Author: Valeriy Avanesov <avanesov@wias-berlin.de>
Author: Valeriy Avanesov <acopich@gmail.com>

Closes #18924 from akopich/master.
2017-10-18 10:46:46 -07:00
Marco Gaido 1f25d8683a [SPARK-22249][FOLLOWUP][SQL] Check if list of value for IN is empty in the optimizer
## What changes were proposed in this pull request?

This PR addresses the comments by gatorsmile on [the previous PR](https://github.com/apache/spark/pull/19494).

## How was this patch tested?

Previous UT and added UT.

Author: Marco Gaido <marcogaido91@gmail.com>

Closes #19522 from mgaido91/SPARK-22249_FOLLOWUP.
2017-10-18 09:14:46 -07:00
maryannxue 72561ecf4b [SPARK-22266][SQL] The same aggregate function was evaluated multiple times
## What changes were proposed in this pull request?

To let the same aggregate function that appear multiple times in an Aggregate be evaluated only once, we need to deduplicate the aggregate expressions. The original code was trying to use a "distinct" call to get a set of aggregate expressions, but did not work, since the "distinct" did not compare semantic equality. And even if it did, further work should be done in result expression rewriting.
In this PR, I changed the "set" to a map mapping the semantic identity of a aggregate expression to itself. Thus, later on, when rewriting result expressions (i.e., output expressions), the aggregate expression reference can be fixed.

## How was this patch tested?

Added a new test in SQLQuerySuite

Author: maryannxue <maryann.xue@gmail.com>

Closes #19488 from maryannxue/spark-22266.
2017-10-18 20:59:40 +08:00
Tathagata Das f3137feecd [SPARK-22278][SS] Expose current event time watermark and current processing time in GroupState
## What changes were proposed in this pull request?

Complex state-updating and/or timeout-handling logic in mapGroupsWithState functions may require taking decisions based on the current event-time watermark and/or processing time. Currently, you can use the SQL function `current_timestamp` to get the current processing time, but it needs to be passed inserted in every row with a select, and then passed through the encoder, which isn't efficient. Furthermore, there is no way to get the current watermark.

This PR exposes both of them through the GroupState API.
Additionally, it also cleans up some of the GroupState docs.

## How was this patch tested?

New unit tests

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

Closes #19495 from tdas/SPARK-22278.
2017-10-17 20:09:12 -07:00
Michael Mior 1437e344ec [SPARK-22050][CORE] Allow BlockUpdated events to be optionally logged to the event log
## What changes were proposed in this pull request?

I see that block updates are not logged to the event log.
This makes sense as a default for performance reasons.
However, I find it helpful when trying to get a better understanding of caching for a job to be able to log these updates.
This PR adds a configuration setting `spark.eventLog.blockUpdates` (defaulting to false) which allows block updates to be recorded in the log.
This contribution is original work which is licensed to the Apache Spark project.

## How was this patch tested?

Current and additional unit tests.

Author: Michael Mior <mmior@uwaterloo.ca>

Closes #19263 from michaelmior/log-block-updates.
2017-10-17 14:30:52 -07:00
Huaxin Gao 28f9f3f225 [SPARK-22271][SQL] mean overflows and returns null for some decimal variables
## What changes were proposed in this pull request?

In Average.scala, it has
```
  override lazy val evaluateExpression = child.dataType match {
    case DecimalType.Fixed(p, s) =>
      // increase the precision and scale to prevent precision loss
      val dt = DecimalType.bounded(p + 14, s + 4)
      Cast(Cast(sum, dt) / Cast(count, dt), resultType)
    case _ =>
      Cast(sum, resultType) / Cast(count, resultType)
  }

  def setChild (newchild: Expression) = {
    child = newchild
  }

```
It is possible that  Cast(count, dt), resultType) will make the precision of the decimal number bigger than 38, and this causes over flow.  Since count is an integer and doesn't need a scale, I will cast it using DecimalType.bounded(38,0)
## How was this patch tested?
In DataFrameSuite, I will add a test case.

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

Author: Huaxin Gao <huaxing@us.ibm.com>

Closes #19496 from huaxingao/spark-22271.
2017-10-17 12:50:41 -07:00
Jose Torres 75d666b95a [SPARK-22136][SS] Evaluate one-sided conditions early in stream-stream joins.
## What changes were proposed in this pull request?

Evaluate one-sided conditions early in stream-stream joins.

This is in addition to normal filter pushdown, because integrating it with the join logic allows it to take place in outer join scenarios. This means that rows which can never satisfy the join condition won't clog up the state.

## How was this patch tested?
new unit tests

Author: Jose Torres <jose@databricks.com>

Closes #19452 from joseph-torres/SPARK-22136.
2017-10-17 12:26:53 -07:00
jerryshao e1960c3d6f [SPARK-22062][CORE] Spill large block to disk in BlockManager's remote fetch to avoid OOM
## What changes were proposed in this pull request?

In the current BlockManager's `getRemoteBytes`, it will call `BlockTransferService#fetchBlockSync` to get remote block. In the `fetchBlockSync`, Spark will allocate a temporary `ByteBuffer` to store the whole fetched block. This will potentially lead to OOM if block size is too big or several blocks are fetched simultaneously in this executor.

So here leveraging the idea of shuffle fetch, to spill the large block to local disk before consumed by upstream code. The behavior is controlled by newly added configuration, if block size is smaller than the threshold, then this block will be persisted in memory; otherwise it will first spill to disk, and then read from disk file.

To achieve this feature, what I did is:

1. Rename `TempShuffleFileManager` to `TempFileManager`, since now it is not only used by shuffle.
2. Add a new `TempFileManager` to manage the files of fetched remote blocks, the files are tracked by weak reference, will be deleted when no use at all.

## How was this patch tested?

This was tested by adding UT, also manual verification in local test to perform GC to clean the files.

Author: jerryshao <sshao@hortonworks.com>

Closes #19476 from jerryshao/SPARK-22062.
2017-10-17 22:54:38 +08:00
Kent Yao 99e32f8ba5 [SPARK-22224][SQL] Override toString of KeyValue/Relational-GroupedDataset
## What changes were proposed in this pull request?
#### before

```scala
scala> val words = spark.read.textFile("README.md").flatMap(_.split(" "))
words: org.apache.spark.sql.Dataset[String] = [value: string]

scala> val grouped = words.groupByKey(identity)
grouped: org.apache.spark.sql.KeyValueGroupedDataset[String,String] = org.apache.spark.sql.KeyValueGroupedDataset65214862
```
#### after
```scala
scala> val words = spark.read.textFile("README.md").flatMap(_.split(" "))
words: org.apache.spark.sql.Dataset[String] = [value: string]

scala> val grouped = words.groupByKey(identity)
grouped: org.apache.spark.sql.KeyValueGroupedDataset[String,String] = [key: [value: string], value: [value: string]]
```

## How was this patch tested?
existing ut

cc gatorsmile cloud-fan

Author: Kent Yao <yaooqinn@hotmail.com>

Closes #19363 from yaooqinn/minor-dataset-tostring.
2017-10-17 17:58:45 +08:00
Marco Gaido 8148f19ca1 [SPARK-22249][SQL] isin with empty list throws exception on cached DataFrame
## What changes were proposed in this pull request?

As pointed out in the JIRA, there is a bug which causes an exception to be thrown if `isin` is called with an empty list on a cached DataFrame. The PR fixes it.

## How was this patch tested?

Added UT.

Author: Marco Gaido <marcogaido91@gmail.com>

Closes #19494 from mgaido91/SPARK-22249.
2017-10-17 09:41:23 +02:00
Ben Barnard e66cabb021 [SPARK-20992][SCHEDULER] Add links in documentation to Nomad integration.
## What changes were proposed in this pull request?

Adds links to the fork that provides integration with Nomad, in the same places the k8s integration is linked to.

## How was this patch tested?

I clicked on the links to make sure they're correct ;)

Author: Ben Barnard <barnardb@gmail.com>

Closes #19354 from barnardb/link-to-nomad-integration.
2017-10-17 09:36:09 +02:00
Dongjoon Hyun c09a2a76b5 [SPARK-22280][SQL][TEST] Improve StatisticsSuite to test convertMetastore properly
## What changes were proposed in this pull request?

This PR aims to improve **StatisticsSuite** to test `convertMetastore` configuration properly. Currently, some test logic in `test statistics of LogicalRelation converted from Hive serde tables` depends on the default configuration. New test case is shorter and covers both(true/false) cases explicitly.

This test case was previously modified by SPARK-17410 and SPARK-17284 in Spark 2.3.0.
- a2460be9c3 (diff-1c464c86b68c2d0b07e73b7354e74ce7R443)

## How was this patch tested?

Pass the Jenkins with the improved test case.

Author: Dongjoon Hyun <dongjoon@apache.org>

Closes #19500 from dongjoon-hyun/SPARK-22280.
2017-10-16 16:16:34 -07:00
Dongjoon Hyun 561505e2fc [SPARK-22282][SQL] Rename OrcRelation to OrcFileFormat and remove ORC_COMPRESSION
## What changes were proposed in this pull request?

This PR aims to
- Rename `OrcRelation` to `OrcFileFormat` object.
- Replace `OrcRelation.ORC_COMPRESSION` with `org.apache.orc.OrcConf.COMPRESS`. Since [SPARK-21422](https://issues.apache.org/jira/browse/SPARK-21422), we can use `OrcConf.COMPRESS` instead of Hive's.

```scala
// The references of Hive's classes will be minimized.
val ORC_COMPRESSION = "orc.compress"
```

## How was this patch tested?

Pass the Jenkins with the existing and updated test cases.

Author: Dongjoon Hyun <dongjoon@apache.org>

Closes #19502 from dongjoon-hyun/SPARK-22282.
2017-10-16 11:27:08 -07:00
Xingbo Jiang 0fa10666cf [SPARK-22233][CORE][FOLLOW-UP] Allow user to filter out empty split in HadoopRDD
## What changes were proposed in this pull request?

Update the config `spark.files.ignoreEmptySplits`, rename it and make it internal.

This is followup of #19464

## How was this patch tested?

Exsiting tests.

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

Closes #19504 from jiangxb1987/partitionsplit.
2017-10-16 22:15:50 +08:00
Liang-Chi Hsieh 0ae96495de [SPARK-22223][SQL] ObjectHashAggregate should not introduce unnecessary shuffle
## What changes were proposed in this pull request?

`ObjectHashAggregateExec` should override `outputPartitioning` in order to avoid unnecessary shuffle.

## How was this patch tested?

Added Jenkins test.

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

Closes #19501 from viirya/SPARK-22223.
2017-10-16 13:37:58 +08:00
Mridul Muralidharan 13c1559587 [SPARK-21549][CORE] Respect OutputFormats with no/invalid output directory provided
## What changes were proposed in this pull request?

PR #19294 added support for null's - but spark 2.1 handled other error cases where path argument can be invalid.
Namely:

* empty string
* URI parse exception while creating Path

This is resubmission of PR #19487, which I messed up while updating my repo.

## How was this patch tested?

Enhanced test to cover new support added.

Author: Mridul Muralidharan <mridul@gmail.com>

Closes #19497 from mridulm/master.
2017-10-15 18:40:53 -07:00
Burak Yavuz e8547ffb49 [SPARK-22238] Fix plan resolution bug caused by EnsureStatefulOpPartitioning
## What changes were proposed in this pull request?

In EnsureStatefulOpPartitioning, we check that the inputRDD to a SparkPlan has the expected partitioning for Streaming Stateful Operators. The problem is that we are not allowed to access this information during planning.
The reason we added that check was because CoalesceExec could actually create RDDs with 0 partitions. We should fix it such that when CoalesceExec says that there is a SinglePartition, there is in fact an inputRDD of 1 partition instead of 0 partitions.

## How was this patch tested?

Regression test in StreamingQuerySuite

Author: Burak Yavuz <brkyvz@gmail.com>

Closes #19467 from brkyvz/stateful-op.
2017-10-14 17:39:15 -07:00
liulijia 014dc84712 [SPARK-22233][CORE] Allow user to filter out empty split in HadoopRDD
## What changes were proposed in this pull request?
Add a flag spark.files.ignoreEmptySplits. When true, methods like that use HadoopRDD and NewHadoopRDD such as SparkContext.textFiles will not create a partition for input splits that are empty.

Author: liulijia <liulijia@meituan.com>

Closes #19464 from liutang123/SPARK-22233.
2017-10-14 17:37:33 +09:00
Takuya UESHIN e0503a7223 [SPARK-22273][SQL] Fix key/value schema field names in HashMapGenerators.
## What changes were proposed in this pull request?

When fixing schema field names using escape characters with `addReferenceMinorObj()` at [SPARK-18952](https://issues.apache.org/jira/browse/SPARK-18952) (#16361), double-quotes around the names were remained and the names become something like `"((java.lang.String) references[1])"`.

```java
/* 055 */     private int maxSteps = 2;
/* 056 */     private int numRows = 0;
/* 057 */     private org.apache.spark.sql.types.StructType keySchema = new org.apache.spark.sql.types.StructType().add("((java.lang.String) references[1])", org.apache.spark.sql.types.DataTypes.StringType);
/* 058 */     private org.apache.spark.sql.types.StructType valueSchema = new org.apache.spark.sql.types.StructType().add("((java.lang.String) references[2])", org.apache.spark.sql.types.DataTypes.LongType);
/* 059 */     private Object emptyVBase;
```

We should remove the double-quotes to refer the values in `references` properly:

```java
/* 055 */     private int maxSteps = 2;
/* 056 */     private int numRows = 0;
/* 057 */     private org.apache.spark.sql.types.StructType keySchema = new org.apache.spark.sql.types.StructType().add(((java.lang.String) references[1]), org.apache.spark.sql.types.DataTypes.StringType);
/* 058 */     private org.apache.spark.sql.types.StructType valueSchema = new org.apache.spark.sql.types.StructType().add(((java.lang.String) references[2]), org.apache.spark.sql.types.DataTypes.LongType);
/* 059 */     private Object emptyVBase;
```

## How was this patch tested?

Existing tests.

Author: Takuya UESHIN <ueshin@databricks.com>

Closes #19491 from ueshin/issues/SPARK-22273.
2017-10-13 23:24:36 -07:00
Steve Loughran e3536406ec [SPARK-21762][SQL] FileFormatWriter/BasicWriteTaskStatsTracker metrics collection fails if a new file isn't yet visible
## What changes were proposed in this pull request?

`BasicWriteTaskStatsTracker.getFileSize()` to catch `FileNotFoundException`, log  info and then return 0 as a file size.

This ensures that if a newly created file isn't visible due to the store not always having create consistency, the metric collection doesn't cause the failure.

## How was this patch tested?

New test suite included, `BasicWriteTaskStatsTrackerSuite`. This not only checks the resilience to missing files, but verifies the existing logic as to how file statistics are gathered.

Note that in the current implementation

1. if you call `Tracker..getFinalStats()` more than once, the file size count will increase by size of the last file. This could be fixed by clearing the filename field inside `getFinalStats()` itself.

2. If you pass in an empty or null string to `Tracker.newFile(path)` then IllegalArgumentException is raised, but only in `getFinalStats()`, rather than in `newFile`.  There's a test for this behaviour in the new suite, as it verifies that only FNFEs get swallowed.

Author: Steve Loughran <stevel@hortonworks.com>

Closes #18979 from steveloughran/cloud/SPARK-21762-missing-files-in-metrics.
2017-10-13 23:08:17 -07:00