Commit graph

5403 commits

Author SHA1 Message Date
Koert Kuipers 608bf30f0b [SPARK-20359][SQL] Avoid unnecessary execution in EliminateOuterJoin optimization that can lead to NPE
Avoid necessary execution that can lead to NPE in EliminateOuterJoin and add test in DataFrameSuite to confirm NPE is no longer thrown

## What changes were proposed in this pull request?
Change leftHasNonNullPredicate and rightHasNonNullPredicate to lazy so they are only executed when needed.

## How was this patch tested?

Added test in DataFrameSuite that failed before this fix and now succeeds. Note that a test in catalyst project would be better but i am unsure how to do this.

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

Author: Koert Kuipers <koert@tresata.com>

Closes #17660 from koertkuipers/feat-catch-npe-in-eliminate-outer-join.
2017-04-19 15:52:47 +08:00
Kazuaki Ishizaki e468a96c40 [SPARK-20254][SQL] Remove unnecessary data conversion for Dataset with primitive array
## What changes were proposed in this pull request?

This PR elminates unnecessary data conversion, which is introduced by SPARK-19716, for Dataset with primitve array in the generated Java code.
When we run the following example program, now we get the Java code "Without this PR". In this code, lines 56-82 are unnecessary since the primitive array in ArrayData can be converted into Java primitive array by using ``toDoubleArray()`` method. ``GenericArrayData`` is not required.

```java
val ds = sparkContext.parallelize(Seq(Array(1.1, 2.2)), 1).toDS.cache
ds.count
ds.map(e => e).show
```

Without this PR
```
== Parsed Logical Plan ==
'SerializeFromObject [staticinvoke(class org.apache.spark.sql.catalyst.expressions.UnsafeArrayData, ArrayType(DoubleType,false), fromPrimitiveArray, input[0, [D, true], true) AS value#25]
+- 'MapElements <function1>, class [D, [StructField(value,ArrayType(DoubleType,false),true)], obj#24: [D
   +- 'DeserializeToObject unresolveddeserializer(unresolvedmapobjects(<function1>, getcolumnbyordinal(0, ArrayType(DoubleType,false)), None).toDoubleArray), obj#23: [D
      +- SerializeFromObject [staticinvoke(class org.apache.spark.sql.catalyst.expressions.UnsafeArrayData, ArrayType(DoubleType,false), fromPrimitiveArray, input[0, [D, true], true) AS value#2]
         +- ExternalRDD [obj#1]

== Analyzed Logical Plan ==
value: array<double>
SerializeFromObject [staticinvoke(class org.apache.spark.sql.catalyst.expressions.UnsafeArrayData, ArrayType(DoubleType,false), fromPrimitiveArray, input[0, [D, true], true) AS value#25]
+- MapElements <function1>, class [D, [StructField(value,ArrayType(DoubleType,false),true)], obj#24: [D
   +- DeserializeToObject mapobjects(MapObjects_loopValue5, MapObjects_loopIsNull5, DoubleType, assertnotnull(lambdavariable(MapObjects_loopValue5, MapObjects_loopIsNull5, DoubleType, true), - array element class: "scala.Double", - root class: "scala.Array"), value#2, None, MapObjects_builderValue5).toDoubleArray, obj#23: [D
      +- SerializeFromObject [staticinvoke(class org.apache.spark.sql.catalyst.expressions.UnsafeArrayData, ArrayType(DoubleType,false), fromPrimitiveArray, input[0, [D, true], true) AS value#2]
         +- ExternalRDD [obj#1]

== Optimized Logical Plan ==
SerializeFromObject [staticinvoke(class org.apache.spark.sql.catalyst.expressions.UnsafeArrayData, ArrayType(DoubleType,false), fromPrimitiveArray, input[0, [D, true], true) AS value#25]
+- MapElements <function1>, class [D, [StructField(value,ArrayType(DoubleType,false),true)], obj#24: [D
   +- DeserializeToObject mapobjects(MapObjects_loopValue5, MapObjects_loopIsNull5, DoubleType, assertnotnull(lambdavariable(MapObjects_loopValue5, MapObjects_loopIsNull5, DoubleType, true), - array element class: "scala.Double", - root class: "scala.Array"), value#2, None, MapObjects_builderValue5).toDoubleArray, obj#23: [D
      +- InMemoryRelation [value#2], true, 10000, StorageLevel(disk, memory, deserialized, 1 replicas)
            +- *SerializeFromObject [staticinvoke(class org.apache.spark.sql.catalyst.expressions.UnsafeArrayData, ArrayType(DoubleType,false), fromPrimitiveArray, input[0, [D, true], true) AS value#2]
               +- Scan ExternalRDDScan[obj#1]

== Physical Plan ==
*SerializeFromObject [staticinvoke(class org.apache.spark.sql.catalyst.expressions.UnsafeArrayData, ArrayType(DoubleType,false), fromPrimitiveArray, input[0, [D, true], true) AS value#25]
+- *MapElements <function1>, obj#24: [D
   +- *DeserializeToObject mapobjects(MapObjects_loopValue5, MapObjects_loopIsNull5, DoubleType, assertnotnull(lambdavariable(MapObjects_loopValue5, MapObjects_loopIsNull5, DoubleType, true), - array element class: "scala.Double", - root class: "scala.Array"), value#2, None, MapObjects_builderValue5).toDoubleArray, obj#23: [D
      +- InMemoryTableScan [value#2]
            +- InMemoryRelation [value#2], true, 10000, StorageLevel(disk, memory, deserialized, 1 replicas)
                  +- *SerializeFromObject [staticinvoke(class org.apache.spark.sql.catalyst.expressions.UnsafeArrayData, ArrayType(DoubleType,false), fromPrimitiveArray, input[0, [D, true], true) AS value#2]
                     +- Scan ExternalRDDScan[obj#1]
```

```java
/* 050 */   protected void processNext() throws java.io.IOException {
/* 051 */     while (inputadapter_input.hasNext() && !stopEarly()) {
/* 052 */       InternalRow inputadapter_row = (InternalRow) inputadapter_input.next();
/* 053 */       boolean inputadapter_isNull = inputadapter_row.isNullAt(0);
/* 054 */       ArrayData inputadapter_value = inputadapter_isNull ? null : (inputadapter_row.getArray(0));
/* 055 */
/* 056 */       ArrayData deserializetoobject_value1 = null;
/* 057 */
/* 058 */       if (!inputadapter_isNull) {
/* 059 */         int deserializetoobject_dataLength = inputadapter_value.numElements();
/* 060 */
/* 061 */         Double[] deserializetoobject_convertedArray = null;
/* 062 */         deserializetoobject_convertedArray = new Double[deserializetoobject_dataLength];
/* 063 */
/* 064 */         int deserializetoobject_loopIndex = 0;
/* 065 */         while (deserializetoobject_loopIndex < deserializetoobject_dataLength) {
/* 066 */           MapObjects_loopValue2 = (double) (inputadapter_value.getDouble(deserializetoobject_loopIndex));
/* 067 */           MapObjects_loopIsNull2 = inputadapter_value.isNullAt(deserializetoobject_loopIndex);
/* 068 */
/* 069 */           if (MapObjects_loopIsNull2) {
/* 070 */             throw new RuntimeException(((java.lang.String) references[0]));
/* 071 */           }
/* 072 */           if (false) {
/* 073 */             deserializetoobject_convertedArray[deserializetoobject_loopIndex] = null;
/* 074 */           } else {
/* 075 */             deserializetoobject_convertedArray[deserializetoobject_loopIndex] = MapObjects_loopValue2;
/* 076 */           }
/* 077 */
/* 078 */           deserializetoobject_loopIndex += 1;
/* 079 */         }
/* 080 */
/* 081 */         deserializetoobject_value1 = new org.apache.spark.sql.catalyst.util.GenericArrayData(deserializetoobject_convertedArray); /*###*/
/* 082 */       }
/* 083 */       boolean deserializetoobject_isNull = true;
/* 084 */       double[] deserializetoobject_value = null;
/* 085 */       if (!inputadapter_isNull) {
/* 086 */         deserializetoobject_isNull = false;
/* 087 */         if (!deserializetoobject_isNull) {
/* 088 */           Object deserializetoobject_funcResult = null;
/* 089 */           deserializetoobject_funcResult = deserializetoobject_value1.toDoubleArray();
/* 090 */           if (deserializetoobject_funcResult == null) {
/* 091 */             deserializetoobject_isNull = true;
/* 092 */           } else {
/* 093 */             deserializetoobject_value = (double[]) deserializetoobject_funcResult;
/* 094 */           }
/* 095 */
/* 096 */         }
/* 097 */         deserializetoobject_isNull = deserializetoobject_value == null;
/* 098 */       }
/* 099 */
/* 100 */       boolean mapelements_isNull = true;
/* 101 */       double[] mapelements_value = null;
/* 102 */       if (!false) {
/* 103 */         mapelements_resultIsNull = false;
/* 104 */
/* 105 */         if (!mapelements_resultIsNull) {
/* 106 */           mapelements_resultIsNull = deserializetoobject_isNull;
/* 107 */           mapelements_argValue = deserializetoobject_value;
/* 108 */         }
/* 109 */
/* 110 */         mapelements_isNull = mapelements_resultIsNull;
/* 111 */         if (!mapelements_isNull) {
/* 112 */           Object mapelements_funcResult = null;
/* 113 */           mapelements_funcResult = ((scala.Function1) references[1]).apply(mapelements_argValue);
/* 114 */           if (mapelements_funcResult == null) {
/* 115 */             mapelements_isNull = true;
/* 116 */           } else {
/* 117 */             mapelements_value = (double[]) mapelements_funcResult;
/* 118 */           }
/* 119 */
/* 120 */         }
/* 121 */         mapelements_isNull = mapelements_value == null;
/* 122 */       }
/* 123 */
/* 124 */       serializefromobject_resultIsNull = false;
/* 125 */
/* 126 */       if (!serializefromobject_resultIsNull) {
/* 127 */         serializefromobject_resultIsNull = mapelements_isNull;
/* 128 */         serializefromobject_argValue = mapelements_value;
/* 129 */       }
/* 130 */
/* 131 */       boolean serializefromobject_isNull = serializefromobject_resultIsNull;
/* 132 */       final ArrayData serializefromobject_value = serializefromobject_resultIsNull ? null : org.apache.spark.sql.catalyst.expressions.UnsafeArrayData.fromPrimitiveArray(serializefromobject_argValue);
/* 133 */       serializefromobject_isNull = serializefromobject_value == null;
/* 134 */       serializefromobject_holder.reset();
/* 135 */
/* 136 */       serializefromobject_rowWriter.zeroOutNullBytes();
/* 137 */
/* 138 */       if (serializefromobject_isNull) {
/* 139 */         serializefromobject_rowWriter.setNullAt(0);
/* 140 */       } else {
/* 141 */         // Remember the current cursor so that we can calculate how many bytes are
/* 142 */         // written later.
/* 143 */         final int serializefromobject_tmpCursor = serializefromobject_holder.cursor;
/* 144 */
/* 145 */         if (serializefromobject_value instanceof UnsafeArrayData) {
/* 146 */           final int serializefromobject_sizeInBytes = ((UnsafeArrayData) serializefromobject_value).getSizeInBytes();
/* 147 */           // grow the global buffer before writing data.
/* 148 */           serializefromobject_holder.grow(serializefromobject_sizeInBytes);
/* 149 */           ((UnsafeArrayData) serializefromobject_value).writeToMemory(serializefromobject_holder.buffer, serializefromobject_holder.cursor);
/* 150 */           serializefromobject_holder.cursor += serializefromobject_sizeInBytes;
/* 151 */
/* 152 */         } else {
/* 153 */           final int serializefromobject_numElements = serializefromobject_value.numElements();
/* 154 */           serializefromobject_arrayWriter.initialize(serializefromobject_holder, serializefromobject_numElements, 8);
/* 155 */
/* 156 */           for (int serializefromobject_index = 0; serializefromobject_index < serializefromobject_numElements; serializefromobject_index++) {
/* 157 */             if (serializefromobject_value.isNullAt(serializefromobject_index)) {
/* 158 */               serializefromobject_arrayWriter.setNullDouble(serializefromobject_index);
/* 159 */             } else {
/* 160 */               final double serializefromobject_element = serializefromobject_value.getDouble(serializefromobject_index);
/* 161 */               serializefromobject_arrayWriter.write(serializefromobject_index, serializefromobject_element);
/* 162 */             }
/* 163 */           }
/* 164 */         }
/* 165 */
/* 166 */         serializefromobject_rowWriter.setOffsetAndSize(0, serializefromobject_tmpCursor, serializefromobject_holder.cursor - serializefromobject_tmpCursor);
/* 167 */       }
/* 168 */       serializefromobject_result.setTotalSize(serializefromobject_holder.totalSize());
/* 169 */       append(serializefromobject_result);
/* 170 */       if (shouldStop()) return;
/* 171 */     }
/* 172 */   }
```

With this PR (eliminated lines 56-62 in the above code)
```java
/* 047 */   protected void processNext() throws java.io.IOException {
/* 048 */     while (inputadapter_input.hasNext() && !stopEarly()) {
/* 049 */       InternalRow inputadapter_row = (InternalRow) inputadapter_input.next();
/* 050 */       boolean inputadapter_isNull = inputadapter_row.isNullAt(0);
/* 051 */       ArrayData inputadapter_value = inputadapter_isNull ? null : (inputadapter_row.getArray(0));
/* 052 */
/* 053 */       boolean deserializetoobject_isNull = true;
/* 054 */       double[] deserializetoobject_value = null;
/* 055 */       if (!inputadapter_isNull) {
/* 056 */         deserializetoobject_isNull = false;
/* 057 */         if (!deserializetoobject_isNull) {
/* 058 */           Object deserializetoobject_funcResult = null;
/* 059 */           deserializetoobject_funcResult = inputadapter_value.toDoubleArray();
/* 060 */           if (deserializetoobject_funcResult == null) {
/* 061 */             deserializetoobject_isNull = true;
/* 062 */           } else {
/* 063 */             deserializetoobject_value = (double[]) deserializetoobject_funcResult;
/* 064 */           }
/* 065 */
/* 066 */         }
/* 067 */         deserializetoobject_isNull = deserializetoobject_value == null;
/* 068 */       }
/* 069 */
/* 070 */       boolean mapelements_isNull = true;
/* 071 */       double[] mapelements_value = null;
/* 072 */       if (!false) {
/* 073 */         mapelements_resultIsNull = false;
/* 074 */
/* 075 */         if (!mapelements_resultIsNull) {
/* 076 */           mapelements_resultIsNull = deserializetoobject_isNull;
/* 077 */           mapelements_argValue = deserializetoobject_value;
/* 078 */         }
/* 079 */
/* 080 */         mapelements_isNull = mapelements_resultIsNull;
/* 081 */         if (!mapelements_isNull) {
/* 082 */           Object mapelements_funcResult = null;
/* 083 */           mapelements_funcResult = ((scala.Function1) references[0]).apply(mapelements_argValue);
/* 084 */           if (mapelements_funcResult == null) {
/* 085 */             mapelements_isNull = true;
/* 086 */           } else {
/* 087 */             mapelements_value = (double[]) mapelements_funcResult;
/* 088 */           }
/* 089 */
/* 090 */         }
/* 091 */         mapelements_isNull = mapelements_value == null;
/* 092 */       }
/* 093 */
/* 094 */       serializefromobject_resultIsNull = false;
/* 095 */
/* 096 */       if (!serializefromobject_resultIsNull) {
/* 097 */         serializefromobject_resultIsNull = mapelements_isNull;
/* 098 */         serializefromobject_argValue = mapelements_value;
/* 099 */       }
/* 100 */
/* 101 */       boolean serializefromobject_isNull = serializefromobject_resultIsNull;
/* 102 */       final ArrayData serializefromobject_value = serializefromobject_resultIsNull ? null : org.apache.spark.sql.catalyst.expressions.UnsafeArrayData.fromPrimitiveArray(serializefromobject_argValue);
/* 103 */       serializefromobject_isNull = serializefromobject_value == null;
/* 104 */       serializefromobject_holder.reset();
/* 105 */
/* 106 */       serializefromobject_rowWriter.zeroOutNullBytes();
/* 107 */
/* 108 */       if (serializefromobject_isNull) {
/* 109 */         serializefromobject_rowWriter.setNullAt(0);
/* 110 */       } else {
/* 111 */         // Remember the current cursor so that we can calculate how many bytes are
/* 112 */         // written later.
/* 113 */         final int serializefromobject_tmpCursor = serializefromobject_holder.cursor;
/* 114 */
/* 115 */         if (serializefromobject_value instanceof UnsafeArrayData) {
/* 116 */           final int serializefromobject_sizeInBytes = ((UnsafeArrayData) serializefromobject_value).getSizeInBytes();
/* 117 */           // grow the global buffer before writing data.
/* 118 */           serializefromobject_holder.grow(serializefromobject_sizeInBytes);
/* 119 */           ((UnsafeArrayData) serializefromobject_value).writeToMemory(serializefromobject_holder.buffer, serializefromobject_holder.cursor);
/* 120 */           serializefromobject_holder.cursor += serializefromobject_sizeInBytes;
/* 121 */
/* 122 */         } else {
/* 123 */           final int serializefromobject_numElements = serializefromobject_value.numElements();
/* 124 */           serializefromobject_arrayWriter.initialize(serializefromobject_holder, serializefromobject_numElements, 8);
/* 125 */
/* 126 */           for (int serializefromobject_index = 0; serializefromobject_index < serializefromobject_numElements; serializefromobject_index++) {
/* 127 */             if (serializefromobject_value.isNullAt(serializefromobject_index)) {
/* 128 */               serializefromobject_arrayWriter.setNullDouble(serializefromobject_index);
/* 129 */             } else {
/* 130 */               final double serializefromobject_element = serializefromobject_value.getDouble(serializefromobject_index);
/* 131 */               serializefromobject_arrayWriter.write(serializefromobject_index, serializefromobject_element);
/* 132 */             }
/* 133 */           }
/* 134 */         }
/* 135 */
/* 136 */         serializefromobject_rowWriter.setOffsetAndSize(0, serializefromobject_tmpCursor, serializefromobject_holder.cursor - serializefromobject_tmpCursor);
/* 137 */       }
/* 138 */       serializefromobject_result.setTotalSize(serializefromobject_holder.totalSize());
/* 139 */       append(serializefromobject_result);
/* 140 */       if (shouldStop()) return;
/* 141 */     }
/* 142 */   }
```

## How was this patch tested?

Add test suites into `DatasetPrimitiveSuite`

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

Closes #17568 from kiszk/SPARK-20254.
2017-04-19 10:58:05 +08:00
wangzhenhua 321b4f03bc [SPARK-20366][SQL] Fix recursive join reordering: inside joins are not reordered
## What changes were proposed in this pull request?

If a plan has multi-level successive joins, e.g.:
```
         Join
         /   \
     Union   t5
      /   \
    Join  t4
    /   \
  Join  t3
  /  \
 t1   t2
```
Currently we fail to reorder the inside joins, i.e. t1, t2, t3.

In join reorder, we use `OrderedJoin` to indicate a join has been ordered, such that when transforming down the plan, these joins don't need to be rerodered again.

But there's a problem in the definition of `OrderedJoin`:
The real join node is a parameter, but not a child. This breaks the transform procedure because `mapChildren` applies transform function on parameters which should be children.

In this patch, we change `OrderedJoin` to a class having the same structure as a join node.

## How was this patch tested?

Add a corresponding test case.

Author: wangzhenhua <wangzhenhua@huawei.com>

Closes #17668 from wzhfy/recursiveReorder.
2017-04-18 20:12:21 +08:00
Felix Cheung b0a1e93e93 [SPARK-17647][SQL][FOLLOWUP][MINOR] fix typo
## What changes were proposed in this pull request?

fix typo

## How was this patch tested?

manual

Author: Felix Cheung <felixcheung_m@hotmail.com>

Closes #17663 from felixcheung/likedoctypo.
2017-04-17 23:55:40 -07:00
Jacek Laskowski 33ea908af9 [TEST][MINOR] Replace repartitionBy with distribute in CollapseRepartitionSuite
## What changes were proposed in this pull request?

Replace non-existent `repartitionBy` with `distribute` in `CollapseRepartitionSuite`.

## How was this patch tested?

local build and `catalyst/testOnly *CollapseRepartitionSuite`

Author: Jacek Laskowski <jacek@japila.pl>

Closes #17657 from jaceklaskowski/CollapseRepartitionSuite.
2017-04-17 17:58:10 -07:00
Jakob Odersky e5fee3e4f8 [SPARK-17647][SQL] Fix backslash escaping in 'LIKE' patterns.
## What changes were proposed in this pull request?

This patch fixes a bug in the way LIKE patterns are translated to Java regexes. The bug causes any character following an escaped backslash to be escaped, i.e. there is double-escaping.
A concrete example is the following pattern:`'%\\%'`. The expected Java regex that this pattern should correspond to (according to the behavior described below) is `'.*\\.*'`, however the current situation leads to `'.*\\%'` instead.

---

Update: in light of the discussion that ensued, we should explicitly define the expected behaviour of LIKE expressions, especially in certain edge cases. With the help of gatorsmile, we put together a list of different RDBMS and their variations wrt to certain standard features.

| RDBMS\Features | Wildcards | Default escape [1] | Case sensitivity |
| --- | --- | --- | --- |
| [MS SQL Server](https://msdn.microsoft.com/en-us/library/ms179859.aspx) | _, %, [], [^] | none | no |
| [Oracle](https://docs.oracle.com/cd/B12037_01/server.101/b10759/conditions016.htm) | _, % | none | yes |
| [DB2 z/OS](http://www.ibm.com/support/knowledgecenter/SSEPEK_11.0.0/sqlref/src/tpc/db2z_likepredicate.html) | _, % | none | yes |
| [MySQL](http://dev.mysql.com/doc/refman/5.7/en/string-comparison-functions.html) | _, % | none | no |
| [PostreSQL](https://www.postgresql.org/docs/9.0/static/functions-matching.html) | _, % | \ | yes |
| [Hive](https://cwiki.apache.org/confluence/display/Hive/LanguageManual+UDF) | _, % | none | yes |
| Current Spark | _, % | \ | yes |

[1] Default escape character: most systems do not have a default escape character, instead the user can specify one by calling a like expression with an escape argument [A] LIKE [B] ESCAPE [C]. This syntax is currently not supported by Spark, however I would volunteer to implement this feature in a separate ticket.

The specifications are often quite terse and certain scenarios are undocumented, so here is a list of scenarios that I am uncertain about and would appreciate any input. Specifically I am looking for feedback on whether or not Spark's current behavior should be changed.
1. [x] Ending a pattern with the escape sequence, e.g. `like 'a\'`.
   PostreSQL gives an error: 'LIKE pattern must not end with escape character', which I personally find logical. Currently, Spark allows "non-terminated" escapes and simply ignores them as part of the pattern.
   According to [DB2's documentation](http://www.ibm.com/support/knowledgecenter/SSEPGG_9.7.0/com.ibm.db2.luw.messages.sql.doc/doc/msql00130n.html), ending a pattern in an escape character is invalid.
   _Proposed new behaviour in Spark: throw AnalysisException_
2. [x] Empty input, e.g. `'' like ''`
   Postgres and DB2 will match empty input only if the pattern is empty as well, any other combination of empty input will not match. Spark currently follows this rule.
3. [x] Escape before a non-special character, e.g. `'a' like '\a'`.
   Escaping a non-wildcard character is not really documented but PostgreSQL just treats it verbatim, which I also find the least surprising behavior. Spark does the same.
   According to [DB2's documentation](http://www.ibm.com/support/knowledgecenter/SSEPGG_9.7.0/com.ibm.db2.luw.messages.sql.doc/doc/msql00130n.html), it is invalid to follow an escape character with anything other than an escape character, an underscore or a percent sign.
   _Proposed new behaviour in Spark: throw AnalysisException_

The current specification is also described in the operator's source code in this patch.
## How was this patch tested?

Extra case in regex unit tests.

Author: Jakob Odersky <jakob@odersky.com>

This patch had conflicts when merged, resolved by
Committer: Reynold Xin <rxin@databricks.com>

Closes #15398 from jodersky/SPARK-17647.
2017-04-17 11:17:57 -07:00
Xiao Li 01ff0350a8 [SPARK-20349][SQL] ListFunctions returns duplicate functions after using persistent functions
### What changes were proposed in this pull request?
The session catalog caches some persistent functions in the `FunctionRegistry`, so there can be duplicates. Our Catalog API `listFunctions` does not handle it.

It would be better if `SessionCatalog` API can de-duplciate the records, instead of doing it by each API caller. In `FunctionRegistry`, our functions are identified by the unquoted string. Thus, this PR is try to parse it using our parser interface and then de-duplicate the names.

### How was this patch tested?
Added test cases.

Author: Xiao Li <gatorsmile@gmail.com>

Closes #17646 from gatorsmile/showFunctions.
2017-04-17 09:50:20 -07:00
Xiao Li e090f3c0ce [SPARK-20335][SQL] Children expressions of Hive UDF impacts the determinism of Hive UDF
### What changes were proposed in this pull request?
```JAVA
  /**
   * Certain optimizations should not be applied if UDF is not deterministic.
   * Deterministic UDF returns same result each time it is invoked with a
   * particular input. This determinism just needs to hold within the context of
   * a query.
   *
   * return true if the UDF is deterministic
   */
  boolean deterministic() default true;
```

Based on the definition of [UDFType](https://github.com/apache/hive/blob/master/ql/src/java/org/apache/hadoop/hive/ql/udf/UDFType.java#L42-L50), when Hive UDF's children are non-deterministic, Hive UDF is also non-deterministic.

### How was this patch tested?
Added test cases.

Author: Xiao Li <gatorsmile@gmail.com>

Closes #17635 from gatorsmile/udfDeterministic.
2017-04-16 12:09:34 +08:00
Wenchen Fan 35e5ae4f81 [SPARK-19716][SQL][FOLLOW-UP] UnresolvedMapObjects should always be serializable
## What changes were proposed in this pull request?

In https://github.com/apache/spark/pull/17398 we introduced `UnresolvedMapObjects` as a placeholder of `MapObjects`. Unfortunately `UnresolvedMapObjects` is not serializable as its `function` may reference Scala `Type` which is not serializable.

Ideally this is fine, as we will never serialize and send unresolved expressions to executors. However users may accidentally do this, e.g. mistakenly reference an encoder instance when implementing `Aggregator`, we should fix it so that it's just a performance issue(more network traffic) and should not fail the query.

## How was this patch tested?

N/A

Author: Wenchen Fan <wenchen@databricks.com>

Closes #17639 from cloud-fan/minor.
2017-04-16 11:14:18 +08:00
ouyangxiaochen 98b41ecbcb [SPARK-20316][SQL] Val and Var should strictly follow the Scala syntax
## What changes were proposed in this pull request?

val and var should strictly follow the Scala syntax

## How was this patch tested?

manual test and exisiting test cases

Author: ouyangxiaochen <ou.yangxiaochen@zte.com.cn>

Closes #17628 from ouyangxiaochen/spark-413.
2017-04-15 10:34:57 +01:00
wangzhenhua fb036c4413 [SPARK-20318][SQL] Use Catalyst type for min/max in ColumnStat for ease of estimation
## What changes were proposed in this pull request?

Currently when estimating predicates like col > literal or col = literal, we will update min or max in column stats based on literal value. However, literal value is of Catalyst type (internal type), while min/max is of external type. Then for the next predicate, we again need to do type conversion to compare and update column stats. This is awkward and causes many unnecessary conversions in estimation.

To solve this, we use Catalyst type for min/max in `ColumnStat`. Note that the persistent format in metastore is still of external type, so there's no inconsistency for statistics in metastore.

This pr also fixes a bug for boolean type in `IN` condition.

## How was this patch tested?

The changes for ColumnStat are covered by existing tests.
For bug fix, a new test for boolean type in IN condition is added

Author: wangzhenhua <wangzhenhua@huawei.com>

Closes #17630 from wzhfy/refactorColumnStat.
2017-04-14 19:16:47 +08:00
Steve Loughran 7536e2849d [SPARK-20038][SQL] FileFormatWriter.ExecuteWriteTask.releaseResources() implementations to be re-entrant
## What changes were proposed in this pull request?

have the`FileFormatWriter.ExecuteWriteTask.releaseResources()` implementations  set `currentWriter=null` in a finally clause. This guarantees that if the first call to `currentWriter()` throws an exception, the second releaseResources() call made during the task cancel process will not trigger a second attempt to close the stream.

## How was this patch tested?

Tricky. I've been fixing the underlying cause when I saw the problem [HADOOP-14204](https://issues.apache.org/jira/browse/HADOOP-14204), but SPARK-10109 shows I'm not the first to have seen this. I can't replicate it locally any more, my code no longer being broken.

code review, however, should be straightforward

Author: Steve Loughran <stevel@hortonworks.com>

Closes #17364 from steveloughran/stevel/SPARK-20038-close.
2017-04-13 15:30:44 -05:00
Ioana Delaney fbe4216e1e [SPARK-20233][SQL] Apply star-join filter heuristics to dynamic programming join enumeration
## What changes were proposed in this pull request?

Implements star-join filter to reduce the search space for dynamic programming join enumeration. Consider the following join graph:

```
T1       D1 - T2 - T3
  \     /
    F1
     |
    D2

star-join: {F1, D1, D2}
non-star: {T1, T2, T3}
```
The following join combinations will be generated:
```
level 0: (F1), (D1), (D2), (T1), (T2), (T3)
level 1: {F1, D1}, {F1, D2}, {T2, T3}
level 2: {F1, D1, D2}
level 3: {F1, D1, D2, T1}, {F1, D1, D2, T2}
level 4: {F1, D1, D2, T1, T2}, {F1, D1, D2, T2, T3 }
level 6: {F1, D1, D2, T1, T2, T3}
```

## How was this patch tested?

New test suite ```StarJOinCostBasedReorderSuite.scala```.

Author: Ioana Delaney <ioanamdelaney@gmail.com>

Closes #17546 from ioana-delaney/starSchemaCBOv3.
2017-04-13 22:27:04 +08:00
Burak Yavuz 924c42477b [SPARK-20301][FLAKY-TEST] Fix Hadoop Shell.runCommand flakiness in Structured Streaming tests
## What changes were proposed in this pull request?

Some Structured Streaming tests show flakiness such as:
```
[info] - prune results by current_date, complete mode - 696 *** FAILED *** (10 seconds, 937 milliseconds)
[info]   Timed out while stopping and waiting for microbatchthread to terminate.: The code passed to failAfter did not complete within 10 seconds.
```

This happens when we wait for the stream to stop, but it doesn't. The reason it doesn't stop is that we interrupt the microBatchThread, but Hadoop's `Shell.runCommand` swallows the interrupt exception, and the exception is not propagated upstream to the microBatchThread. Then this thread continues to run, only to start blocking on the `streamManualClock`.

## How was this patch tested?

Thousand retries locally and [Jenkins](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/75720/testReport) of the flaky tests

Author: Burak Yavuz <brkyvz@gmail.com>

Closes #17613 from brkyvz/flaky-stream-agg.
2017-04-12 11:24:59 -07:00
Reynold Xin 540855382c [SPARK-20304][SQL] AssertNotNull should not include path in string representation
## What changes were proposed in this pull request?
AssertNotNull's toString/simpleString dumps the entire walkedTypePath. walkedTypePath is used for error message reporting and shouldn't be part of the output.

## How was this patch tested?
Manually tested.

Author: Reynold Xin <rxin@databricks.com>

Closes #17616 from rxin/SPARK-20304.
2017-04-12 09:05:05 -07:00
Xiao Li 504e62e2f4 [SPARK-20303][SQL] Rename createTempFunction to registerFunction
### What changes were proposed in this pull request?
Session catalog API `createTempFunction` is being used by Hive build-in functions, persistent functions, and temporary functions. Thus, the name is confusing. This PR is to rename it by `registerFunction`. Also we can move construction of `FunctionBuilder` and `ExpressionInfo` into the new `registerFunction`, instead of duplicating the logics everywhere.

In the next PRs, the remaining Function-related APIs also need cleanups.

### How was this patch tested?
Existing test cases.

Author: Xiao Li <gatorsmile@gmail.com>

Closes #17615 from gatorsmile/cleanupCreateTempFunction.
2017-04-12 09:01:26 -07:00
hyukjinkwon ceaf77ae43 [SPARK-18692][BUILD][DOCS] Test Java 8 unidoc build on Jenkins
## What changes were proposed in this pull request?

This PR proposes to run Spark unidoc to test Javadoc 8 build as Javadoc 8 is easily re-breakable.

There are several problems with it:

- It introduces little extra bit of time to run the tests. In my case, it took 1.5 mins more (`Elapsed :[94.8746569157]`). How it was tested is described in "How was this patch tested?".

- > One problem that I noticed was that Unidoc appeared to be processing test sources: if we can find a way to exclude those from being processed in the first place then that might significantly speed things up.

  (see  joshrosen's [comment](https://issues.apache.org/jira/browse/SPARK-18692?focusedCommentId=15947627&page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel#comment-15947627))

To complete this automated build, It also suggests to fix existing Javadoc breaks / ones introduced by test codes as described above.

There fixes are similar instances that previously fixed. Please refer https://github.com/apache/spark/pull/15999 and https://github.com/apache/spark/pull/16013

Note that this only fixes **errors** not **warnings**. Please see my observation https://github.com/apache/spark/pull/17389#issuecomment-288438704 for spurious errors by warnings.

## How was this patch tested?

Manually via `jekyll build` for building tests. Also, tested via running `./dev/run-tests`.

This was tested via manually adding `time.time()` as below:

```diff
     profiles_and_goals = build_profiles + sbt_goals

     print("[info] Building Spark unidoc (w/Hive 1.2.1) using SBT with these arguments: ",
           " ".join(profiles_and_goals))

+    import time
+    st = time.time()
     exec_sbt(profiles_and_goals)
+    print("Elapsed :[%s]" % str(time.time() - st))
```

produces

```
...
========================================================================
Building Unidoc API Documentation
========================================================================
...
[info] Main Java API documentation successful.
...
Elapsed :[94.8746569157]
...

Author: hyukjinkwon <gurwls223@gmail.com>

Closes #17477 from HyukjinKwon/SPARK-18692.
2017-04-12 12:38:48 +01:00
jtoka 2e1fd46e12 [SPARK-20296][TRIVIAL][DOCS] Count distinct error message for streaming
## What changes were proposed in this pull request?
Update count distinct error message for streaming datasets/dataframes to match current behavior. These aggregations are not yet supported, regardless of whether the dataset/dataframe is aggregated.

Author: jtoka <jason.tokayer@gmail.com>

Closes #17609 from jtoka/master.
2017-04-12 11:36:08 +01:00
Reynold Xin ffc57b0118 [SPARK-20302][SQL] Short circuit cast when from and to types are structurally the same
## What changes were proposed in this pull request?
When we perform a cast expression and the from and to types are structurally the same (having the same structure but different field names), we should be able to skip the actual cast.

## How was this patch tested?
Added unit tests for the newly introduced functions.

Author: Reynold Xin <rxin@databricks.com>

Closes #17614 from rxin/SPARK-20302.
2017-04-12 01:30:00 -07:00
hyukjinkwon bca4259f12 [MINOR][DOCS] JSON APIs related documentation fixes
## What changes were proposed in this pull request?

This PR proposes corrections related to JSON APIs as below:

- Rendering links in Python documentation
- Replacing `RDD` to `Dataset` in programing guide
- Adding missing description about JSON Lines consistently in `DataFrameReader.json` in Python API
- De-duplicating little bit of `DataFrameReader.json` in Scala/Java API

## How was this patch tested?

Manually build the documentation via `jekyll build`. Corresponding snapstops will be left on the codes.

Note that currently there are Javadoc8 breaks in several places. These are proposed to be handled in https://github.com/apache/spark/pull/17477. So, this PR does not fix those.

Author: hyukjinkwon <gurwls223@gmail.com>

Closes #17602 from HyukjinKwon/minor-json-documentation.
2017-04-12 09:16:39 +01:00
Dilip Biswal b14bfc3f8e [SPARK-19993][SQL] Caching logical plans containing subquery expressions does not work.
## What changes were proposed in this pull request?
The sameResult() method does not work when the logical plan contains subquery expressions.

**Before the fix**
```SQL
scala> val ds = spark.sql("select * from s1 where s1.c1 in (select s2.c1 from s2 where s1.c1 = s2.c1)")
ds: org.apache.spark.sql.DataFrame = [c1: int]

scala> ds.cache
res13: ds.type = [c1: int]

scala> spark.sql("select * from s1 where s1.c1 in (select s2.c1 from s2 where s1.c1 = s2.c1)").explain(true)
== Analyzed Logical Plan ==
c1: int
Project [c1#86]
+- Filter c1#86 IN (list#78 [c1#86])
   :  +- Project [c1#87]
   :     +- Filter (outer(c1#86) = c1#87)
   :        +- SubqueryAlias s2
   :           +- Relation[c1#87] parquet
   +- SubqueryAlias s1
      +- Relation[c1#86] parquet

== Optimized Logical Plan ==
Join LeftSemi, ((c1#86 = c1#87) && (c1#86 = c1#87))
:- Relation[c1#86] parquet
+- Relation[c1#87] parquet
```
**Plan after fix**
```SQL
== Analyzed Logical Plan ==
c1: int
Project [c1#22]
+- Filter c1#22 IN (list#14 [c1#22])
   :  +- Project [c1#23]
   :     +- Filter (outer(c1#22) = c1#23)
   :        +- SubqueryAlias s2
   :           +- Relation[c1#23] parquet
   +- SubqueryAlias s1
      +- Relation[c1#22] parquet

== Optimized Logical Plan ==
InMemoryRelation [c1#22], true, 10000, StorageLevel(disk, memory, deserialized, 1 replicas)
   +- *BroadcastHashJoin [c1#1, c1#1], [c1#2, c1#2], LeftSemi, BuildRight
      :- *FileScan parquet default.s1[c1#1] Batched: true, Format: Parquet, Location: InMemoryFileIndex[file:/Users/dbiswal/mygit/apache/spark/bin/spark-warehouse/s1], PartitionFilters: [], PushedFilters: [], ReadSchema: struct<c1:int>
      +- BroadcastExchange HashedRelationBroadcastMode(List((shiftleft(cast(input[0, int, true] as bigint), 32) | (cast(input[0, int, true] as bigint) & 4294967295))))
         +- *FileScan parquet default.s2[c1#2] Batched: true, Format: Parquet, Location: InMemoryFileIndex[file:/Users/dbiswal/mygit/apache/spark/bin/spark-warehouse/s2], PartitionFilters: [], PushedFilters: [], ReadSchema: struct<c1:int>
```
## How was this patch tested?
New tests are added to CachedTableSuite.

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

Closes #17330 from dilipbiswal/subquery_cache_final.
2017-04-12 12:18:01 +08:00
DB Tsai 8ad63ee158 [SPARK-20291][SQL] NaNvl(FloatType, NullType) should not be cast to NaNvl(DoubleType, DoubleType)
## What changes were proposed in this pull request?

`NaNvl(float value, null)` will be converted into `NaNvl(float value, Cast(null, DoubleType))` and finally `NaNvl(Cast(float value, DoubleType), Cast(null, DoubleType))`.

This will cause mismatching in the output type when the input type is float.

By adding extra rule in TypeCoercion can resolve this issue.

## How was this patch tested?

unite tests.

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

Author: DB Tsai <dbt@netflix.com>

Closes #17606 from dbtsai/fixNaNvl.
2017-04-12 11:19:20 +08:00
Reynold Xin 123b4fbbc3 [SPARK-20289][SQL] Use StaticInvoke to box primitive types
## What changes were proposed in this pull request?
Dataset typed API currently uses NewInstance to box primitive types (i.e. calling the constructor). Instead, it'd be slightly more idiomatic in Java to use PrimitiveType.valueOf, which can be invoked using StaticInvoke expression.

## How was this patch tested?
The change should be covered by existing tests for Dataset encoders.

Author: Reynold Xin <rxin@databricks.com>

Closes #17604 from rxin/SPARK-20289.
2017-04-11 11:12:31 -07:00
Liang-Chi Hsieh cd91f96714 [SPARK-20175][SQL] Exists should not be evaluated in Join operator
## What changes were proposed in this pull request?

Similar to `ListQuery`, `Exists` should not be evaluated in `Join` operator too.

## How was this patch tested?

Jenkins tests.

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

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

Closes #17491 from viirya/dont-push-exists-to-join.
2017-04-11 20:33:10 +08:00
Wenchen Fan c8706980ae [SPARK-20274][SQL] support compatible array element type in encoder
## What changes were proposed in this pull request?

This is a regression caused by SPARK-19716.

Before SPARK-19716, we will cast an array field to the expected array type. However, after SPARK-19716, the cast is removed, but we forgot to push the cast to the element level.

## How was this patch tested?

new regression tests

Author: Wenchen Fan <wenchen@databricks.com>

Closes #17587 from cloud-fan/array.
2017-04-11 20:21:04 +08:00
Reynold Xin 379b0b0bbd [SPARK-20283][SQL] Add preOptimizationBatches
## What changes were proposed in this pull request?
We currently have postHocOptimizationBatches, but not preOptimizationBatches. This patch adds preOptimizationBatches so the optimizer debugging extensions are symmetric.

## How was this patch tested?
N/A

Author: Reynold Xin <rxin@databricks.com>

Closes #17595 from rxin/SPARK-20283.
2017-04-10 14:14:09 -07:00
Shixiong Zhu a35b9d9712 [SPARK-20282][SS][TESTS] Write the commit log first to fix a race contion in tests
## What changes were proposed in this pull request?

This PR fixes the following failure:
```
sbt.ForkMain$ForkError: org.scalatest.exceptions.TestFailedException:
Assert on query failed:

== Progress ==
   AssertOnQuery(<condition>, )
   StopStream
   AddData to MemoryStream[value#30891]: 1,2
   StartStream(OneTimeTrigger,org.apache.spark.util.SystemClock35cdc93a,Map())
   CheckAnswer: [6],[3]
   StopStream
=> AssertOnQuery(<condition>, )
   AssertOnQuery(<condition>, )
   StartStream(OneTimeTrigger,org.apache.spark.util.SystemClockcdb247d,Map())
   CheckAnswer: [6],[3]
   StopStream
   AddData to MemoryStream[value#30891]: 3
   StartStream(OneTimeTrigger,org.apache.spark.util.SystemClock55394e4d,Map())
   CheckLastBatch: [2]
   StopStream
   AddData to MemoryStream[value#30891]: 0
   StartStream(OneTimeTrigger,org.apache.spark.util.SystemClock749aa997,Map())
   ExpectFailure[org.apache.spark.SparkException, isFatalError: false]
   AssertOnQuery(<condition>, )
   AssertOnQuery(<condition>, incorrect start offset or end offset on exception)

== Stream ==
Output Mode: Append
Stream state: not started
Thread state: dead

== Sink ==
0: [6] [3]

== Plan ==

	at org.scalatest.Assertions$class.newAssertionFailedException(Assertions.scala:495)
	at org.scalatest.FunSuite.newAssertionFailedException(FunSuite.scala:1555)
	at org.scalatest.Assertions$class.fail(Assertions.scala:1328)
	at org.scalatest.FunSuite.fail(FunSuite.scala:1555)
	at org.apache.spark.sql.streaming.StreamTest$class.failTest$1(StreamTest.scala:347)
	at org.apache.spark.sql.streaming.StreamTest$class.verify$1(StreamTest.scala:318)
	at org.apache.spark.sql.streaming.StreamTest$$anonfun$liftedTree1$1$1.apply(StreamTest.scala:483)
	at org.apache.spark.sql.streaming.StreamTest$$anonfun$liftedTree1$1$1.apply(StreamTest.scala:357)
	at scala.collection.mutable.ResizableArray$class.foreach(ResizableArray.scala:59)
	at scala.collection.mutable.ArrayBuffer.foreach(ArrayBuffer.scala:48)
	at org.apache.spark.sql.streaming.StreamTest$class.liftedTree1$1(StreamTest.scala:357)
	at org.apache.spark.sql.streaming.StreamTest$class.testStream(StreamTest.scala:356)
	at org.apache.spark.sql.streaming.StreamingQuerySuite.testStream(StreamingQuerySuite.scala:41)
	at org.apache.spark.sql.streaming.StreamingQuerySuite$$anonfun$6.apply$mcV$sp(StreamingQuerySuite.scala:166)
	at org.apache.spark.sql.streaming.StreamingQuerySuite$$anonfun$6.apply(StreamingQuerySuite.scala:161)
	at org.apache.spark.sql.streaming.StreamingQuerySuite$$anonfun$6.apply(StreamingQuerySuite.scala:161)
	at org.apache.spark.sql.catalyst.util.package$.quietly(package.scala:42)
	at org.apache.spark.sql.test.SQLTestUtils$$anonfun$testQuietly$1.apply$mcV$sp(SQLTestUtils.scala:268)
	at org.apache.spark.sql.test.SQLTestUtils$$anonfun$testQuietly$1.apply(SQLTestUtils.scala:268)
	at org.apache.spark.sql.test.SQLTestUtils$$anonfun$testQuietly$1.apply(SQLTestUtils.scala:268)
	at org.scalatest.Transformer$$anonfun$apply$1.apply$mcV$sp(Transformer.scala:22)
	at org.scalatest.OutcomeOf$class.outcomeOf(OutcomeOf.scala:85)
	at org.scalatest.OutcomeOf$.outcomeOf(OutcomeOf.scala:104)
	at org.scalatest.Transformer.apply(Transformer.scala:22)
	at org.scalatest.Transformer.apply(Transformer.scala:20)
	at org.scalatest.FunSuiteLike$$anon$1.apply(FunSuiteLike.scala:166)
	at org.apache.spark.SparkFunSuite.withFixture(SparkFunSuite.scala:68)
	at org.scalatest.FunSuiteLike$class.invokeWithFixture$1(FunSuiteLike.scala:163)
	at org.scalatest.FunSuiteLike$$anonfun$runTest$1.apply(FunSuiteLike.scala:175)
	at org.scalatest.FunSuiteLike$$anonfun$runTest$1.apply(FunSuiteLike.scala:175)
	at org.scalatest.SuperEngine.runTestImpl(Engine.scala:306)
	at org.scalatest.FunSuiteLike$class.runTest(FunSuiteLike.scala:175)
	at org.apache.spark.sql.streaming.StreamingQuerySuite.org$scalatest$BeforeAndAfterEach$$super$runTest(StreamingQuerySuite.scala:41)
	at org.scalatest.BeforeAndAfterEach$class.runTest(BeforeAndAfterEach.scala:255)
	at org.apache.spark.sql.streaming.StreamingQuerySuite.org$scalatest$BeforeAndAfter$$super$runTest(StreamingQuerySuite.scala:41)
	at org.scalatest.BeforeAndAfter$class.runTest(BeforeAndAfter.scala:200)
	at org.apache.spark.sql.streaming.StreamingQuerySuite.runTest(StreamingQuerySuite.scala:41)
	at org.scalatest.FunSuiteLike$$anonfun$runTests$1.apply(FunSuiteLike.scala:208)
	at org.scalatest.FunSuiteLike$$anonfun$runTests$1.apply(FunSuiteLike.scala:208)
	at org.scalatest.SuperEngine$$anonfun$traverseSubNodes$1$1.apply(Engine.scala:413)
	at org.scalatest.SuperEngine$$anonfun$traverseSubNodes$1$1.apply(Engine.scala:401)
	at scala.collection.immutable.List.foreach(List.scala:381)
	at org.scalatest.SuperEngine.traverseSubNodes$1(Engine.scala:401)
	at org.scalatest.SuperEngine.org$scalatest$SuperEngine$$runTestsInBranch(Engine.scala:396)
	at org.scalatest.SuperEngine.runTestsImpl(Engine.scala:483)
	at org.scalatest.FunSuiteLike$class.runTests(FunSuiteLike.scala:208)
	at org.scalatest.FunSuite.runTests(FunSuite.scala:1555)
	at org.scalatest.Suite$class.run(Suite.scala:1424)
	at org.scalatest.FunSuite.org$scalatest$FunSuiteLike$$super$run(FunSuite.scala:1555)
	at org.scalatest.FunSuiteLike$$anonfun$run$1.apply(FunSuiteLike.scala:212)
	at org.scalatest.FunSuiteLike$$anonfun$run$1.apply(FunSuiteLike.scala:212)
	at org.scalatest.SuperEngine.runImpl(Engine.scala:545)
	at org.scalatest.FunSuiteLike$class.run(FunSuiteLike.scala:212)
	at org.apache.spark.SparkFunSuite.org$scalatest$BeforeAndAfterAll$$super$run(SparkFunSuite.scala:31)
	at org.scalatest.BeforeAndAfterAll$class.liftedTree1$1(BeforeAndAfterAll.scala:257)
	at org.scalatest.BeforeAndAfterAll$class.run(BeforeAndAfterAll.scala:256)
	at org.apache.spark.sql.streaming.StreamingQuerySuite.org$scalatest$BeforeAndAfter$$super$run(StreamingQuerySuite.scala:41)
	at org.scalatest.BeforeAndAfter$class.run(BeforeAndAfter.scala:241)
	at org.apache.spark.sql.streaming.StreamingQuerySuite.run(StreamingQuerySuite.scala:41)
	at org.scalatest.tools.Framework.org$scalatest$tools$Framework$$runSuite(Framework.scala:357)
	at org.scalatest.tools.Framework$ScalaTestTask.execute(Framework.scala:502)
	at sbt.ForkMain$Run$2.call(ForkMain.java:296)
	at sbt.ForkMain$Run$2.call(ForkMain.java:286)
	at java.util.concurrent.FutureTask.run(FutureTask.java:266)
	at java.util.concurrent.ThreadPoolExecutor.runWorker(ThreadPoolExecutor.java:1142)
	at java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:617)
	at java.lang.Thread.run(Thread.java:745)
```

The failure is because `CheckAnswer` will run once `committedOffsets` is updated. Then writing the commit log may be interrupted by the following `StopStream`.

This PR just change the order to write the commit log first.

## How was this patch tested?

Jenkins

Author: Shixiong Zhu <shixiong@databricks.com>

Closes #17594 from zsxwing/SPARK-20282.
2017-04-10 14:09:32 -07:00
Bogdan Raducanu f6dd8e0e16 [SPARK-20280][CORE] FileStatusCache Weigher integer overflow
## What changes were proposed in this pull request?

Weigher.weigh needs to return Int but it is possible for an Array[FileStatus] to have size > Int.maxValue. To avoid this, the size is scaled down by a factor of 32. The maximumWeight of the cache is also scaled down by the same factor.

## How was this patch tested?
New test in FileIndexSuite

Author: Bogdan Raducanu <bogdan@databricks.com>

Closes #17591 from bogdanrdc/SPARK-20280.
2017-04-10 21:56:21 +02:00
Sean Owen a26e3ed5e4 [SPARK-20156][CORE][SQL][STREAMING][MLLIB] Java String toLowerCase "Turkish locale bug" causes Spark problems
## What changes were proposed in this pull request?

Add Locale.ROOT to internal calls to String `toLowerCase`, `toUpperCase`, to avoid inadvertent locale-sensitive variation in behavior (aka the "Turkish locale problem").

The change looks large but it is just adding `Locale.ROOT` (the locale with no country or language specified) to every call to these methods.

## How was this patch tested?

Existing tests.

Author: Sean Owen <sowen@cloudera.com>

Closes #17527 from srowen/SPARK-20156.
2017-04-10 20:11:56 +01:00
Xiao Li fd711ea13e [SPARK-20273][SQL] Disallow Non-deterministic Filter push-down into Join Conditions
## What changes were proposed in this pull request?
```
sql("SELECT t1.b, rand(0) as r FROM cachedData, cachedData t1 GROUP BY t1.b having r > 0.5").show()
```
We will get the following error:
```
Job aborted due to stage failure: Task 1 in stage 4.0 failed 1 times, most recent failure: Lost task 1.0 in stage 4.0 (TID 8, localhost, executor driver): java.lang.NullPointerException
	at org.apache.spark.sql.catalyst.expressions.GeneratedClass$SpecificPredicate.eval(Unknown Source)
	at org.apache.spark.sql.execution.joins.BroadcastNestedLoopJoinExec$$anonfun$org$apache$spark$sql$execution$joins$BroadcastNestedLoopJoinExec$$boundCondition$1.apply(BroadcastNestedLoopJoinExec.scala:87)
	at org.apache.spark.sql.execution.joins.BroadcastNestedLoopJoinExec$$anonfun$org$apache$spark$sql$execution$joins$BroadcastNestedLoopJoinExec$$boundCondition$1.apply(BroadcastNestedLoopJoinExec.scala:87)
	at scala.collection.Iterator$$anon$13.hasNext(Iterator.scala:463)
```
Filters could be pushed down to the join conditions by the optimizer rule `PushPredicateThroughJoin`. However, Analyzer [blocks users to add non-deterministics conditions](https://github.com/apache/spark/blob/master/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/CheckAnalysis.scala#L386-L395) (For details, see the PR https://github.com/apache/spark/pull/7535).

We should not push down non-deterministic conditions; otherwise, we need to explicitly initialize the non-deterministic expressions. This PR is to simply block it.

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

Author: Xiao Li <gatorsmile@gmail.com>

Closes #17585 from gatorsmile/joinRandCondition.
2017-04-10 09:15:04 -07:00
hyukjinkwon 5acaf8c0c6 [SPARK-19518][SQL] IGNORE NULLS in first / last in SQL
## What changes were proposed in this pull request?

This PR proposes to add `IGNORE NULLS` keyword in `first`/`last` in Spark's parser likewise http://docs.oracle.com/cd/B19306_01/server.102/b14200/functions057.htm.  This simply maps the keywords to existing `ignoreNullsExpr`.

**Before**

```scala
scala> sql("select first('a' IGNORE NULLS)").show()
```

```
org.apache.spark.sql.catalyst.parser.ParseException:
extraneous input 'NULLS' expecting {')', ','}(line 1, pos 24)

== SQL ==
select first('a' IGNORE NULLS)
------------------------^^^

  at org.apache.spark.sql.catalyst.parser.ParseException.withCommand(ParseDriver.scala:210)
  at org.apache.spark.sql.catalyst.parser.AbstractSqlParser.parse(ParseDriver.scala:112)
  at org.apache.spark.sql.execution.SparkSqlParser.parse(SparkSqlParser.scala:46)
  at org.apache.spark.sql.catalyst.parser.AbstractSqlParser.parsePlan(ParseDriver.scala:66)
  at org.apache.spark.sql.SparkSession.sql(SparkSession.scala:622)
  ... 48 elided
```

**After**

```scala
scala> sql("select first('a' IGNORE NULLS)").show()
```

```
+--------------+
|first(a, true)|
+--------------+
|             a|
+--------------+
```

## How was this patch tested?

Unit tests in `ExpressionParserSuite`.

Author: hyukjinkwon <gurwls223@gmail.com>

Closes #17566 from HyukjinKwon/SPARK-19518.
2017-04-10 17:45:27 +02:00
Wenchen Fan 3d7f201f2a [SPARK-20229][SQL] add semanticHash to QueryPlan
## What changes were proposed in this pull request?

Like `Expression`, `QueryPlan` should also have a `semanticHash` method, then we can put plans to a hash map and look it up fast. This PR refactors `QueryPlan` to follow `Expression` and put all the normalization logic in `QueryPlan.canonicalized`, so that it's very natural to implement `semanticHash`.

follow-up: improve `CacheManager` to leverage this `semanticHash` and speed up plan lookup, instead of iterating all cached plans.

## How was this patch tested?

existing tests. Note that we don't need to test the `semanticHash` method, once the existing tests prove `sameResult` is correct, we are good.

Author: Wenchen Fan <wenchen@databricks.com>

Closes #17541 from cloud-fan/plan-semantic.
2017-04-10 13:36:08 +08:00
DB Tsai 1a0bc41659
[SPARK-20270][SQL] na.fill should not change the values in long or integer when the default value is in double
## What changes were proposed in this pull request?

This bug was partially addressed in SPARK-18555 https://github.com/apache/spark/pull/15994, but the root cause isn't completely solved. This bug is pretty critical since it changes the member id in Long in our application if the member id can not be represented by Double losslessly when the member id is very big.

Here is an example how this happens, with
```
      Seq[(java.lang.Long, java.lang.Double)]((null, 3.14), (9123146099426677101L, null),
        (9123146560113991650L, 1.6), (null, null)).toDF("a", "b").na.fill(0.2),
```
the logical plan will be
```
== Analyzed Logical Plan ==
a: bigint, b: double
Project [cast(coalesce(cast(a#232L as double), cast(0.2 as double)) as bigint) AS a#240L, cast(coalesce(nanvl(b#233, cast(null as double)), 0.2) as double) AS b#241]
+- Project [_1#229L AS a#232L, _2#230 AS b#233]
   +- LocalRelation [_1#229L, _2#230]
```

Note that even the value is not null, Spark will cast the Long into Double first. Then if it's not null, Spark will cast it back to Long which results in losing precision.

The behavior should be that the original value should not be changed if it's not null, but Spark will change the value which is wrong.

With the PR, the logical plan will be
```
== Analyzed Logical Plan ==
a: bigint, b: double
Project [coalesce(a#232L, cast(0.2 as bigint)) AS a#240L, coalesce(nanvl(b#233, cast(null as double)), cast(0.2 as double)) AS b#241]
+- Project [_1#229L AS a#232L, _2#230 AS b#233]
   +- LocalRelation [_1#229L, _2#230]
```
which behaves correctly without changing the original Long values and also avoids extra cost of unnecessary casting.

## How was this patch tested?

unit test added.

+cc srowen rxin cloud-fan gatorsmile

Thanks.

Author: DB Tsai <dbt@netflix.com>

Closes #17577 from dbtsai/fixnafill.
2017-04-10 05:16:34 +00:00
Reynold Xin 7bfa05e0a5 [SPARK-20264][SQL] asm should be non-test dependency in sql/core
## What changes were proposed in this pull request?
sq/core module currently declares asm as a test scope dependency. Transitively it should actually be a normal dependency since the actual core module defines it. This occasionally confuses IntelliJ.

## How was this patch tested?
N/A - This is a build change.

Author: Reynold Xin <rxin@databricks.com>

Closes #17574 from rxin/SPARK-20264.
2017-04-09 20:32:07 -07:00
Kazuaki Ishizaki 7a63f5e827 [SPARK-20253][SQL] Remove unnecessary nullchecks of a return value from Spark runtime routines in generated Java code
## What changes were proposed in this pull request?

This PR elminates unnecessary nullchecks of a return value from known Spark runtime routines. We know whether a given Spark runtime routine returns ``null`` or not (e.g. ``ArrayData.toDoubleArray()`` never returns ``null``). Thus, we can eliminate a null check for the return value from the Spark runtime routine.

When we run the following example program, now we get the Java code "Without this PR". In this code, since we know ``ArrayData.toDoubleArray()`` never returns ``null```, we can eliminate null checks at lines 90-92, and 97.

```java
val ds = sparkContext.parallelize(Seq(Array(1.1, 2.2)), 1).toDS.cache
ds.count
ds.map(e => e).show
```

Without this PR
```java
/* 050 */   protected void processNext() throws java.io.IOException {
/* 051 */     while (inputadapter_input.hasNext() && !stopEarly()) {
/* 052 */       InternalRow inputadapter_row = (InternalRow) inputadapter_input.next();
/* 053 */       boolean inputadapter_isNull = inputadapter_row.isNullAt(0);
/* 054 */       ArrayData inputadapter_value = inputadapter_isNull ? null : (inputadapter_row.getArray(0));
/* 055 */
/* 056 */       ArrayData deserializetoobject_value1 = null;
/* 057 */
/* 058 */       if (!inputadapter_isNull) {
/* 059 */         int deserializetoobject_dataLength = inputadapter_value.numElements();
/* 060 */
/* 061 */         Double[] deserializetoobject_convertedArray = null;
/* 062 */         deserializetoobject_convertedArray = new Double[deserializetoobject_dataLength];
/* 063 */
/* 064 */         int deserializetoobject_loopIndex = 0;
/* 065 */         while (deserializetoobject_loopIndex < deserializetoobject_dataLength) {
/* 066 */           MapObjects_loopValue2 = (double) (inputadapter_value.getDouble(deserializetoobject_loopIndex));
/* 067 */           MapObjects_loopIsNull2 = inputadapter_value.isNullAt(deserializetoobject_loopIndex);
/* 068 */
/* 069 */           if (MapObjects_loopIsNull2) {
/* 070 */             throw new RuntimeException(((java.lang.String) references[0]));
/* 071 */           }
/* 072 */           if (false) {
/* 073 */             deserializetoobject_convertedArray[deserializetoobject_loopIndex] = null;
/* 074 */           } else {
/* 075 */             deserializetoobject_convertedArray[deserializetoobject_loopIndex] = MapObjects_loopValue2;
/* 076 */           }
/* 077 */
/* 078 */           deserializetoobject_loopIndex += 1;
/* 079 */         }
/* 080 */
/* 081 */         deserializetoobject_value1 = new org.apache.spark.sql.catalyst.util.GenericArrayData(deserializetoobject_convertedArray); /*###*/
/* 082 */       }
/* 083 */       boolean deserializetoobject_isNull = true;
/* 084 */       double[] deserializetoobject_value = null;
/* 085 */       if (!inputadapter_isNull) {
/* 086 */         deserializetoobject_isNull = false;
/* 087 */         if (!deserializetoobject_isNull) {
/* 088 */           Object deserializetoobject_funcResult = null;
/* 089 */           deserializetoobject_funcResult = deserializetoobject_value1.toDoubleArray();
/* 090 */           if (deserializetoobject_funcResult == null) {
/* 091 */             deserializetoobject_isNull = true;
/* 092 */           } else {
/* 093 */             deserializetoobject_value = (double[]) deserializetoobject_funcResult;
/* 094 */           }
/* 095 */
/* 096 */         }
/* 097 */         deserializetoobject_isNull = deserializetoobject_value == null;
/* 098 */       }
/* 099 */
/* 100 */       boolean mapelements_isNull = true;
/* 101 */       double[] mapelements_value = null;
/* 102 */       if (!false) {
/* 103 */         mapelements_resultIsNull = false;
/* 104 */
/* 105 */         if (!mapelements_resultIsNull) {
/* 106 */           mapelements_resultIsNull = deserializetoobject_isNull;
/* 107 */           mapelements_argValue = deserializetoobject_value;
/* 108 */         }
/* 109 */
/* 110 */         mapelements_isNull = mapelements_resultIsNull;
/* 111 */         if (!mapelements_isNull) {
/* 112 */           Object mapelements_funcResult = null;
/* 113 */           mapelements_funcResult = ((scala.Function1) references[1]).apply(mapelements_argValue);
/* 114 */           if (mapelements_funcResult == null) {
/* 115 */             mapelements_isNull = true;
/* 116 */           } else {
/* 117 */             mapelements_value = (double[]) mapelements_funcResult;
/* 118 */           }
/* 119 */
/* 120 */         }
/* 121 */         mapelements_isNull = mapelements_value == null;
/* 122 */       }
/* 123 */
/* 124 */       serializefromobject_resultIsNull = false;
/* 125 */
/* 126 */       if (!serializefromobject_resultIsNull) {
/* 127 */         serializefromobject_resultIsNull = mapelements_isNull;
/* 128 */         serializefromobject_argValue = mapelements_value;
/* 129 */       }
/* 130 */
/* 131 */       boolean serializefromobject_isNull = serializefromobject_resultIsNull;
/* 132 */       final ArrayData serializefromobject_value = serializefromobject_resultIsNull ? null : org.apache.spark.sql.catalyst.expressions.UnsafeArrayData.fromPrimitiveArray(serializefromobject_argValue);
/* 133 */       serializefromobject_isNull = serializefromobject_value == null;
/* 134 */       serializefromobject_holder.reset();
/* 135 */
/* 136 */       serializefromobject_rowWriter.zeroOutNullBytes();
/* 137 */
/* 138 */       if (serializefromobject_isNull) {
/* 139 */         serializefromobject_rowWriter.setNullAt(0);
/* 140 */       } else {
/* 141 */         // Remember the current cursor so that we can calculate how many bytes are
/* 142 */         // written later.
/* 143 */         final int serializefromobject_tmpCursor = serializefromobject_holder.cursor;
/* 144 */
/* 145 */         if (serializefromobject_value instanceof UnsafeArrayData) {
/* 146 */           final int serializefromobject_sizeInBytes = ((UnsafeArrayData) serializefromobject_value).getSizeInBytes();
/* 147 */           // grow the global buffer before writing data.
/* 148 */           serializefromobject_holder.grow(serializefromobject_sizeInBytes);
/* 149 */           ((UnsafeArrayData) serializefromobject_value).writeToMemory(serializefromobject_holder.buffer, serializefromobject_holder.cursor);
/* 150 */           serializefromobject_holder.cursor += serializefromobject_sizeInBytes;
/* 151 */
/* 152 */         } else {
/* 153 */           final int serializefromobject_numElements = serializefromobject_value.numElements();
/* 154 */           serializefromobject_arrayWriter.initialize(serializefromobject_holder, serializefromobject_numElements, 8);
/* 155 */
/* 156 */           for (int serializefromobject_index = 0; serializefromobject_index < serializefromobject_numElements; serializefromobject_index++) {
/* 157 */             if (serializefromobject_value.isNullAt(serializefromobject_index)) {
/* 158 */               serializefromobject_arrayWriter.setNullDouble(serializefromobject_index);
/* 159 */             } else {
/* 160 */               final double serializefromobject_element = serializefromobject_value.getDouble(serializefromobject_index);
/* 161 */               serializefromobject_arrayWriter.write(serializefromobject_index, serializefromobject_element);
/* 162 */             }
/* 163 */           }
/* 164 */         }
/* 165 */
/* 166 */         serializefromobject_rowWriter.setOffsetAndSize(0, serializefromobject_tmpCursor, serializefromobject_holder.cursor - serializefromobject_tmpCursor);
/* 167 */       }
/* 168 */       serializefromobject_result.setTotalSize(serializefromobject_holder.totalSize());
/* 169 */       append(serializefromobject_result);
/* 170 */       if (shouldStop()) return;
/* 171 */     }
/* 172 */   }
```

With this PR (removed most of lines 90-97 in the above code)
```java
/* 050 */   protected void processNext() throws java.io.IOException {
/* 051 */     while (inputadapter_input.hasNext() && !stopEarly()) {
/* 052 */       InternalRow inputadapter_row = (InternalRow) inputadapter_input.next();
/* 053 */       boolean inputadapter_isNull = inputadapter_row.isNullAt(0);
/* 054 */       ArrayData inputadapter_value = inputadapter_isNull ? null : (inputadapter_row.getArray(0));
/* 055 */
/* 056 */       ArrayData deserializetoobject_value1 = null;
/* 057 */
/* 058 */       if (!inputadapter_isNull) {
/* 059 */         int deserializetoobject_dataLength = inputadapter_value.numElements();
/* 060 */
/* 061 */         Double[] deserializetoobject_convertedArray = null;
/* 062 */         deserializetoobject_convertedArray = new Double[deserializetoobject_dataLength];
/* 063 */
/* 064 */         int deserializetoobject_loopIndex = 0;
/* 065 */         while (deserializetoobject_loopIndex < deserializetoobject_dataLength) {
/* 066 */           MapObjects_loopValue2 = (double) (inputadapter_value.getDouble(deserializetoobject_loopIndex));
/* 067 */           MapObjects_loopIsNull2 = inputadapter_value.isNullAt(deserializetoobject_loopIndex);
/* 068 */
/* 069 */           if (MapObjects_loopIsNull2) {
/* 070 */             throw new RuntimeException(((java.lang.String) references[0]));
/* 071 */           }
/* 072 */           if (false) {
/* 073 */             deserializetoobject_convertedArray[deserializetoobject_loopIndex] = null;
/* 074 */           } else {
/* 075 */             deserializetoobject_convertedArray[deserializetoobject_loopIndex] = MapObjects_loopValue2;
/* 076 */           }
/* 077 */
/* 078 */           deserializetoobject_loopIndex += 1;
/* 079 */         }
/* 080 */
/* 081 */         deserializetoobject_value1 = new org.apache.spark.sql.catalyst.util.GenericArrayData(deserializetoobject_convertedArray); /*###*/
/* 082 */       }
/* 083 */       boolean deserializetoobject_isNull = true;
/* 084 */       double[] deserializetoobject_value = null;
/* 085 */       if (!inputadapter_isNull) {
/* 086 */         deserializetoobject_isNull = false;
/* 087 */         if (!deserializetoobject_isNull) {
/* 088 */           Object deserializetoobject_funcResult = null;
/* 089 */           deserializetoobject_funcResult = deserializetoobject_value1.toDoubleArray();
/* 090 */           deserializetoobject_value = (double[]) deserializetoobject_funcResult;
/* 091 */
/* 092 */         }
/* 093 */
/* 094 */       }
/* 095 */
/* 096 */       boolean mapelements_isNull = true;
/* 097 */       double[] mapelements_value = null;
/* 098 */       if (!false) {
/* 099 */         mapelements_resultIsNull = false;
/* 100 */
/* 101 */         if (!mapelements_resultIsNull) {
/* 102 */           mapelements_resultIsNull = deserializetoobject_isNull;
/* 103 */           mapelements_argValue = deserializetoobject_value;
/* 104 */         }
/* 105 */
/* 106 */         mapelements_isNull = mapelements_resultIsNull;
/* 107 */         if (!mapelements_isNull) {
/* 108 */           Object mapelements_funcResult = null;
/* 109 */           mapelements_funcResult = ((scala.Function1) references[1]).apply(mapelements_argValue);
/* 110 */           if (mapelements_funcResult == null) {
/* 111 */             mapelements_isNull = true;
/* 112 */           } else {
/* 113 */             mapelements_value = (double[]) mapelements_funcResult;
/* 114 */           }
/* 115 */
/* 116 */         }
/* 117 */         mapelements_isNull = mapelements_value == null;
/* 118 */       }
/* 119 */
/* 120 */       serializefromobject_resultIsNull = false;
/* 121 */
/* 122 */       if (!serializefromobject_resultIsNull) {
/* 123 */         serializefromobject_resultIsNull = mapelements_isNull;
/* 124 */         serializefromobject_argValue = mapelements_value;
/* 125 */       }
/* 126 */
/* 127 */       boolean serializefromobject_isNull = serializefromobject_resultIsNull;
/* 128 */       final ArrayData serializefromobject_value = serializefromobject_resultIsNull ? null : org.apache.spark.sql.catalyst.expressions.UnsafeArrayData.fromPrimitiveArray(serializefromobject_argValue);
/* 129 */       serializefromobject_isNull = serializefromobject_value == null;
/* 130 */       serializefromobject_holder.reset();
/* 131 */
/* 132 */       serializefromobject_rowWriter.zeroOutNullBytes();
/* 133 */
/* 134 */       if (serializefromobject_isNull) {
/* 135 */         serializefromobject_rowWriter.setNullAt(0);
/* 136 */       } else {
/* 137 */         // Remember the current cursor so that we can calculate how many bytes are
/* 138 */         // written later.
/* 139 */         final int serializefromobject_tmpCursor = serializefromobject_holder.cursor;
/* 140 */
/* 141 */         if (serializefromobject_value instanceof UnsafeArrayData) {
/* 142 */           final int serializefromobject_sizeInBytes = ((UnsafeArrayData) serializefromobject_value).getSizeInBytes();
/* 143 */           // grow the global buffer before writing data.
/* 144 */           serializefromobject_holder.grow(serializefromobject_sizeInBytes);
/* 145 */           ((UnsafeArrayData) serializefromobject_value).writeToMemory(serializefromobject_holder.buffer, serializefromobject_holder.cursor);
/* 146 */           serializefromobject_holder.cursor += serializefromobject_sizeInBytes;
/* 147 */
/* 148 */         } else {
/* 149 */           final int serializefromobject_numElements = serializefromobject_value.numElements();
/* 150 */           serializefromobject_arrayWriter.initialize(serializefromobject_holder, serializefromobject_numElements, 8);
/* 151 */
/* 152 */           for (int serializefromobject_index = 0; serializefromobject_index < serializefromobject_numElements; serializefromobject_index++) {
/* 153 */             if (serializefromobject_value.isNullAt(serializefromobject_index)) {
/* 154 */               serializefromobject_arrayWriter.setNullDouble(serializefromobject_index);
/* 155 */             } else {
/* 156 */               final double serializefromobject_element = serializefromobject_value.getDouble(serializefromobject_index);
/* 157 */               serializefromobject_arrayWriter.write(serializefromobject_index, serializefromobject_element);
/* 158 */             }
/* 159 */           }
/* 160 */         }
/* 161 */
/* 162 */         serializefromobject_rowWriter.setOffsetAndSize(0, serializefromobject_tmpCursor, serializefromobject_holder.cursor - serializefromobject_tmpCursor);
/* 163 */       }
/* 164 */       serializefromobject_result.setTotalSize(serializefromobject_holder.totalSize());
/* 165 */       append(serializefromobject_result);
/* 166 */       if (shouldStop()) return;
/* 167 */     }
/* 168 */   }
```

## How was this patch tested?

Add test suites to ``DatasetPrimitiveSuite``

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

Closes #17569 from kiszk/SPARK-20253.
2017-04-10 10:47:17 +08:00
Vijay Ramesh 261eaf5149 [SPARK-20260][MLLIB] String interpolation required for error message
## What changes were proposed in this pull request?
This error message doesn't get properly formatted because of a missing `s`.  Currently the error looks like:

```
Caused by: java.lang.IllegalArgumentException: requirement failed: indices should be one-based and in ascending order; found current=$current, previous=$previous; line="$line"
```
(note the literal `$current` instead of the interpolated value)

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

Author: Vijay Ramesh <vramesh@demandbase.com>

Closes #17572 from vijaykramesh/master.
2017-04-09 19:39:09 +01:00
Reynold Xin e1afc4dcca [SPARK-20262][SQL] AssertNotNull should throw NullPointerException
## What changes were proposed in this pull request?
AssertNotNull currently throws RuntimeException. It should throw NullPointerException, which is more specific.

## How was this patch tested?
N/A

Author: Reynold Xin <rxin@databricks.com>

Closes #17573 from rxin/SPARK-20262.
2017-04-07 21:14:50 -07:00
Wenchen Fan 7577e9c356 [SPARK-20246][SQL] should not push predicate down through aggregate with non-deterministic expressions
## What changes were proposed in this pull request?

Similar to `Project`, when `Aggregate` has non-deterministic expressions, we should not push predicate down through it, as it will change the number of input rows and thus change the evaluation result of non-deterministic expressions in `Aggregate`.

## How was this patch tested?

new regression test

Author: Wenchen Fan <wenchen@databricks.com>

Closes #17562 from cloud-fan/filter.
2017-04-07 20:54:18 -07:00
Adrian Ionescu 589f3edb82 [SPARK-20255] Move listLeafFiles() to InMemoryFileIndex
## What changes were proposed in this pull request

Trying to get a grip on the `FileIndex` hierarchy, I was confused by the following inconsistency:

On the one hand, `PartitioningAwareFileIndex` defines `leafFiles` and `leafDirToChildrenFiles` as abstract, but on the other it fully implements `listLeafFiles` which does all the listing of files. However, the latter is only used by `InMemoryFileIndex`.

I'm hereby proposing to move this method (and all its dependencies) to the implementation class that actually uses it, and thus unclutter the `PartitioningAwareFileIndex` interface.

## How was this patch tested?

`./build/sbt sql/test`

Author: Adrian Ionescu <adrian@databricks.com>

Closes #17570 from adrian-ionescu/list-leaf-files.
2017-04-07 14:00:23 -07:00
Wenchen Fan ad3cc1312d [SPARK-20245][SQL][MINOR] pass output to LogicalRelation directly
## What changes were proposed in this pull request?

Currently `LogicalRelation` has a `expectedOutputAttributes` parameter, which makes it hard to reason about what the actual output is. Like other leaf nodes, `LogicalRelation` should also take `output` as a parameter, to simplify the logic

## How was this patch tested?

existing tests

Author: Wenchen Fan <wenchen@databricks.com>

Closes #17552 from cloud-fan/minor.
2017-04-07 15:58:50 +08:00
Reynold Xin 626b4cafce [SPARK-19495][SQL] Make SQLConf slightly more extensible - addendum
## What changes were proposed in this pull request?
This is a tiny addendum to SPARK-19495 to remove the private visibility for copy, which is the only package private method in the entire file.

## How was this patch tested?
N/A - no semantic change.

Author: Reynold Xin <rxin@databricks.com>

Closes #17555 from rxin/SPARK-19495-2.
2017-04-06 19:24:03 -07:00
Felix Cheung bccc330193 [SPARK-20196][PYTHON][SQL] update doc for catalog functions for all languages, add pyspark refreshByPath API
## What changes were proposed in this pull request?

Update doc to remove external for createTable, add refreshByPath in python

## How was this patch tested?

manual

Author: Felix Cheung <felixcheung_m@hotmail.com>

Closes #17512 from felixcheung/catalogdoc.
2017-04-06 09:09:43 -07:00
Ioana Delaney 4000f128b7 [SPARK-20231][SQL] Refactor star schema code for the subsequent star join detection in CBO
## What changes were proposed in this pull request?

This commit moves star schema code from ```join.scala``` to ```StarSchemaDetection.scala```. It also applies some minor fixes in ```StarJoinReorderSuite.scala```.

## How was this patch tested?
Run existing ```StarJoinReorderSuite.scala```.

Author: Ioana Delaney <ioanamdelaney@gmail.com>

Closes #17544 from ioana-delaney/starSchemaCBOv2.
2017-04-05 18:02:53 -07:00
Dilip Biswal 9d68c67235 [SPARK-20204][SQL][FOLLOWUP] SQLConf should react to change in default timezone settings
## What changes were proposed in this pull request?
Make sure SESSION_LOCAL_TIMEZONE reflects the change in JVM's default timezone setting. Currently several timezone related tests fail as the change to default timezone is not picked up by SQLConf.

## How was this patch tested?
Added an unit test in ConfigEntrySuite

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

Closes #17537 from dilipbiswal/timezone_debug.
2017-04-06 08:33:14 +08:00
Tathagata Das 9543fc0e08 [SPARK-20224][SS] Updated docs for streaming dropDuplicates and mapGroupsWithState
## What changes were proposed in this pull request?

- Fixed bug in Java API not passing timeout conf to scala API
- Updated markdown docs
- Updated scala docs
- Added scala and Java example

## How was this patch tested?
Manually ran examples.

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

Closes #17539 from tdas/SPARK-20224.
2017-04-05 16:03:04 -07:00
wangzhenhua a2d8d767d9 [SPARK-20223][SQL] Fix typo in tpcds q77.sql
## What changes were proposed in this pull request?

Fix typo in tpcds q77.sql

## How was this patch tested?

N/A

Author: wangzhenhua <wangzhenhua@huawei.com>

Closes #17538 from wzhfy/typoQ77.
2017-04-05 10:21:43 -07:00
Tathagata Das dad499f324 [SPARK-20209][SS] Execute next trigger immediately if previous batch took longer than trigger interval
## What changes were proposed in this pull request?

For large trigger intervals (e.g. 10 minutes), if a batch takes 11 minutes, then it will wait for 9 mins before starting the next batch. This does not make sense. The processing time based trigger policy should be to do process batches as fast as possible, but no faster than 1 in every trigger interval. If batches are taking longer than trigger interval anyways, then no point waiting extra trigger interval.

In this PR, I modified the ProcessingTimeExecutor to do so. Another minor change I did was to extract our StreamManualClock into a separate class so that it can be used outside subclasses of StreamTest. For example, ProcessingTimeExecutorSuite does not need to create any context for testing, just needs the StreamManualClock.

## How was this patch tested?
Added new unit tests to comprehensively test this behavior.

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

Closes #17525 from tdas/SPARK-20209.
2017-04-04 23:20:17 -07:00
Reynold Xin b6e71032d9 Small doc fix for ReuseSubquery. 2017-04-04 22:46:42 -07:00
Wenchen Fan 295747e597 [SPARK-19716][SQL] support by-name resolution for struct type elements in array
## What changes were proposed in this pull request?

Previously when we construct deserializer expression for array type, we will first cast the corresponding field to expected array type and then apply `MapObjects`.

However, by doing that, we lose the opportunity to do by-name resolution for struct type inside array type. In this PR, I introduce a `UnresolvedMapObjects` to hold the lambda function and the input array expression. Then during analysis, after the input array expression is resolved, we get the actual array element type and apply by-name resolution. Then we don't need to add `Cast` for array type when constructing the deserializer expression, as the element type is determined later at analyzer.

## How was this patch tested?

new regression test

Author: Wenchen Fan <wenchen@databricks.com>

Closes #17398 from cloud-fan/dataset.
2017-04-04 16:38:32 -07:00
Wenchen Fan 402bf2a50d [SPARK-20204][SQL] remove SimpleCatalystConf and CatalystConf type alias
## What changes were proposed in this pull request?

This is a follow-up of https://github.com/apache/spark/pull/17285 .

## How was this patch tested?

existing tests

Author: Wenchen Fan <wenchen@databricks.com>

Closes #17521 from cloud-fan/conf.
2017-04-04 11:56:21 -07:00
Xiao Li 26e7bca229 [SPARK-20198][SQL] Remove the inconsistency in table/function name conventions in SparkSession.Catalog APIs
### What changes were proposed in this pull request?
Observed by felixcheung , in `SparkSession`.`Catalog` APIs, we have different conventions/rules for table/function identifiers/names. Most APIs accept the qualified name (i.e., `databaseName`.`tableName` or `databaseName`.`functionName`). However, the following five APIs do not accept it.
- def listColumns(tableName: String): Dataset[Column]
- def getTable(tableName: String): Table
- def getFunction(functionName: String): Function
- def tableExists(tableName: String): Boolean
- def functionExists(functionName: String): Boolean

To make them consistent with the other Catalog APIs, this PR does the changes, updates the function/API comments and adds the `params` to clarify the inputs we allow.

### How was this patch tested?
Added the test cases .

Author: Xiao Li <gatorsmile@gmail.com>

Closes #17518 from gatorsmile/tableIdentifier.
2017-04-04 18:57:46 +08:00
Xiao Li 51d3c854c5 [SPARK-20067][SQL] Unify and Clean Up Desc Commands Using Catalog Interface
### What changes were proposed in this pull request?

This PR is to unify and clean up the outputs of `DESC EXTENDED/FORMATTED` and `SHOW TABLE EXTENDED` by moving the logics into the Catalog interface. The output formats are improved. We also add the missing attributes. It impacts the DDL commands like `SHOW TABLE EXTENDED`, `DESC EXTENDED` and `DESC FORMATTED`.

In addition, by following what we did in Dataset API `printSchema`, we can use `treeString` to show the schema in the more readable way.

Below is the current way:
```
Schema: STRUCT<`a`: STRING (nullable = true), `b`: INT (nullable = true), `c`: STRING (nullable = true), `d`: STRING (nullable = true)>
```
After the change, it should look like
```
Schema: root
 |-- a: string (nullable = true)
 |-- b: integer (nullable = true)
 |-- c: string (nullable = true)
 |-- d: string (nullable = true)
```

### How was this patch tested?
`describe.sql` and `show-tables.sql`

Author: Xiao Li <gatorsmile@gmail.com>

Closes #17394 from gatorsmile/descFollowUp.
2017-04-03 23:30:12 -07:00
Dilip Biswal 3bfb639cb7 [SPARK-10364][SQL] Support Parquet logical type TIMESTAMP_MILLIS
## What changes were proposed in this pull request?

**Description** from JIRA

The TimestampType in Spark SQL is of microsecond precision. Ideally, we should convert Spark SQL timestamp values into Parquet TIMESTAMP_MICROS. But unfortunately parquet-mr hasn't supported it yet.
For the read path, we should be able to read TIMESTAMP_MILLIS Parquet values and pad a 0 microsecond part to read values.
For the write path, currently we are writing timestamps as INT96, similar to Impala and Hive. One alternative is that, we can have a separate SQL option to let users be able to write Spark SQL timestamp values as TIMESTAMP_MILLIS. Of course, in this way the microsecond part will be truncated.
## How was this patch tested?

Added new tests in ParquetQuerySuite and ParquetIOSuite

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

Closes #15332 from dilipbiswal/parquet-time-millis.
2017-04-04 09:53:05 +09:00
Ron Hu e7877fd472 [SPARK-19408][SQL] filter estimation on two columns of same table
## What changes were proposed in this pull request?

In SQL queries, we also see predicate expressions involving two columns such as "column-1 (op) column-2" where column-1 and column-2 belong to same table. Note that, if column-1 and column-2 belong to different tables, then it is a join operator's work, NOT a filter operator's work.

This PR estimates filter selectivity on two columns of same table.  For example, multiple tpc-h queries have this predicate "WHERE l_commitdate < l_receiptdate"

## How was this patch tested?

We added 6 new test cases to test various logical predicates involving two columns of same table.

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

Author: Ron Hu <ron.hu@huawei.com>
Author: U-CHINA\r00754707 <r00754707@R00754707-SC04.china.huawei.com>

Closes #17415 from ron8hu/filterTwoColumns.
2017-04-03 17:27:12 -07:00
samelamin 58c9e6e77a [SPARK-20145] Fix range case insensitive bug in SQL
## What changes were proposed in this pull request?
Range in SQL should be case insensitive

## How was this patch tested?
unit test

Author: samelamin <hussam.elamin@gmail.com>
Author: samelamin <sam_elamin@discovery.com>

Closes #17487 from samelamin/SPARK-20145.
2017-04-03 17:16:31 -07:00
Adrian Ionescu 703c42c398 [SPARK-20194] Add support for partition pruning to in-memory catalog
## What changes were proposed in this pull request?
This patch implements `listPartitionsByFilter()` for `InMemoryCatalog` and thus resolves an outstanding TODO causing the `PruneFileSourcePartitions` optimizer rule not to apply when "spark.sql.catalogImplementation" is set to "in-memory" (which is the default).

The change is straightforward: it extracts the code for further filtering of the list of partitions returned by the metastore's `getPartitionsByFilter()` out from `HiveExternalCatalog` into `ExternalCatalogUtils` and calls this new function from `InMemoryCatalog` on the whole list of partitions.

Now that this method is implemented we can always pass the `CatalogTable` to the `DataSource` in `FindDataSourceTable`, so that the latter is resolved to a relation with a `CatalogFileIndex`, which is what the `PruneFileSourcePartitions` rule matches for.

## How was this patch tested?
Ran existing tests and added new test for `listPartitionsByFilter` in `ExternalCatalogSuite`, which is subclassed by both `InMemoryCatalogSuite` and `HiveExternalCatalogSuite`.

Author: Adrian Ionescu <adrian@databricks.com>

Closes #17510 from adrian-ionescu/InMemoryCatalog.
2017-04-03 08:48:49 -07:00
hyukjinkwon 4fa1a43af6 [SPARK-19641][SQL] JSON schema inference in DROPMALFORMED mode produces incorrect schema for non-array/object JSONs
## What changes were proposed in this pull request?

Currently, when we infer the types for vaild JSON strings but object or array, we are producing empty schemas regardless of parse modes as below:

```scala
scala> spark.read.option("mode", "DROPMALFORMED").json(Seq("""{"a": 1}""", """"a"""").toDS).printSchema()
root
```

```scala
scala> spark.read.option("mode", "FAILFAST").json(Seq("""{"a": 1}""", """"a"""").toDS).printSchema()
root
```

This PR proposes to handle parse modes in type inference.

After this PR,

```scala

scala> spark.read.option("mode", "DROPMALFORMED").json(Seq("""{"a": 1}""", """"a"""").toDS).printSchema()
root
 |-- a: long (nullable = true)
```

```
scala> spark.read.option("mode", "FAILFAST").json(Seq("""{"a": 1}""", """"a"""").toDS).printSchema()
java.lang.RuntimeException: Failed to infer a common schema. Struct types are expected but string was found.
```

This PR is based on e233fd0334 and I and NathanHowell talked about this in https://issues.apache.org/jira/browse/SPARK-19641

## How was this patch tested?

Unit tests in `JsonSuite` for both `DROPMALFORMED` and `FAILFAST` modes.

Author: hyukjinkwon <gurwls223@gmail.com>

Closes #17492 from HyukjinKwon/SPARK-19641.
2017-04-03 17:44:39 +08:00
hyukjinkwon cff11fd20e [SPARK-20166][SQL] Use XXX for ISO 8601 timezone instead of ZZ (FastDateFormat specific) in CSV/JSON timeformat options
## What changes were proposed in this pull request?

This PR proposes to use `XXX` format instead of `ZZ`. `ZZ` seems a `FastDateFormat` specific.

`ZZ` supports "ISO 8601 extended format time zones" but it seems `FastDateFormat` specific option.
I misunderstood this is compatible format with `SimpleDateFormat` when this change is introduced.
Please see [SimpleDateFormat documentation]( https://docs.oracle.com/javase/7/docs/api/java/text/SimpleDateFormat.html#iso8601timezone) and [FastDateFormat documentation](https://commons.apache.org/proper/commons-lang/apidocs/org/apache/commons/lang3/time/FastDateFormat.html).

It seems we better replace `ZZ` to `XXX` because they look using the same strategy - [FastDateParser.java#L930](8767cd4f1a/src/main/java/org/apache/commons/lang3/time/FastDateParser.java (L930)), [FastDateParser.java#L932-L951 ](8767cd4f1a/src/main/java/org/apache/commons/lang3/time/FastDateParser.java (L932-L951)) and [FastDateParser.java#L596-L601](8767cd4f1a/src/main/java/org/apache/commons/lang3/time/FastDateParser.java (L596-L601)).

I also checked the codes and manually debugged it for sure. It seems both cases use the same pattern `( Z|(?:[+-]\\d{2}(?::)\\d{2}))`.

_Note that this should be rather a fix about documentation and not the behaviour change because `ZZ` seems invalid date format in `SimpleDateFormat` as documented in `DataFrameReader` and etc, and both `ZZ` and `XXX` look identically working with `FastDateFormat`_

Current documentation is as below:

```
   * <li>`timestampFormat` (default `yyyy-MM-dd'T'HH:mm:ss.SSSZZ`): sets the string that
   * indicates a timestamp format. Custom date formats follow the formats at
   * `java.text.SimpleDateFormat`. This applies to timestamp type.</li>
```

## How was this patch tested?

Existing tests should cover this. Also, manually tested as below (BTW, I don't think these are worth being added as tests within Spark):

**Parse**

```scala
scala> new java.text.SimpleDateFormat("yyyy-MM-dd'T'HH:mm:ss.SSSXXX").parse("2017-03-21T00:00:00.000-11:00")
res4: java.util.Date = Tue Mar 21 20:00:00 KST 2017

scala>  new java.text.SimpleDateFormat("yyyy-MM-dd'T'HH:mm:ss.SSSXXX").parse("2017-03-21T00:00:00.000Z")
res10: java.util.Date = Tue Mar 21 09:00:00 KST 2017

scala> new java.text.SimpleDateFormat("yyyy-MM-dd'T'HH:mm:ss.SSSZZ").parse("2017-03-21T00:00:00.000-11:00")
java.text.ParseException: Unparseable date: "2017-03-21T00:00:00.000-11:00"
  at java.text.DateFormat.parse(DateFormat.java:366)
  ... 48 elided
scala>  new java.text.SimpleDateFormat("yyyy-MM-dd'T'HH:mm:ss.SSSZZ").parse("2017-03-21T00:00:00.000Z")
java.text.ParseException: Unparseable date: "2017-03-21T00:00:00.000Z"
  at java.text.DateFormat.parse(DateFormat.java:366)
  ... 48 elided
```

```scala
scala> org.apache.commons.lang3.time.FastDateFormat.getInstance("yyyy-MM-dd'T'HH:mm:ss.SSSXXX").parse("2017-03-21T00:00:00.000-11:00")
res7: java.util.Date = Tue Mar 21 20:00:00 KST 2017

scala> org.apache.commons.lang3.time.FastDateFormat.getInstance("yyyy-MM-dd'T'HH:mm:ss.SSSXXX").parse("2017-03-21T00:00:00.000Z")
res1: java.util.Date = Tue Mar 21 09:00:00 KST 2017

scala> org.apache.commons.lang3.time.FastDateFormat.getInstance("yyyy-MM-dd'T'HH:mm:ss.SSSZZ").parse("2017-03-21T00:00:00.000-11:00")
res8: java.util.Date = Tue Mar 21 20:00:00 KST 2017

scala> org.apache.commons.lang3.time.FastDateFormat.getInstance("yyyy-MM-dd'T'HH:mm:ss.SSSZZ").parse("2017-03-21T00:00:00.000Z")
res2: java.util.Date = Tue Mar 21 09:00:00 KST 2017
```

**Format**

```scala
scala> new java.text.SimpleDateFormat("yyyy-MM-dd'T'HH:mm:ss.SSSXXX").format(new java.text.SimpleDateFormat("yyyy-MM-dd'T'HH:mm:ss.SSSXXX").parse("2017-03-21T00:00:00.000-11:00"))
res6: String = 2017-03-21T20:00:00.000+09:00
```

```scala
scala> val fd = org.apache.commons.lang3.time.FastDateFormat.getInstance("yyyy-MM-dd'T'HH:mm:ss.SSSZZ")
fd: org.apache.commons.lang3.time.FastDateFormat = FastDateFormat[yyyy-MM-dd'T'HH:mm:ss.SSSZZ,ko_KR,Asia/Seoul]

scala> fd.format(fd.parse("2017-03-21T00:00:00.000-11:00"))
res1: String = 2017-03-21T20:00:00.000+09:00

scala> val fd = org.apache.commons.lang3.time.FastDateFormat.getInstance("yyyy-MM-dd'T'HH:mm:ss.SSSXXX")
fd: org.apache.commons.lang3.time.FastDateFormat = FastDateFormat[yyyy-MM-dd'T'HH:mm:ss.SSSXXX,ko_KR,Asia/Seoul]

scala> fd.format(fd.parse("2017-03-21T00:00:00.000-11:00"))
res2: String = 2017-03-21T20:00:00.000+09:00
```

Author: hyukjinkwon <gurwls223@gmail.com>

Closes #17489 from HyukjinKwon/SPARK-20166.
2017-04-03 10:07:41 +01:00
zuotingbing 657cb9541d [SPARK-20173][SQL][HIVE-THRIFTSERVER] Throw NullPointerException when HiveThriftServer2 is shutdown
## What changes were proposed in this pull request?

If the shutdown hook called before the variable `uiTab` is set , it will throw a NullPointerException.

## How was this patch tested?

manual tests

Author: zuotingbing <zuo.tingbing9@zte.com.cn>

Closes #17496 from zuotingbing/SPARK-HiveThriftServer2.
2017-04-02 15:39:51 +01:00
hyukjinkwon d40cbb8618 [SPARK-20143][SQL] DataType.fromJson should throw an exception with better message
## What changes were proposed in this pull request?

Currently, `DataType.fromJson` throws `scala.MatchError` or `java.util.NoSuchElementException` in some cases when the JSON input is invalid as below:

```scala
DataType.fromJson(""""abcd"""")
```

```
java.util.NoSuchElementException: key not found: abcd
  at ...
```

```scala
DataType.fromJson("""{"abcd":"a"}""")
```

```
scala.MatchError: JObject(List((abcd,JString(a)))) (of class org.json4s.JsonAST$JObject)
  at ...
```

```scala
DataType.fromJson("""{"fields": [{"a":123}], "type": "struct"}""")
```

```
scala.MatchError: JObject(List((a,JInt(123)))) (of class org.json4s.JsonAST$JObject)
  at ...
```

After this PR,

```scala
DataType.fromJson(""""abcd"""")
```

```
java.lang.IllegalArgumentException: Failed to convert the JSON string 'abcd' to a data type.
  at ...
```

```scala
DataType.fromJson("""{"abcd":"a"}""")
```

```
java.lang.IllegalArgumentException: Failed to convert the JSON string '{"abcd":"a"}' to a data type.
  at ...
```

```scala
DataType.fromJson("""{"fields": [{"a":123}], "type": "struct"}""")
  at ...
```

```
java.lang.IllegalArgumentException: Failed to convert the JSON string '{"a":123}' to a field.
```

## How was this patch tested?

Unit test added in `DataTypeSuite`.

Author: hyukjinkwon <gurwls223@gmail.com>

Closes #17468 from HyukjinKwon/fromjson_exception.
2017-04-02 07:26:49 -07:00
wangzhenhua 2287f3d0b8 [SPARK-20186][SQL] BroadcastHint should use child's stats
## What changes were proposed in this pull request?

`BroadcastHint` should use child's statistics and set `isBroadcastable` to true.

## How was this patch tested?

Added a new stats estimation test for `BroadcastHint`.

Author: wangzhenhua <wangzhenhua@huawei.com>

Closes #17504 from wzhfy/broadcastHintEstimation.
2017-04-01 22:19:08 +08:00
Xiao Li 89d6822f72 [SPARK-19148][SQL][FOLLOW-UP] do not expose the external table concept in Catalog
### What changes were proposed in this pull request?
After we renames `Catalog`.`createExternalTable` to `createTable` in the PR: https://github.com/apache/spark/pull/16528, we also need to deprecate the corresponding functions in `SQLContext`.

### How was this patch tested?
N/A

Author: Xiao Li <gatorsmile@gmail.com>

Closes #17502 from gatorsmile/deprecateCreateExternalTable.
2017-04-01 20:43:13 +08:00
Tathagata Das 567a50acfb [SPARK-20165][SS] Resolve state encoder's deserializer in driver in FlatMapGroupsWithStateExec
## What changes were proposed in this pull request?

- Encoder's deserializer must be resolved at the driver where the class is defined. Otherwise there are corner cases using nested classes where resolving at the executor can fail.

- Fixed flaky test related to processing time timeout. The flakiness is caused because the test thread (that adds data to memory source) has a race condition with the streaming query thread. When testing the manual clock, the goal is to add data and increment clock together atomically, such that a trigger sees new data AND updated clock simultaneously (both or none). This fix adds additional synchronization in when adding data; it makes sure that the streaming query thread is waiting on the manual clock to be incremented (so no batch is currently running) before adding data.

- Added`testQuietly` on some tests that generate a lot of error logs.

## How was this patch tested?
Multiple runs on existing unit tests

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

Closes #17488 from tdas/SPARK-20165.
2017-03-31 10:58:43 -07:00
Xiao Li b2349e6a00 [SPARK-20160][SQL] Move ParquetConversions and OrcConversions Out Of HiveSessionCatalog
### What changes were proposed in this pull request?
`ParquetConversions` and `OrcConversions` should be treated as regular `Analyzer` rules. It is not reasonable to be part of `HiveSessionCatalog`. This PR also combines two rules `ParquetConversions` and `OrcConversions` to build a new rule `RelationConversions `.

After moving these two rules out of HiveSessionCatalog, the next step is to clean up, rename and move `HiveMetastoreCatalog` because it is not related to the hive package any more.

### How was this patch tested?
The existing test cases

Author: Xiao Li <gatorsmile@gmail.com>

Closes #17484 from gatorsmile/cleanup.
2017-04-01 00:56:18 +08:00
Kunal Khamar 254877c2f0 [SPARK-20164][SQL] AnalysisException not tolerant of null query plan.
## What changes were proposed in this pull request?

The query plan in an `AnalysisException` may be `null` when an `AnalysisException` object is serialized and then deserialized, since `plan` is marked `transient`. Or when someone throws an `AnalysisException` with a null query plan (which should not happen).
`def getMessage` is not tolerant of this and throws a `NullPointerException`, leading to loss of information about the original exception.
The fix is to add a `null` check in `getMessage`.

## How was this patch tested?

- Unit test

Author: Kunal Khamar <kkhamar@outlook.com>

Closes #17486 from kunalkhamar/spark-20164.
2017-03-31 09:17:22 -07:00
Reynold Xin a8a765b3f3 [SPARK-20151][SQL] Account for partition pruning in scan metadataTime metrics
## What changes were proposed in this pull request?
After SPARK-20136, we report metadata timing metrics in scan operator. However, that timing metric doesn't include one of the most important part of metadata, which is partition pruning. This patch adds that time measurement to the scan metrics.

## How was this patch tested?
N/A - I tried adding a test in SQLMetricsSuite but it was extremely convoluted to the point that I'm not sure if this is worth it.

Author: Reynold Xin <rxin@databricks.com>

Closes #17476 from rxin/SPARK-20151.
2017-03-30 23:09:33 -07:00
Wenchen Fan c734fc504a [SPARK-20121][SQL] simplify NullPropagation with NullIntolerant
## What changes were proposed in this pull request?

Instead of iterating all expressions that can return null for null inputs, we can just check `NullIntolerant`.

## How was this patch tested?

existing tests

Author: Wenchen Fan <wenchen@databricks.com>

Closes #17450 from cloud-fan/null.
2017-03-30 13:36:36 -07:00
Jacek Laskowski 0197262a35 [DOCS] Docs-only improvements
…adoc

## What changes were proposed in this pull request?

Use recommended values for row boundaries in Window's scaladoc, i.e. `Window.unboundedPreceding`, `Window.unboundedFollowing`, and `Window.currentRow` (that were introduced in 2.1.0).

## How was this patch tested?

Local build

Author: Jacek Laskowski <jacek@japila.pl>

Closes #17417 from jaceklaskowski/window-expression-scaladoc.
2017-03-30 16:07:27 +01:00
Eric Liang 79636054f6 [SPARK-20148][SQL] Extend the file commit API to allow subscribing to task commit messages
## What changes were proposed in this pull request?

The internal FileCommitProtocol interface returns all task commit messages in bulk to the implementation when a job finishes. However, it is sometimes useful to access those messages before the job completes, so that the driver gets incremental progress updates before the job finishes.

This adds an `onTaskCommit` listener to the internal api.

## How was this patch tested?

Unit tests.

cc rxin

Author: Eric Liang <ekl@databricks.com>

Closes #17475 from ericl/file-commit-api-ext.
2017-03-29 20:59:48 -07:00
Reynold Xin 60977889ea [SPARK-20136][SQL] Add num files and metadata operation timing to scan operator metrics
## What changes were proposed in this pull request?
This patch adds explicit metadata operation timing and number of files in data source metrics. Those would be useful to include for performance profiling.

Screenshot of a UI with this change (num files and metadata time are new metrics):

<img width="321" alt="screen shot 2017-03-29 at 12 29 28 am" src="https://cloud.githubusercontent.com/assets/323388/24443272/d4ea58c0-1416-11e7-8940-ecb69375554a.png">

## How was this patch tested?
N/A

Author: Reynold Xin <rxin@databricks.com>

Closes #17465 from rxin/SPARK-20136.
2017-03-29 19:06:51 -07:00
bomeng 22f07fefe1 [SPARK-20146][SQL] fix comment missing issue for thrift server
## What changes were proposed in this pull request?

The column comment was missing while constructing the Hive TableSchema. This fix will preserve the original comment.

## How was this patch tested?

I have added a new test case to test the column with/without comment.

Author: bomeng <bmeng@us.ibm.com>

Closes #17470 from bomeng/SPARK-20146.
2017-03-29 18:57:35 -07:00
Takuya UESHIN dd2e7d528c [SPARK-19088][SQL] Fix 2.10 build.
## What changes were proposed in this pull request?

Commit 6c70a38 broke the build for scala 2.10. The commit uses some reflections which are not available in Scala 2.10. This PR fixes them.

## How was this patch tested?

Existing tests.

Author: Takuya UESHIN <ueshin@databricks.com>

Closes #17473 from ueshin/issues/SPARK-19088.
2017-03-29 17:32:01 -07:00
Yuming Wang fe1d6b05d4 [SPARK-20120][SQL] spark-sql support silent mode
## What changes were proposed in this pull request?

It is similar to Hive silent mode, just show the query result. see: [Hive LanguageManual+Cli](https://cwiki.apache.org/confluence/display/Hive/LanguageManual+Cli) and [the implementation of Hive silent mode](https://github.com/apache/hive/blob/release-1.2.1/ql/src/java/org/apache/hadoop/hive/ql/session/SessionState.java#L948-L950).

This PR set the Logger level to `WARN` to get similar result.

## How was this patch tested?

manual tests

![manual test spark sql silent mode](https://cloud.githubusercontent.com/assets/5399861/24390165/989b7780-13b9-11e7-8496-6e68f55757e3.gif)

Author: Yuming Wang <wgyumg@gmail.com>

Closes #17449 from wangyum/SPARK-20120.
2017-03-29 15:23:24 -07:00
Xiao Li 5c8ef376e8 [SPARK-17075][SQL][FOLLOWUP] Add Estimation of Constant Literal
### What changes were proposed in this pull request?
`FalseLiteral` and `TrueLiteral` should have been eliminated by optimizer rule `BooleanSimplification`, but null literals might be added by optimizer rule `NullPropagation`. For safety, our filter estimation should handle all the eligible literal cases.

Our optimizer rule BooleanSimplification is unable to remove the null literal in many cases. For example, `a < 0 or null`. Thus, we need to handle null literal in filter estimation.

`Not` can be pushed down below `And` and `Or`. Then, we could see two consecutive `Not`, which need to be collapsed into one. Because of the limited expression support for filter estimation, we just need to handle the case `Not(null)` for avoiding incorrect error due to the boolean operation on null. For details, see below matrix.

```
not NULL = NULL
NULL or false = NULL
NULL or true = true
NULL or NULL = NULL
NULL and false = false
NULL and true = NULL
NULL and NULL = NULL
```
### How was this patch tested?
Added the test cases.

Author: Xiao Li <gatorsmile@gmail.com>

Closes #17446 from gatorsmile/constantFilterEstimation.
2017-03-29 12:43:22 -07:00
Takeshi Yamamuro c4008480b7 [SPARK-20009][SQL] Support DDL strings for defining schema in functions.from_json
## What changes were proposed in this pull request?
This pr added `StructType.fromDDL`  to convert a DDL format string into `StructType` for defining schemas in `functions.from_json`.

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

Author: Takeshi Yamamuro <yamamuro@apache.org>

Closes #17406 from maropu/SPARK-20009.
2017-03-29 12:37:49 -07:00
Kunal Khamar 142f6d1492 [SPARK-20048][SQL] Cloning SessionState does not clone query execution listeners
## What changes were proposed in this pull request?

Bugfix from [SPARK-19540.](https://github.com/apache/spark/pull/16826)
Cloning SessionState does not clone query execution listeners, so cloned session is unable to listen to events on queries.

## How was this patch tested?

- Unit test

Author: Kunal Khamar <kkhamar@outlook.com>

Closes #17379 from kunalkhamar/clone-bugfix.
2017-03-29 12:35:19 -07:00
Reynold Xin 9712bd3954 [SPARK-20134][SQL] SQLMetrics.postDriverMetricUpdates to simplify driver side metric updates
## What changes were proposed in this pull request?
It is not super intuitive how to update SQLMetric on the driver side. This patch introduces a new SQLMetrics.postDriverMetricUpdates function to do that, and adds documentation to make it more obvious.

## How was this patch tested?
Updated a test case to use this method.

Author: Reynold Xin <rxin@databricks.com>

Closes #17464 from rxin/SPARK-20134.
2017-03-29 00:02:15 -07:00
Wenchen Fan d4fac410e0 [SPARK-20125][SQL] Dataset of type option of map does not work
## What changes were proposed in this pull request?

When we build the deserializer expression for map type, we will use `StaticInvoke` to call `ArrayBasedMapData.toScalaMap`, and declare the return type as `scala.collection.immutable.Map`. If the map is inside an Option, we will wrap this `StaticInvoke` with `WrapOption`, which requires the input to be `scala.collect.Map`. Ideally this should be fine, as `scala.collection.immutable.Map` extends `scala.collect.Map`, but our `ObjectType` is too strict about this, this PR fixes it.

## How was this patch tested?

new regression test

Author: Wenchen Fan <wenchen@databricks.com>

Closes #17454 from cloud-fan/map.
2017-03-28 11:47:43 -07:00
Herman van Hovell f82461fc11 [SPARK-20126][SQL] Remove HiveSessionState
## What changes were proposed in this pull request?
Commit ea361165e1 moved most of the logic from the SessionState classes into an accompanying builder. This makes the existence of the `HiveSessionState` redundant. This PR removes the `HiveSessionState`.

## How was this patch tested?
Existing tests.

Author: Herman van Hovell <hvanhovell@databricks.com>

Closes #17457 from hvanhovell/SPARK-20126.
2017-03-28 23:14:31 +08:00
wangzhenhua 4fcc214d9e [SPARK-20124][SQL] Join reorder should keep the same order of final project attributes
## What changes were proposed in this pull request?

Join reorder algorithm should keep exactly the same order of output attributes in the top project.
For example, if user want to select a, b, c, after reordering, we should output a, b, c in the same order as specified by user, instead of b, a, c or other orders.

## How was this patch tested?

A new test case is added in `JoinReorderSuite`.

Author: wangzhenhua <wangzhenhua@huawei.com>

Closes #17453 from wzhfy/keepOrderInProject.
2017-03-28 22:22:38 +08:00
wangzhenhua 91559d277f [SPARK-20094][SQL] Preventing push down of IN subquery to Join operator
## What changes were proposed in this pull request?

TPCDS q45 fails becuase:
`ReorderJoin` collects all predicates and try to put them into join condition when creating ordered join. If a predicate with an IN subquery (`ListQuery`) is in a join condition instead of a filter condition, `RewritePredicateSubquery.rewriteExistentialExpr` would fail to convert the subquery to an `ExistenceJoin`, and thus result in error.

We should prevent push down of IN subquery to Join operator.

## How was this patch tested?

Add a new test case in `FilterPushdownSuite`.

Author: wangzhenhua <wangzhenhua@huawei.com>

Closes #17428 from wzhfy/noSubqueryInJoinCond.
2017-03-28 13:43:23 +02:00
Xiao Li a9abff281b [SPARK-20119][TEST-MAVEN] Fix the test case fail in DataSourceScanExecRedactionSuite
### What changes were proposed in this pull request?
Changed the pattern to match the first n characters in the location field so that the string truncation does not affect it.

### How was this patch tested?
N/A

Author: Xiao Li <gatorsmile@gmail.com>

Closes #17448 from gatorsmile/fixTestCAse.
2017-03-28 09:37:28 +02:00
Michal Senkyr 6c70a38c2e [SPARK-19088][SQL] Optimize sequence type deserialization codegen
## What changes were proposed in this pull request?

Optimization of arbitrary Scala sequence deserialization introduced by #16240.

The previous implementation constructed an array which was then converted by `to`. This required two passes in most cases.

This implementation attempts to remedy that by using `Builder`s provided by the `newBuilder` method on every Scala collection's companion object to build the resulting collection directly.

Example codegen for simple `List` (obtained using `Seq(List(1)).toDS().map(identity).queryExecution.debug.codegen`):

Before:

```
/* 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 inputadapter_input;
/* 009 */   private boolean deserializetoobject_resultIsNull;
/* 010 */   private java.lang.Object[] deserializetoobject_argValue;
/* 011 */   private boolean MapObjects_loopIsNull1;
/* 012 */   private int MapObjects_loopValue0;
/* 013 */   private boolean deserializetoobject_resultIsNull1;
/* 014 */   private scala.collection.generic.CanBuildFrom deserializetoobject_argValue1;
/* 015 */   private UnsafeRow deserializetoobject_result;
/* 016 */   private org.apache.spark.sql.catalyst.expressions.codegen.BufferHolder deserializetoobject_holder;
/* 017 */   private org.apache.spark.sql.catalyst.expressions.codegen.UnsafeRowWriter deserializetoobject_rowWriter;
/* 018 */   private scala.collection.immutable.List mapelements_argValue;
/* 019 */   private UnsafeRow mapelements_result;
/* 020 */   private org.apache.spark.sql.catalyst.expressions.codegen.BufferHolder mapelements_holder;
/* 021 */   private org.apache.spark.sql.catalyst.expressions.codegen.UnsafeRowWriter mapelements_rowWriter;
/* 022 */   private scala.collection.immutable.List serializefromobject_argValue;
/* 023 */   private UnsafeRow serializefromobject_result;
/* 024 */   private org.apache.spark.sql.catalyst.expressions.codegen.BufferHolder serializefromobject_holder;
/* 025 */   private org.apache.spark.sql.catalyst.expressions.codegen.UnsafeRowWriter serializefromobject_rowWriter;
/* 026 */   private org.apache.spark.sql.catalyst.expressions.codegen.UnsafeArrayWriter serializefromobject_arrayWriter;
/* 027 */
/* 028 */   public GeneratedIterator(Object[] references) {
/* 029 */     this.references = references;
/* 030 */   }
/* 031 */
/* 032 */   public void init(int index, scala.collection.Iterator[] inputs) {
/* 033 */     partitionIndex = index;
/* 034 */     this.inputs = inputs;
/* 035 */     inputadapter_input = inputs[0];
/* 036 */
/* 037 */     deserializetoobject_result = new UnsafeRow(1);
/* 038 */     this.deserializetoobject_holder = new org.apache.spark.sql.catalyst.expressions.codegen.BufferHolder(deserializetoobject_result, 32);
/* 039 */     this.deserializetoobject_rowWriter = new org.apache.spark.sql.catalyst.expressions.codegen.UnsafeRowWriter(deserializetoobject_holder, 1);
/* 040 */
/* 041 */     mapelements_result = new UnsafeRow(1);
/* 042 */     this.mapelements_holder = new org.apache.spark.sql.catalyst.expressions.codegen.BufferHolder(mapelements_result, 32);
/* 043 */     this.mapelements_rowWriter = new org.apache.spark.sql.catalyst.expressions.codegen.UnsafeRowWriter(mapelements_holder, 1);
/* 044 */
/* 045 */     serializefromobject_result = new UnsafeRow(1);
/* 046 */     this.serializefromobject_holder = new org.apache.spark.sql.catalyst.expressions.codegen.BufferHolder(serializefromobject_result, 32);
/* 047 */     this.serializefromobject_rowWriter = new org.apache.spark.sql.catalyst.expressions.codegen.UnsafeRowWriter(serializefromobject_holder, 1);
/* 048 */     this.serializefromobject_arrayWriter = new org.apache.spark.sql.catalyst.expressions.codegen.UnsafeArrayWriter();
/* 049 */
/* 050 */   }
/* 051 */
/* 052 */   protected void processNext() throws java.io.IOException {
/* 053 */     while (inputadapter_input.hasNext() && !stopEarly()) {
/* 054 */       InternalRow inputadapter_row = (InternalRow) inputadapter_input.next();
/* 055 */       ArrayData inputadapter_value = inputadapter_row.getArray(0);
/* 056 */
/* 057 */       deserializetoobject_resultIsNull = false;
/* 058 */
/* 059 */       if (!deserializetoobject_resultIsNull) {
/* 060 */         ArrayData deserializetoobject_value3 = null;
/* 061 */
/* 062 */         if (!false) {
/* 063 */           Integer[] deserializetoobject_convertedArray = null;
/* 064 */           int deserializetoobject_dataLength = inputadapter_value.numElements();
/* 065 */           deserializetoobject_convertedArray = new Integer[deserializetoobject_dataLength];
/* 066 */
/* 067 */           int deserializetoobject_loopIndex = 0;
/* 068 */           while (deserializetoobject_loopIndex < deserializetoobject_dataLength) {
/* 069 */             MapObjects_loopValue0 = (int) (inputadapter_value.getInt(deserializetoobject_loopIndex));
/* 070 */             MapObjects_loopIsNull1 = inputadapter_value.isNullAt(deserializetoobject_loopIndex);
/* 071 */
/* 072 */             if (MapObjects_loopIsNull1) {
/* 073 */               throw new RuntimeException(((java.lang.String) references[0]));
/* 074 */             }
/* 075 */             if (false) {
/* 076 */               deserializetoobject_convertedArray[deserializetoobject_loopIndex] = null;
/* 077 */             } else {
/* 078 */               deserializetoobject_convertedArray[deserializetoobject_loopIndex] = MapObjects_loopValue0;
/* 079 */             }
/* 080 */
/* 081 */             deserializetoobject_loopIndex += 1;
/* 082 */           }
/* 083 */
/* 084 */           deserializetoobject_value3 = new org.apache.spark.sql.catalyst.util.GenericArrayData(deserializetoobject_convertedArray);
/* 085 */         }
/* 086 */         boolean deserializetoobject_isNull2 = true;
/* 087 */         java.lang.Object[] deserializetoobject_value2 = null;
/* 088 */         if (!false) {
/* 089 */           deserializetoobject_isNull2 = false;
/* 090 */           if (!deserializetoobject_isNull2) {
/* 091 */             Object deserializetoobject_funcResult = null;
/* 092 */             deserializetoobject_funcResult = deserializetoobject_value3.array();
/* 093 */             if (deserializetoobject_funcResult == null) {
/* 094 */               deserializetoobject_isNull2 = true;
/* 095 */             } else {
/* 096 */               deserializetoobject_value2 = (java.lang.Object[]) deserializetoobject_funcResult;
/* 097 */             }
/* 098 */
/* 099 */           }
/* 100 */           deserializetoobject_isNull2 = deserializetoobject_value2 == null;
/* 101 */         }
/* 102 */         deserializetoobject_resultIsNull = deserializetoobject_isNull2;
/* 103 */         deserializetoobject_argValue = deserializetoobject_value2;
/* 104 */       }
/* 105 */
/* 106 */       boolean deserializetoobject_isNull1 = deserializetoobject_resultIsNull;
/* 107 */       final scala.collection.Seq deserializetoobject_value1 = deserializetoobject_resultIsNull ? null : scala.collection.mutable.WrappedArray.make(deserializetoobject_argValue);
/* 108 */       deserializetoobject_isNull1 = deserializetoobject_value1 == null;
/* 109 */       boolean deserializetoobject_isNull = true;
/* 110 */       scala.collection.immutable.List deserializetoobject_value = null;
/* 111 */       if (!deserializetoobject_isNull1) {
/* 112 */         deserializetoobject_resultIsNull1 = false;
/* 113 */
/* 114 */         if (!deserializetoobject_resultIsNull1) {
/* 115 */           boolean deserializetoobject_isNull6 = false;
/* 116 */           final scala.collection.generic.CanBuildFrom deserializetoobject_value6 = false ? null : scala.collection.immutable.List.canBuildFrom();
/* 117 */           deserializetoobject_isNull6 = deserializetoobject_value6 == null;
/* 118 */           deserializetoobject_resultIsNull1 = deserializetoobject_isNull6;
/* 119 */           deserializetoobject_argValue1 = deserializetoobject_value6;
/* 120 */         }
/* 121 */
/* 122 */         deserializetoobject_isNull = deserializetoobject_resultIsNull1;
/* 123 */         if (!deserializetoobject_isNull) {
/* 124 */           Object deserializetoobject_funcResult1 = null;
/* 125 */           deserializetoobject_funcResult1 = deserializetoobject_value1.to(deserializetoobject_argValue1);
/* 126 */           if (deserializetoobject_funcResult1 == null) {
/* 127 */             deserializetoobject_isNull = true;
/* 128 */           } else {
/* 129 */             deserializetoobject_value = (scala.collection.immutable.List) deserializetoobject_funcResult1;
/* 130 */           }
/* 131 */
/* 132 */         }
/* 133 */         deserializetoobject_isNull = deserializetoobject_value == null;
/* 134 */       }
/* 135 */
/* 136 */       boolean mapelements_isNull = true;
/* 137 */       scala.collection.immutable.List mapelements_value = null;
/* 138 */       if (!false) {
/* 139 */         mapelements_argValue = deserializetoobject_value;
/* 140 */
/* 141 */         mapelements_isNull = false;
/* 142 */         if (!mapelements_isNull) {
/* 143 */           Object mapelements_funcResult = null;
/* 144 */           mapelements_funcResult = ((scala.Function1) references[1]).apply(mapelements_argValue);
/* 145 */           if (mapelements_funcResult == null) {
/* 146 */             mapelements_isNull = true;
/* 147 */           } else {
/* 148 */             mapelements_value = (scala.collection.immutable.List) mapelements_funcResult;
/* 149 */           }
/* 150 */
/* 151 */         }
/* 152 */         mapelements_isNull = mapelements_value == null;
/* 153 */       }
/* 154 */
/* 155 */       if (mapelements_isNull) {
/* 156 */         throw new RuntimeException(((java.lang.String) references[2]));
/* 157 */       }
/* 158 */       serializefromobject_argValue = mapelements_value;
/* 159 */
/* 160 */       final ArrayData serializefromobject_value = false ? null : new org.apache.spark.sql.catalyst.util.GenericArrayData(serializefromobject_argValue);
/* 161 */       serializefromobject_holder.reset();
/* 162 */
/* 163 */       // Remember the current cursor so that we can calculate how many bytes are
/* 164 */       // written later.
/* 165 */       final int serializefromobject_tmpCursor = serializefromobject_holder.cursor;
/* 166 */
/* 167 */       if (serializefromobject_value instanceof UnsafeArrayData) {
/* 168 */         final int serializefromobject_sizeInBytes = ((UnsafeArrayData) serializefromobject_value).getSizeInBytes();
/* 169 */         // grow the global buffer before writing data.
/* 170 */         serializefromobject_holder.grow(serializefromobject_sizeInBytes);
/* 171 */         ((UnsafeArrayData) serializefromobject_value).writeToMemory(serializefromobject_holder.buffer, serializefromobject_holder.cursor);
/* 172 */         serializefromobject_holder.cursor += serializefromobject_sizeInBytes;
/* 173 */
/* 174 */       } else {
/* 175 */         final int serializefromobject_numElements = serializefromobject_value.numElements();
/* 176 */         serializefromobject_arrayWriter.initialize(serializefromobject_holder, serializefromobject_numElements, 4);
/* 177 */
/* 178 */         for (int serializefromobject_index = 0; serializefromobject_index < serializefromobject_numElements; serializefromobject_index++) {
/* 179 */           if (serializefromobject_value.isNullAt(serializefromobject_index)) {
/* 180 */             serializefromobject_arrayWriter.setNullInt(serializefromobject_index);
/* 181 */           } else {
/* 182 */             final int serializefromobject_element = serializefromobject_value.getInt(serializefromobject_index);
/* 183 */             serializefromobject_arrayWriter.write(serializefromobject_index, serializefromobject_element);
/* 184 */           }
/* 185 */         }
/* 186 */       }
/* 187 */
/* 188 */       serializefromobject_rowWriter.setOffsetAndSize(0, serializefromobject_tmpCursor, serializefromobject_holder.cursor - serializefromobject_tmpCursor);
/* 189 */       serializefromobject_result.setTotalSize(serializefromobject_holder.totalSize());
/* 190 */       append(serializefromobject_result);
/* 191 */       if (shouldStop()) return;
/* 192 */     }
/* 193 */   }
/* 194 */ }
```

After:

```
/* 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 inputadapter_input;
/* 009 */   private boolean CollectObjects_loopIsNull1;
/* 010 */   private int CollectObjects_loopValue0;
/* 011 */   private UnsafeRow deserializetoobject_result;
/* 012 */   private org.apache.spark.sql.catalyst.expressions.codegen.BufferHolder deserializetoobject_holder;
/* 013 */   private org.apache.spark.sql.catalyst.expressions.codegen.UnsafeRowWriter deserializetoobject_rowWriter;
/* 014 */   private scala.collection.immutable.List mapelements_argValue;
/* 015 */   private UnsafeRow mapelements_result;
/* 016 */   private org.apache.spark.sql.catalyst.expressions.codegen.BufferHolder mapelements_holder;
/* 017 */   private org.apache.spark.sql.catalyst.expressions.codegen.UnsafeRowWriter mapelements_rowWriter;
/* 018 */   private scala.collection.immutable.List serializefromobject_argValue;
/* 019 */   private UnsafeRow serializefromobject_result;
/* 020 */   private org.apache.spark.sql.catalyst.expressions.codegen.BufferHolder serializefromobject_holder;
/* 021 */   private org.apache.spark.sql.catalyst.expressions.codegen.UnsafeRowWriter serializefromobject_rowWriter;
/* 022 */   private org.apache.spark.sql.catalyst.expressions.codegen.UnsafeArrayWriter serializefromobject_arrayWriter;
/* 023 */
/* 024 */   public GeneratedIterator(Object[] references) {
/* 025 */     this.references = references;
/* 026 */   }
/* 027 */
/* 028 */   public void init(int index, scala.collection.Iterator[] inputs) {
/* 029 */     partitionIndex = index;
/* 030 */     this.inputs = inputs;
/* 031 */     inputadapter_input = inputs[0];
/* 032 */
/* 033 */     deserializetoobject_result = new UnsafeRow(1);
/* 034 */     this.deserializetoobject_holder = new org.apache.spark.sql.catalyst.expressions.codegen.BufferHolder(deserializetoobject_result, 32);
/* 035 */     this.deserializetoobject_rowWriter = new org.apache.spark.sql.catalyst.expressions.codegen.UnsafeRowWriter(deserializetoobject_holder, 1);
/* 036 */
/* 037 */     mapelements_result = new UnsafeRow(1);
/* 038 */     this.mapelements_holder = new org.apache.spark.sql.catalyst.expressions.codegen.BufferHolder(mapelements_result, 32);
/* 039 */     this.mapelements_rowWriter = new org.apache.spark.sql.catalyst.expressions.codegen.UnsafeRowWriter(mapelements_holder, 1);
/* 040 */
/* 041 */     serializefromobject_result = new UnsafeRow(1);
/* 042 */     this.serializefromobject_holder = new org.apache.spark.sql.catalyst.expressions.codegen.BufferHolder(serializefromobject_result, 32);
/* 043 */     this.serializefromobject_rowWriter = new org.apache.spark.sql.catalyst.expressions.codegen.UnsafeRowWriter(serializefromobject_holder, 1);
/* 044 */     this.serializefromobject_arrayWriter = new org.apache.spark.sql.catalyst.expressions.codegen.UnsafeArrayWriter();
/* 045 */
/* 046 */   }
/* 047 */
/* 048 */   protected void processNext() throws java.io.IOException {
/* 049 */     while (inputadapter_input.hasNext() && !stopEarly()) {
/* 050 */       InternalRow inputadapter_row = (InternalRow) inputadapter_input.next();
/* 051 */       ArrayData inputadapter_value = inputadapter_row.getArray(0);
/* 052 */
/* 053 */       scala.collection.immutable.List deserializetoobject_value = null;
/* 054 */
/* 055 */       if (!false) {
/* 056 */         int deserializetoobject_dataLength = inputadapter_value.numElements();
/* 057 */         scala.collection.mutable.Builder CollectObjects_builderValue2 = scala.collection.immutable.List$.MODULE$.newBuilder();
/* 058 */         CollectObjects_builderValue2.sizeHint(deserializetoobject_dataLength);
/* 059 */
/* 060 */         int deserializetoobject_loopIndex = 0;
/* 061 */         while (deserializetoobject_loopIndex < deserializetoobject_dataLength) {
/* 062 */           CollectObjects_loopValue0 = (int) (inputadapter_value.getInt(deserializetoobject_loopIndex));
/* 063 */           CollectObjects_loopIsNull1 = inputadapter_value.isNullAt(deserializetoobject_loopIndex);
/* 064 */
/* 065 */           if (CollectObjects_loopIsNull1) {
/* 066 */             throw new RuntimeException(((java.lang.String) references[0]));
/* 067 */           }
/* 068 */           if (false) {
/* 069 */             CollectObjects_builderValue2.$plus$eq(null);
/* 070 */           } else {
/* 071 */             CollectObjects_builderValue2.$plus$eq(CollectObjects_loopValue0);
/* 072 */           }
/* 073 */
/* 074 */           deserializetoobject_loopIndex += 1;
/* 075 */         }
/* 076 */
/* 077 */         deserializetoobject_value = (scala.collection.immutable.List) CollectObjects_builderValue2.result();
/* 078 */       }
/* 079 */
/* 080 */       boolean mapelements_isNull = true;
/* 081 */       scala.collection.immutable.List mapelements_value = null;
/* 082 */       if (!false) {
/* 083 */         mapelements_argValue = deserializetoobject_value;
/* 084 */
/* 085 */         mapelements_isNull = false;
/* 086 */         if (!mapelements_isNull) {
/* 087 */           Object mapelements_funcResult = null;
/* 088 */           mapelements_funcResult = ((scala.Function1) references[1]).apply(mapelements_argValue);
/* 089 */           if (mapelements_funcResult == null) {
/* 090 */             mapelements_isNull = true;
/* 091 */           } else {
/* 092 */             mapelements_value = (scala.collection.immutable.List) mapelements_funcResult;
/* 093 */           }
/* 094 */
/* 095 */         }
/* 096 */         mapelements_isNull = mapelements_value == null;
/* 097 */       }
/* 098 */
/* 099 */       if (mapelements_isNull) {
/* 100 */         throw new RuntimeException(((java.lang.String) references[2]));
/* 101 */       }
/* 102 */       serializefromobject_argValue = mapelements_value;
/* 103 */
/* 104 */       final ArrayData serializefromobject_value = false ? null : new org.apache.spark.sql.catalyst.util.GenericArrayData(serializefromobject_argValue);
/* 105 */       serializefromobject_holder.reset();
/* 106 */
/* 107 */       // Remember the current cursor so that we can calculate how many bytes are
/* 108 */       // written later.
/* 109 */       final int serializefromobject_tmpCursor = serializefromobject_holder.cursor;
/* 110 */
/* 111 */       if (serializefromobject_value instanceof UnsafeArrayData) {
/* 112 */         final int serializefromobject_sizeInBytes = ((UnsafeArrayData) serializefromobject_value).getSizeInBytes();
/* 113 */         // grow the global buffer before writing data.
/* 114 */         serializefromobject_holder.grow(serializefromobject_sizeInBytes);
/* 115 */         ((UnsafeArrayData) serializefromobject_value).writeToMemory(serializefromobject_holder.buffer, serializefromobject_holder.cursor);
/* 116 */         serializefromobject_holder.cursor += serializefromobject_sizeInBytes;
/* 117 */
/* 118 */       } else {
/* 119 */         final int serializefromobject_numElements = serializefromobject_value.numElements();
/* 120 */         serializefromobject_arrayWriter.initialize(serializefromobject_holder, serializefromobject_numElements, 4);
/* 121 */
/* 122 */         for (int serializefromobject_index = 0; serializefromobject_index < serializefromobject_numElements; serializefromobject_index++) {
/* 123 */           if (serializefromobject_value.isNullAt(serializefromobject_index)) {
/* 124 */             serializefromobject_arrayWriter.setNullInt(serializefromobject_index);
/* 125 */           } else {
/* 126 */             final int serializefromobject_element = serializefromobject_value.getInt(serializefromobject_index);
/* 127 */             serializefromobject_arrayWriter.write(serializefromobject_index, serializefromobject_element);
/* 128 */           }
/* 129 */         }
/* 130 */       }
/* 131 */
/* 132 */       serializefromobject_rowWriter.setOffsetAndSize(0, serializefromobject_tmpCursor, serializefromobject_holder.cursor - serializefromobject_tmpCursor);
/* 133 */       serializefromobject_result.setTotalSize(serializefromobject_holder.totalSize());
/* 134 */       append(serializefromobject_result);
/* 135 */       if (shouldStop()) return;
/* 136 */     }
/* 137 */   }
/* 138 */ }
```

Benchmark results before:

```
OpenJDK 64-Bit Server VM 1.8.0_112-b15 on Linux 4.8.13-1-ARCH
AMD A10-4600M APU with Radeon(tm) HD Graphics
collect:                                 Best/Avg Time(ms)    Rate(M/s)   Per Row(ns)   Relative
------------------------------------------------------------------------------------------------
Seq                                            269 /  370          0.0      269125.8       1.0X
List                                           154 /  176          0.0      154453.5       1.7X
mutable.Queue                                  210 /  233          0.0      209691.6       1.3X
```

Benchmark results after:

```
OpenJDK 64-Bit Server VM 1.8.0_112-b15 on Linux 4.8.13-1-ARCH
AMD A10-4600M APU with Radeon(tm) HD Graphics
collect:                                 Best/Avg Time(ms)    Rate(M/s)   Per Row(ns)   Relative
------------------------------------------------------------------------------------------------
Seq                                            255 /  316          0.0      254697.3       1.0X
List                                           152 /  177          0.0      152410.0       1.7X
mutable.Queue                                  213 /  235          0.0      213470.0       1.2X
```

## How was this patch tested?

```bash
./build/mvn -DskipTests clean package && ./dev/run-tests
```

Additionally in Spark Shell:

```scala
case class QueueClass(q: scala.collection.immutable.Queue[Int])

spark.createDataset(Seq(List(1,2,3))).map(x => QueueClass(scala.collection.immutable.Queue(x: _*))).map(_.q.dequeue).collect
```

Author: Michal Senkyr <mike.senkyr@gmail.com>

Closes #16541 from michalsenkyr/dataset-seq-builder.
2017-03-28 10:09:49 +08:00
Herman van Hovell ea361165e1 [SPARK-20100][SQL] Refactor SessionState initialization
## What changes were proposed in this pull request?
The current SessionState initialization code path is quite complex. A part of the creation is done in the SessionState companion objects, a part of the creation is one inside the SessionState class, and a part is done by passing functions.

This PR refactors this code path, and consolidates SessionState initialization into a builder class. This SessionState will not do any initialization and just becomes a place holder for the various Spark SQL internals. This also lays the ground work for two future improvements:

1. This provides us with a start for removing the `HiveSessionState`. Removing the `HiveSessionState` would also require us to move resource loading into a separate class, and to (re)move metadata hive.
2. This makes it easier to customize the Spark Session. Currently you will need to create a custom version of the builder. I have added hooks to facilitate this. A future step will be to create a semi stable API on top of this.

## How was this patch tested?
Existing tests.

Author: Herman van Hovell <hvanhovell@databricks.com>

Closes #17433 from hvanhovell/SPARK-20100.
2017-03-28 10:07:24 +08:00
Tathagata Das 8a6f33f048 [SPARK-19876][SS] Follow up: Refactored BatchCommitLog to simplify logic
## What changes were proposed in this pull request?

Existing logic seemingly writes null to the BatchCommitLog, even though it does additional checks to write '{}' (valid json) to the log. This PR simplifies the logic by disallowing use of `log.add(batchId, metadata)` and instead using `log.add(batchId)`. No question of specifying metadata, so no confusion related to null.

## How was this patch tested?
Existing tests pass.

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

Closes #17444 from tdas/SPARK-19876-1.
2017-03-27 19:04:16 -07:00
wangzhenhua 890493458d [SPARK-20104][SQL] Don't estimate IsNull or IsNotNull predicates for non-leaf node
## What changes were proposed in this pull request?

In current stage, we don't have advanced statistics such as sketches or histograms. As a result, some operator can't estimate `nullCount` accurately. E.g. left outer join estimation does not accurately update `nullCount` currently. So for `IsNull` and `IsNotNull` predicates, we only estimate them when the child is a leaf node, whose `nullCount` is accurate.

## How was this patch tested?

A new test case is added in `FilterEstimationSuite`.

Author: wangzhenhua <wangzhenhua@huawei.com>

Closes #17438 from wzhfy/nullEstimation.
2017-03-27 23:41:27 +08:00
Herman van Hovell 617ab6445e [SPARK-20086][SQL] CollapseWindow should not collapse dependent adjacent windows
## What changes were proposed in this pull request?
The `CollapseWindow` is currently to aggressive when collapsing adjacent windows. It also collapses windows in the which the parent produces a column that is consumed by the child; this creates an invalid window which will fail at runtime.

This PR fixes this by adding a check for dependent adjacent windows to the `CollapseWindow` rule.

## How was this patch tested?
Added a new test case to `CollapseWindowSuite`

Author: Herman van Hovell <hvanhovell@databricks.com>

Closes #17432 from hvanhovell/SPARK-20086.
2017-03-26 22:47:31 +02:00
Kazuaki Ishizaki 93bb0b911b [SPARK-20046][SQL] Facilitate loop optimizations in a JIT compiler regarding sqlContext.read.parquet()
## What changes were proposed in this pull request?

This PR improves performance of operations with `sqlContext.read.parquet()` by changing Java code generated by Catalyst. This PR is inspired by [the blog article](https://databricks.com/blog/2017/02/16/processing-trillion-rows-per-second-single-machine-can-nested-loop-joins-fast.html) and [this stackoverflow entry](http://stackoverflow.com/questions/40629435/fast-parquet-row-count-in-spark).

This PR changes generated code in the following two points.
1. Replace a while-loop with long instance variables a for-loop with int local variables
2. Suppress generation of `shouldStop()` method if this method is unnecessary (e.g. `append()` is not generated).

These points facilitates compiler optimizations in a JIT compiler by feeding the simplified Java code into the JIT compiler. The performance of `sqlContext.read.parquet().count` is improved by 1.09x.

Benchmark program:
```java
val dir = "/dev/shm/parquet"
val N = 1000 * 1000 * 40
val iters = 20
val benchmark = new Benchmark("Parquet", N * iters, minNumIters = 5, warmupTime = 30.seconds)
sparkSession.range(n).write.mode("overwrite").parquet(dir)

benchmark.addCase("count") { i: Int =>
  var n = 0
  var len = 0L
  while (n < iters) {
    len += sparkSession.read.parquet(dir).count
    n += 1
  }
}
benchmark.run
```

Performance result without this PR
```
OpenJDK 64-Bit Server VM 1.8.0_121-8u121-b13-0ubuntu1.16.04.2-b13 on Linux 4.4.0-47-generic
Intel(R) Xeon(R) CPU E5-2667 v3  3.20GHz
Parquet:                                 Best/Avg Time(ms)    Rate(M/s)   Per Row(ns)   Relative
------------------------------------------------------------------------------------------------
w/o this PR                                   1152 / 1211        694.7           1.4       1.0X
```

Performance result with this PR
```
OpenJDK 64-Bit Server VM 1.8.0_121-8u121-b13-0ubuntu1.16.04.2-b13 on Linux 4.4.0-47-generic
Intel(R) Xeon(R) CPU E5-2667 v3  3.20GHz
Parquet:                                 Best/Avg Time(ms)    Rate(M/s)   Per Row(ns)   Relative
------------------------------------------------------------------------------------------------
with this PR                                  1053 / 1121        760.0           1.3       1.0X
```

Here is a comparison between generated code w/o and with this PR. Only the method ```agg_doAggregateWithoutKey``` is changed.

Generated code without this PR
```java
/* 005 */ final class GeneratedIterator extends org.apache.spark.sql.execution.BufferedRowIterator {
/* 006 */   private Object[] references;
/* 007 */   private scala.collection.Iterator[] inputs;
/* 008 */   private boolean agg_initAgg;
/* 009 */   private boolean agg_bufIsNull;
/* 010 */   private long agg_bufValue;
/* 011 */   private scala.collection.Iterator scan_input;
/* 012 */   private org.apache.spark.sql.execution.metric.SQLMetric scan_numOutputRows;
/* 013 */   private org.apache.spark.sql.execution.metric.SQLMetric scan_scanTime;
/* 014 */   private long scan_scanTime1;
/* 015 */   private org.apache.spark.sql.execution.vectorized.ColumnarBatch scan_batch;
/* 016 */   private int scan_batchIdx;
/* 017 */   private org.apache.spark.sql.execution.metric.SQLMetric agg_numOutputRows;
/* 018 */   private org.apache.spark.sql.execution.metric.SQLMetric agg_aggTime;
/* 019 */   private UnsafeRow agg_result;
/* 020 */   private org.apache.spark.sql.catalyst.expressions.codegen.BufferHolder agg_holder;
/* 021 */   private org.apache.spark.sql.catalyst.expressions.codegen.UnsafeRowWriter agg_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 */     agg_initAgg = false;
/* 031 */
/* 032 */     scan_input = inputs[0];
/* 033 */     this.scan_numOutputRows = (org.apache.spark.sql.execution.metric.SQLMetric) references[0];
/* 034 */     this.scan_scanTime = (org.apache.spark.sql.execution.metric.SQLMetric) references[1];
/* 035 */     scan_scanTime1 = 0;
/* 036 */     scan_batch = null;
/* 037 */     scan_batchIdx = 0;
/* 038 */     this.agg_numOutputRows = (org.apache.spark.sql.execution.metric.SQLMetric) references[2];
/* 039 */     this.agg_aggTime = (org.apache.spark.sql.execution.metric.SQLMetric) references[3];
/* 040 */     agg_result = new UnsafeRow(1);
/* 041 */     this.agg_holder = new org.apache.spark.sql.catalyst.expressions.codegen.BufferHolder(agg_result, 0);
/* 042 */     this.agg_rowWriter = new org.apache.spark.sql.catalyst.expressions.codegen.UnsafeRowWriter(agg_holder, 1);
/* 043 */
/* 044 */   }
/* 045 */
/* 046 */   private void agg_doAggregateWithoutKey() throws java.io.IOException {
/* 047 */     // initialize aggregation buffer
/* 048 */     agg_bufIsNull = false;
/* 049 */     agg_bufValue = 0L;
/* 050 */
/* 051 */     if (scan_batch == null) {
/* 052 */       scan_nextBatch();
/* 053 */     }
/* 054 */     while (scan_batch != null) {
/* 055 */       int numRows = scan_batch.numRows();
/* 056 */       while (scan_batchIdx < numRows) {
/* 057 */         int scan_rowIdx = scan_batchIdx++;
/* 058 */         // do aggregate
/* 059 */         // common sub-expressions
/* 060 */
/* 061 */         // evaluate aggregate function
/* 062 */         boolean agg_isNull1 = false;
/* 063 */
/* 064 */         long agg_value1 = -1L;
/* 065 */         agg_value1 = agg_bufValue + 1L;
/* 066 */         // update aggregation buffer
/* 067 */         agg_bufIsNull = false;
/* 068 */         agg_bufValue = agg_value1;
/* 069 */         if (shouldStop()) return;
/* 070 */       }
/* 071 */       scan_batch = null;
/* 072 */       scan_nextBatch();
/* 073 */     }
/* 074 */     scan_scanTime.add(scan_scanTime1 / (1000 * 1000));
/* 075 */     scan_scanTime1 = 0;
/* 076 */
/* 077 */   }
/* 078 */
/* 079 */   private void scan_nextBatch() throws java.io.IOException {
/* 080 */     long getBatchStart = System.nanoTime();
/* 081 */     if (scan_input.hasNext()) {
/* 082 */       scan_batch = (org.apache.spark.sql.execution.vectorized.ColumnarBatch)scan_input.next();
/* 083 */       scan_numOutputRows.add(scan_batch.numRows());
/* 084 */       scan_batchIdx = 0;
/* 085 */
/* 086 */     }
/* 087 */     scan_scanTime1 += System.nanoTime() - getBatchStart;
/* 088 */   }
/* 089 */
/* 090 */   protected void processNext() throws java.io.IOException {
/* 091 */     while (!agg_initAgg) {
/* 092 */       agg_initAgg = true;
/* 093 */       long agg_beforeAgg = System.nanoTime();
/* 094 */       agg_doAggregateWithoutKey();
/* 095 */       agg_aggTime.add((System.nanoTime() - agg_beforeAgg) / 1000000);
/* 096 */
/* 097 */       // output the result
/* 098 */
/* 099 */       agg_numOutputRows.add(1);
/* 100 */       agg_rowWriter.zeroOutNullBytes();
/* 101 */
/* 102 */       if (agg_bufIsNull) {
/* 103 */         agg_rowWriter.setNullAt(0);
/* 104 */       } else {
/* 105 */         agg_rowWriter.write(0, agg_bufValue);
/* 106 */       }
/* 107 */       append(agg_result);
/* 108 */     }
/* 109 */   }
/* 110 */ }
```

Generated code with this PR
```java
/* 005 */ final class GeneratedIterator extends org.apache.spark.sql.execution.BufferedRowIterator {
/* 006 */   private Object[] references;
/* 007 */   private scala.collection.Iterator[] inputs;
/* 008 */   private boolean agg_initAgg;
/* 009 */   private boolean agg_bufIsNull;
/* 010 */   private long agg_bufValue;
/* 011 */   private scala.collection.Iterator scan_input;
/* 012 */   private org.apache.spark.sql.execution.metric.SQLMetric scan_numOutputRows;
/* 013 */   private org.apache.spark.sql.execution.metric.SQLMetric scan_scanTime;
/* 014 */   private long scan_scanTime1;
/* 015 */   private org.apache.spark.sql.execution.vectorized.ColumnarBatch scan_batch;
/* 016 */   private int scan_batchIdx;
/* 017 */   private org.apache.spark.sql.execution.metric.SQLMetric agg_numOutputRows;
/* 018 */   private org.apache.spark.sql.execution.metric.SQLMetric agg_aggTime;
/* 019 */   private UnsafeRow agg_result;
/* 020 */   private org.apache.spark.sql.catalyst.expressions.codegen.BufferHolder agg_holder;
/* 021 */   private org.apache.spark.sql.catalyst.expressions.codegen.UnsafeRowWriter agg_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 */     agg_initAgg = false;
/* 031 */
/* 032 */     scan_input = inputs[0];
/* 033 */     this.scan_numOutputRows = (org.apache.spark.sql.execution.metric.SQLMetric) references[0];
/* 034 */     this.scan_scanTime = (org.apache.spark.sql.execution.metric.SQLMetric) references[1];
/* 035 */     scan_scanTime1 = 0;
/* 036 */     scan_batch = null;
/* 037 */     scan_batchIdx = 0;
/* 038 */     this.agg_numOutputRows = (org.apache.spark.sql.execution.metric.SQLMetric) references[2];
/* 039 */     this.agg_aggTime = (org.apache.spark.sql.execution.metric.SQLMetric) references[3];
/* 040 */     agg_result = new UnsafeRow(1);
/* 041 */     this.agg_holder = new org.apache.spark.sql.catalyst.expressions.codegen.BufferHolder(agg_result, 0);
/* 042 */     this.agg_rowWriter = new org.apache.spark.sql.catalyst.expressions.codegen.UnsafeRowWriter(agg_holder, 1);
/* 043 */
/* 044 */   }
/* 045 */
/* 046 */   private void agg_doAggregateWithoutKey() throws java.io.IOException {
/* 047 */     // initialize aggregation buffer
/* 048 */     agg_bufIsNull = false;
/* 049 */     agg_bufValue = 0L;
/* 050 */
/* 051 */     if (scan_batch == null) {
/* 052 */       scan_nextBatch();
/* 053 */     }
/* 054 */     while (scan_batch != null) {
/* 055 */       int numRows = scan_batch.numRows();
/* 056 */       int scan_localEnd = numRows - scan_batchIdx;
/* 057 */       for (int scan_localIdx = 0; scan_localIdx < scan_localEnd; scan_localIdx++) {
/* 058 */         int scan_rowIdx = scan_batchIdx + scan_localIdx;
/* 059 */         // do aggregate
/* 060 */         // common sub-expressions
/* 061 */
/* 062 */         // evaluate aggregate function
/* 063 */         boolean agg_isNull1 = false;
/* 064 */
/* 065 */         long agg_value1 = -1L;
/* 066 */         agg_value1 = agg_bufValue + 1L;
/* 067 */         // update aggregation buffer
/* 068 */         agg_bufIsNull = false;
/* 069 */         agg_bufValue = agg_value1;
/* 070 */         // shouldStop check is eliminated
/* 071 */       }
/* 072 */       scan_batchIdx = numRows;
/* 073 */       scan_batch = null;
/* 074 */       scan_nextBatch();
/* 075 */     }
/* 079 */   }
/* 080 */
/* 081 */   private void scan_nextBatch() throws java.io.IOException {
/* 082 */     long getBatchStart = System.nanoTime();
/* 083 */     if (scan_input.hasNext()) {
/* 084 */       scan_batch = (org.apache.spark.sql.execution.vectorized.ColumnarBatch)scan_input.next();
/* 085 */       scan_numOutputRows.add(scan_batch.numRows());
/* 086 */       scan_batchIdx = 0;
/* 087 */
/* 088 */     }
/* 089 */     scan_scanTime1 += System.nanoTime() - getBatchStart;
/* 090 */   }
/* 091 */
/* 092 */   protected void processNext() throws java.io.IOException {
/* 093 */     while (!agg_initAgg) {
/* 094 */       agg_initAgg = true;
/* 095 */       long agg_beforeAgg = System.nanoTime();
/* 096 */       agg_doAggregateWithoutKey();
/* 097 */       agg_aggTime.add((System.nanoTime() - agg_beforeAgg) / 1000000);
/* 098 */
/* 099 */       // output the result
/* 100 */
/* 101 */       agg_numOutputRows.add(1);
/* 102 */       agg_rowWriter.zeroOutNullBytes();
/* 103 */
/* 104 */       if (agg_bufIsNull) {
/* 105 */         agg_rowWriter.setNullAt(0);
/* 106 */       } else {
/* 107 */         agg_rowWriter.write(0, agg_bufValue);
/* 108 */       }
/* 109 */       append(agg_result);
/* 110 */     }
/* 111 */   }
/* 112 */ }
```

## How was this patch tested?

Tested existing test suites

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

Closes #17378 from kiszk/SPARK-20046.
2017-03-26 09:20:22 +02:00
Wenchen Fan 0b903caef3 [SPARK-19949][SQL][FOLLOW-UP] move FailureSafeParser from catalyst to sql core
## What changes were proposed in this pull request?

The `FailureSafeParser` is only used in sql core, it doesn't make sense to put it in catalyst module.

## How was this patch tested?

N/A

Author: Wenchen Fan <wenchen@databricks.com>

Closes #17408 from cloud-fan/minor.
2017-03-25 11:46:54 -07:00
Xiao Li a2ce0a2e30 [HOTFIX][SQL] Fix the failed test cases in GeneratorFunctionSuite
### What changes were proposed in this pull request?
Multiple tests failed. Revert the changes on `supportCodegen` of `GenerateExec`. For example,

- https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/75194/testReport/

### How was this patch tested?
N/A

Author: Xiao Li <gatorsmile@gmail.com>

Closes #17425 from gatorsmile/turnOnCodeGenGenerateExec.
2017-03-24 23:27:42 -07:00
Roxanne Moslehi f88f56b835 [DOCS] Clarify round mode for format_number & round functions
## What changes were proposed in this pull request?

Updated the description for the `format_number` description to indicate that it uses `HALF_EVEN` rounding. Updated the description for the `round` description to indicate that it uses `HALF_UP` rounding.

## How was this patch tested?

Just changing the two function comments so no testing involved.

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

Author: Roxanne Moslehi <rmoslehi@palantir.com>
Author: roxannemoslehi <rmoslehi@berkeley.edu>

Closes #17399 from roxannemoslehi/patch-1.
2017-03-25 00:10:30 +01:00
Liang-Chi Hsieh e011004bed [SPARK-19846][SQL] Add a flag to disable constraint propagation
## What changes were proposed in this pull request?

Constraint propagation can be computation expensive and block the driver execution for long time. For example, the below benchmark needs 30mins.

Compared with previous PRs #16998, #16785, this is a much simpler option: add a flag to disable constraint propagation.

### Benchmark

Run the following codes locally.

    import org.apache.spark.ml.{Pipeline, PipelineStage}
    import org.apache.spark.ml.feature.{OneHotEncoder, StringIndexer, VectorAssembler}
    import org.apache.spark.sql.internal.SQLConf

    spark.conf.set(SQLConf.CONSTRAINT_PROPAGATION_ENABLED.key, false)

    val df = (1 to 40).foldLeft(Seq((1, "foo"), (2, "bar"), (3, "baz")).toDF("id", "x0"))((df, i) => df.withColumn(s"x$i", $"x0"))

    val indexers = df.columns.tail.map(c => new StringIndexer()
      .setInputCol(c)
      .setOutputCol(s"${c}_indexed")
      .setHandleInvalid("skip"))

    val encoders = indexers.map(indexer => new OneHotEncoder()
      .setInputCol(indexer.getOutputCol)
      .setOutputCol(s"${indexer.getOutputCol}_encoded")
      .setDropLast(true))

    val stages: Array[PipelineStage] = indexers ++ encoders
    val pipeline = new Pipeline().setStages(stages)

    val startTime = System.nanoTime
    pipeline.fit(df).transform(df).show
    val runningTime = System.nanoTime - startTime

Before this patch: 1786001 ms ~= 30 mins
After this patch: 26392 ms = less than half of a minute

Related PRs: #16998, #16785.

## How was this patch tested?

Jenkins tests.

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

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

Closes #17186 from viirya/add-flag-disable-constraint-propagation.
2017-03-25 00:04:51 +01:00
Reynold Xin b5c5bd98ea Disable generate codegen since it fails my workload. 2017-03-24 23:57:29 +01:00
Herman van Hovell 91fa80fe8a [SPARK-20070][SQL] Redact DataSourceScanExec treeString
## What changes were proposed in this pull request?
The explain output of `DataSourceScanExec` can contain sensitive information (like Amazon keys). Such information should not end up in logs, or be exposed to non privileged users.

This PR addresses this by adding a redaction facility for the `DataSourceScanExec.treeString`. A user can enable this by setting a regex in the `spark.redaction.string.regex` configuration.

## How was this patch tested?
Added a unit test to check the output of DataSourceScanExec.

Author: Herman van Hovell <hvanhovell@databricks.com>

Closes #17397 from hvanhovell/SPARK-20070.
2017-03-24 15:52:48 -07:00
Jacek Laskowski 9299d071f9 [SQL][MINOR] Fix for typo in Analyzer
## What changes were proposed in this pull request?

Fix for typo in Analyzer

## How was this patch tested?

local build

Author: Jacek Laskowski <jacek@japila.pl>

Closes #17409 from jaceklaskowski/analyzer-typo.
2017-03-24 09:56:05 -07:00
Xiao Li 344f38b04b [SPARK-19970][SQL][FOLLOW-UP] Table owner should be USER instead of PRINCIPAL in kerberized clusters #17311
### What changes were proposed in this pull request?
This is a follow-up for the PR: https://github.com/apache/spark/pull/17311

- For safety, use `sessionState` to get the user name, instead of calling `SessionState.get()` in the function `toHiveTable`.
- Passing `user names` instead of `conf` when calling `toHiveTable`.

### How was this patch tested?
N/A

Author: Xiao Li <gatorsmile@gmail.com>

Closes #17405 from gatorsmile/user.
2017-03-24 14:42:33 +08:00
Eric Liang 8e558041aa [SPARK-19820][CORE] Add interface to kill tasks w/ a reason
This commit adds a killTaskAttempt method to SparkContext, to allow users to
kill tasks so that they can be re-scheduled elsewhere.

This also refactors the task kill path to allow specifying a reason for the task kill. The reason is propagated opaquely through events, and will show up in the UI automatically as `(N killed: $reason)` and `TaskKilled: $reason`. Without this change, there is no way to provide the user feedback through the UI.

Currently used reasons are "stage cancelled", "another attempt succeeded", and "killed via SparkContext.killTask". The user can also specify a custom reason through `SparkContext.killTask`.

cc rxin

In the stage overview UI the reasons are summarized:
![1](https://cloud.githubusercontent.com/assets/14922/23929209/a83b2862-08e1-11e7-8b3e-ae1967bbe2e5.png)

Within the stage UI you can see individual task kill reasons:
![2](https://cloud.githubusercontent.com/assets/14922/23929200/9a798692-08e1-11e7-8697-72b27ad8a287.png)

Existing tests, tried killing some stages in the UI and verified the messages are as expected.

Author: Eric Liang <ekl@databricks.com>
Author: Eric Liang <ekl@google.com>

Closes #17166 from ericl/kill-reason.
2017-03-23 23:30:44 -07:00
Kazuaki Ishizaki bb823ca4b4 [SPARK-19959][SQL] Fix to throw NullPointerException in df[java.lang.Long].collect
## What changes were proposed in this pull request?

This PR fixes `NullPointerException` in the generated code by Catalyst. When we run the following code, we get the following `NullPointerException`. This is because there is no null checks for `inputadapter_value`  while `java.lang.Long inputadapter_value` at Line 30 may have `null`.

This happen when a type of DataFrame is nullable primitive type such as `java.lang.Long` and the wholestage codegen is used. While the physical plan keeps `nullable=true` in `input[0, java.lang.Long, true].longValue`, `BoundReference.doGenCode` ignores `nullable=true`. Thus, nullcheck code will not be generated and `NullPointerException` will occur.

This PR checks the nullability and correctly generates nullcheck if needed.
```java
sparkContext.parallelize(Seq[java.lang.Long](0L, null, 2L), 1).toDF.collect
```

```java
Caused by: java.lang.NullPointerException
	at org.apache.spark.sql.catalyst.expressions.GeneratedClass$GeneratedIterator.processNext(generated.java:37)
	at org.apache.spark.sql.execution.BufferedRowIterator.hasNext(BufferedRowIterator.java:43)
	at org.apache.spark.sql.execution.WholeStageCodegenExec$$anonfun$8$$anon$1.hasNext(WholeStageCodegenExec.scala:393)
...
```

Generated code without this PR
```java
/* 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 inputadapter_input;
/* 009 */   private UnsafeRow serializefromobject_result;
/* 010 */   private org.apache.spark.sql.catalyst.expressions.codegen.BufferHolder serializefromobject_holder;
/* 011 */   private org.apache.spark.sql.catalyst.expressions.codegen.UnsafeRowWriter serializefromobject_rowWriter;
/* 012 */
/* 013 */   public GeneratedIterator(Object[] references) {
/* 014 */     this.references = references;
/* 015 */   }
/* 016 */
/* 017 */   public void init(int index, scala.collection.Iterator[] inputs) {
/* 018 */     partitionIndex = index;
/* 019 */     this.inputs = inputs;
/* 020 */     inputadapter_input = inputs[0];
/* 021 */     serializefromobject_result = new UnsafeRow(1);
/* 022 */     this.serializefromobject_holder = new org.apache.spark.sql.catalyst.expressions.codegen.BufferHolder(serializefromobject_result, 0);
/* 023 */     this.serializefromobject_rowWriter = new org.apache.spark.sql.catalyst.expressions.codegen.UnsafeRowWriter(serializefromobject_holder, 1);
/* 024 */
/* 025 */   }
/* 026 */
/* 027 */   protected void processNext() throws java.io.IOException {
/* 028 */     while (inputadapter_input.hasNext() && !stopEarly()) {
/* 029 */       InternalRow inputadapter_row = (InternalRow) inputadapter_input.next();
/* 030 */       java.lang.Long inputadapter_value = (java.lang.Long)inputadapter_row.get(0, null);
/* 031 */
/* 032 */       boolean serializefromobject_isNull = true;
/* 033 */       long serializefromobject_value = -1L;
/* 034 */       if (!false) {
/* 035 */         serializefromobject_isNull = false;
/* 036 */         if (!serializefromobject_isNull) {
/* 037 */           serializefromobject_value = inputadapter_value.longValue();
/* 038 */         }
/* 039 */
/* 040 */       }
/* 041 */       serializefromobject_rowWriter.zeroOutNullBytes();
/* 042 */
/* 043 */       if (serializefromobject_isNull) {
/* 044 */         serializefromobject_rowWriter.setNullAt(0);
/* 045 */       } else {
/* 046 */         serializefromobject_rowWriter.write(0, serializefromobject_value);
/* 047 */       }
/* 048 */       append(serializefromobject_result);
/* 049 */       if (shouldStop()) return;
/* 050 */     }
/* 051 */   }
/* 052 */ }
```

Generated code with this PR

```java
/* 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 inputadapter_input;
/* 009 */   private UnsafeRow serializefromobject_result;
/* 010 */   private org.apache.spark.sql.catalyst.expressions.codegen.BufferHolder serializefromobject_holder;
/* 011 */   private org.apache.spark.sql.catalyst.expressions.codegen.UnsafeRowWriter serializefromobject_rowWriter;
/* 012 */
/* 013 */   public GeneratedIterator(Object[] references) {
/* 014 */     this.references = references;
/* 015 */   }
/* 016 */
/* 017 */   public void init(int index, scala.collection.Iterator[] inputs) {
/* 018 */     partitionIndex = index;
/* 019 */     this.inputs = inputs;
/* 020 */     inputadapter_input = inputs[0];
/* 021 */     serializefromobject_result = new UnsafeRow(1);
/* 022 */     this.serializefromobject_holder = new org.apache.spark.sql.catalyst.expressions.codegen.BufferHolder(serializefromobject_result, 0);
/* 023 */     this.serializefromobject_rowWriter = new org.apache.spark.sql.catalyst.expressions.codegen.UnsafeRowWriter(serializefromobject_holder, 1);
/* 024 */
/* 025 */   }
/* 026 */
/* 027 */   protected void processNext() throws java.io.IOException {
/* 028 */     while (inputadapter_input.hasNext() && !stopEarly()) {
/* 029 */       InternalRow inputadapter_row = (InternalRow) inputadapter_input.next();
/* 030 */       boolean inputadapter_isNull = inputadapter_row.isNullAt(0);
/* 031 */       java.lang.Long inputadapter_value = inputadapter_isNull ? null : ((java.lang.Long)inputadapter_row.get(0, null));
/* 032 */
/* 033 */       boolean serializefromobject_isNull = true;
/* 034 */       long serializefromobject_value = -1L;
/* 035 */       if (!inputadapter_isNull) {
/* 036 */         serializefromobject_isNull = false;
/* 037 */         if (!serializefromobject_isNull) {
/* 038 */           serializefromobject_value = inputadapter_value.longValue();
/* 039 */         }
/* 040 */
/* 041 */       }
/* 042 */       serializefromobject_rowWriter.zeroOutNullBytes();
/* 043 */
/* 044 */       if (serializefromobject_isNull) {
/* 045 */         serializefromobject_rowWriter.setNullAt(0);
/* 046 */       } else {
/* 047 */         serializefromobject_rowWriter.write(0, serializefromobject_value);
/* 048 */       }
/* 049 */       append(serializefromobject_result);
/* 050 */       if (shouldStop()) return;
/* 051 */     }
/* 052 */   }
/* 053 */ }
```

## How was this patch tested?

Added new test suites in `DataFrameSuites`

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

Closes #17302 from kiszk/SPARK-19959.
2017-03-24 12:57:56 +08:00
Burak Yavuz 93581fbc18 Fix compilation of the Scala 2.10 master branch
## What changes were proposed in this pull request?

Fixes break caused by: 746a558de2

## How was this patch tested?

Compiled with `build/sbt -Dscala2.10 sql/compile` locally

Author: Burak Yavuz <brkyvz@gmail.com>

Closes #17403 from brkyvz/onceTrigger2.10.
2017-03-23 17:57:31 -07:00
sureshthalamati c791180705 [SPARK-10849][SQL] Adds option to the JDBC data source write for user to specify database column type for the create table
## What changes were proposed in this pull request?
Currently JDBC data source creates tables in the target database using the default type mapping, and the JDBC dialect mechanism.  If users want to specify different database data type for only some of columns, there is no option available. In scenarios where default mapping does not work, users are forced to create tables on the target database before writing. This workaround is probably not acceptable from a usability point of view. This PR is to provide a user-defined type mapping for specific columns.

The solution is to allow users to specify database column data type for the create table  as JDBC datasource option(createTableColumnTypes) on write. Data type information can be specified in the same format as table schema DDL format (e.g: `name CHAR(64), comments VARCHAR(1024)`).

All supported target database types can not be specified ,  the data types has to be valid spark sql data types also.  For example user can not specify target database  CLOB data type. This will be supported in the follow-up PR.

Example:
```Scala
df.write
.option("createTableColumnTypes", "name CHAR(64), comments VARCHAR(1024)")
.jdbc(url, "TEST.DBCOLTYPETEST", properties)
```
## How was this patch tested?
Added new test cases to the JDBCWriteSuite

Author: sureshthalamati <suresh.thalamati@gmail.com>

Closes #16209 from sureshthalamati/jdbc_custom_dbtype_option_json-spark-10849.
2017-03-23 17:39:33 -07:00
Tyson Condie 746a558de2 [SPARK-19876][SS][WIP] OneTime Trigger Executor
## What changes were proposed in this pull request?

An additional trigger and trigger executor that will execute a single trigger only. One can use this OneTime trigger to have more control over the scheduling of triggers.

In addition, this patch requires an optimization to StreamExecution that logs a commit record at the end of successfully processing a batch. This new commit log will be used to determine the next batch (offsets) to process after a restart, instead of using the offset log itself to determine what batch to process next after restart; using the offset log to determine this would process the previously logged batch, always, thus not permitting a OneTime trigger feature.

## How was this patch tested?

A number of existing tests have been revised. These tests all assumed that when restarting a stream, the last batch in the offset log is to be re-processed. Given that we now have a commit log that will tell us if that last batch was processed successfully, the results/assumptions of those tests needed to be revised accordingly.

In addition, a OneTime trigger test was added to StreamingQuerySuite, which tests:
- The semantics of OneTime trigger (i.e., on start, execute a single batch, then stop).
- The case when the commit log was not able to successfully log the completion of a batch before restart, which would mean that we should fall back to what's in the offset log.
- A OneTime trigger execution that results in an exception being thrown.

marmbrus tdas zsxwing

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

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

Closes #17219 from tcondie/stream-commit.
2017-03-23 14:32:05 -07:00
hyukjinkwon aefe798905 [MINOR][BUILD] Fix javadoc8 break
## What changes were proposed in this pull request?

Several javadoc8 breaks have been introduced. This PR proposes fix those instances so that we can build Scala/Java API docs.

```
[error] .../spark/sql/core/target/java/org/apache/spark/sql/streaming/GroupState.java:6: error: reference not found
[error]  * <code>flatMapGroupsWithState</code> operations on {link KeyValueGroupedDataset}.
[error]                                                             ^
[error] .../spark/sql/core/target/java/org/apache/spark/sql/streaming/GroupState.java:10: error: reference not found
[error]  * Both, <code>mapGroupsWithState</code> and <code>flatMapGroupsWithState</code> in {link KeyValueGroupedDataset}
[error]                                                                                            ^
[error] .../spark/sql/core/target/java/org/apache/spark/sql/streaming/GroupState.java:51: error: reference not found
[error]  *    {link GroupStateTimeout.ProcessingTimeTimeout}) or event time (i.e.
[error]              ^
[error] .../spark/sql/core/target/java/org/apache/spark/sql/streaming/GroupState.java:52: error: reference not found
[error]  *    {link GroupStateTimeout.EventTimeTimeout}).
[error]              ^
[error] .../spark/sql/core/target/java/org/apache/spark/sql/streaming/GroupState.java:158: error: reference not found
[error]  *           Spark SQL types (see {link Encoder} for more details).
[error]                                          ^
[error] .../spark/mllib/target/java/org/apache/spark/ml/fpm/FPGrowthParams.java:26: error: bad use of '>'
[error]    * Number of partitions (>=1) used by parallel FP-growth. By default the param is not set, and
[error]                            ^
[error] .../spark/sql/core/src/main/java/org/apache/spark/api/java/function/FlatMapGroupsWithStateFunction.java:30: error: reference not found
[error]  * {link org.apache.spark.sql.KeyValueGroupedDataset#flatMapGroupsWithState(
[error]           ^
[error] .../spark/sql/core/target/java/org/apache/spark/sql/KeyValueGroupedDataset.java:211: error: reference not found
[error]    * See {link GroupState} for more details.
[error]                 ^
[error] .../spark/sql/core/target/java/org/apache/spark/sql/KeyValueGroupedDataset.java:232: error: reference not found
[error]    * See {link GroupState} for more details.
[error]                 ^
[error] .../spark/sql/core/target/java/org/apache/spark/sql/KeyValueGroupedDataset.java:254: error: reference not found
[error]    * See {link GroupState} for more details.
[error]                 ^
[error] .../spark/sql/core/target/java/org/apache/spark/sql/KeyValueGroupedDataset.java:277: error: reference not found
[error]    * See {link GroupState} for more details.
[error]                 ^
[error] .../spark/core/target/java/org/apache/spark/TaskContextImpl.java:10: error: reference not found
[error]  * {link TaskMetrics} &amp; {link MetricsSystem} objects are not thread safe.
[error]           ^
[error] .../spark/core/target/java/org/apache/spark/TaskContextImpl.java:10: error: reference not found
[error]  * {link TaskMetrics} &amp; {link MetricsSystem} objects are not thread safe.
[error]                                     ^
[info] 13 errors
```

```
jekyll 3.3.1 | Error:  Unidoc generation failed
```

## How was this patch tested?

Manually via `jekyll build`

Author: hyukjinkwon <gurwls223@gmail.com>

Closes #17389 from HyukjinKwon/minor-javadoc8-fix.
2017-03-23 08:41:30 +00:00
hyukjinkwon 07c12c09a7 [SPARK-18579][SQL] Use ignoreLeadingWhiteSpace and ignoreTrailingWhiteSpace options in CSV writing
## What changes were proposed in this pull request?

This PR proposes to support _not_ trimming the white spaces when writing out. These are `false` by default in CSV reading path but these are `true` by default in CSV writing in univocity parser.

Both `ignoreLeadingWhiteSpace` and `ignoreTrailingWhiteSpace` options are not being used for writing and therefore, we are always trimming the white spaces.

It seems we should provide a way to keep this white spaces easily.

WIth the data below:

```scala
val df = spark.read.csv(Seq("a , b  , c").toDS)
df.show()
```

```
+---+----+---+
|_c0| _c1|_c2|
+---+----+---+
| a | b  |  c|
+---+----+---+
```

**Before**

```scala
df.write.csv("/tmp/text.csv")
spark.read.text("/tmp/text.csv").show()
```

```
+-----+
|value|
+-----+
|a,b,c|
+-----+
```

It seems this can't be worked around via `quoteAll` too.

```scala
df.write.option("quoteAll", true).csv("/tmp/text.csv")
spark.read.text("/tmp/text.csv").show()
```
```
+-----------+
|      value|
+-----------+
|"a","b","c"|
+-----------+
```

**After**

```scala
df.write.option("ignoreLeadingWhiteSpace", false).option("ignoreTrailingWhiteSpace", false).csv("/tmp/text.csv")
spark.read.text("/tmp/text.csv").show()
```

```
+----------+
|     value|
+----------+
|a , b  , c|
+----------+
```

Note that this case is possible in R

```r
> system("cat text.csv")
f1,f2,f3
a , b  , c
> df <- read.csv(file="text.csv")
> df
  f1   f2 f3
1 a   b    c
> write.csv(df, file="text1.csv", quote=F, row.names=F)
> system("cat text1.csv")
f1,f2,f3
a , b  , c
```

## How was this patch tested?

Unit tests in `CSVSuite` and manual tests for Python.

Author: hyukjinkwon <gurwls223@gmail.com>

Closes #17310 from HyukjinKwon/SPARK-18579.
2017-03-23 00:25:01 -07:00
Sameer Agarwal 12cd00706c [BUILD][MINOR] Fix 2.10 build
## What changes were proposed in this pull request?

https://github.com/apache/spark/pull/17385 breaks the 2.10 sbt/maven builds by hitting an empty-string interpolation bug (https://issues.scala-lang.org/browse/SI-7919).

https://amplab.cs.berkeley.edu/jenkins/view/Spark%20QA%20Compile/job/spark-master-compile-sbt-scala-2.10/4072/
https://amplab.cs.berkeley.edu/jenkins/view/Spark%20QA%20Compile/job/spark-master-compile-maven-scala-2.10/3987/

## How was this patch tested?

Compiles

Author: Sameer Agarwal <sameerag@cs.berkeley.edu>

Closes #17391 from sameeragarwal/build-fix.
2017-03-22 15:58:42 -07:00
Tathagata Das 82b598b963 [SPARK-20057][SS] Renamed KeyedState to GroupState in mapGroupsWithState
## What changes were proposed in this pull request?

Since the state is tied a "group" in the "mapGroupsWithState" operations, its better to call the state "GroupState" instead of a key. This would make it more general if you extends this operation to RelationGroupedDataset and python APIs.

## How was this patch tested?
Existing unit tests.

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

Closes #17385 from tdas/SPARK-20057.
2017-03-22 12:30:36 -07:00
hyukjinkwon 80fd070389 [SPARK-20018][SQL] Pivot with timestamp and count should not print internal representation
## What changes were proposed in this pull request?

Currently, when we perform count with timestamp types, it prints the internal representation as the column name as below:

```scala
Seq(new java.sql.Timestamp(1)).toDF("a").groupBy("a").pivot("a").count().show()
```

```
+--------------------+----+
|                   a|1000|
+--------------------+----+
|1969-12-31 16:00:...|   1|
+--------------------+----+
```

This PR proposes to use external Scala value instead of the internal representation in the column names as below:

```
+--------------------+-----------------------+
|                   a|1969-12-31 16:00:00.001|
+--------------------+-----------------------+
|1969-12-31 16:00:...|                      1|
+--------------------+-----------------------+
```

## How was this patch tested?

Unit test in `DataFramePivotSuite` and manual tests.

Author: hyukjinkwon <gurwls223@gmail.com>

Closes #17348 from HyukjinKwon/SPARK-20018.
2017-03-22 09:58:46 -07:00
hyukjinkwon 465818389a [SPARK-19949][SQL][FOLLOW-UP] Clean up parse modes and update related comments
## What changes were proposed in this pull request?

This PR proposes to make `mode` options in both CSV and JSON to use `cass object` and fix some related comments related previous fix.

Also, this PR modifies some tests related parse modes.

## How was this patch tested?

Modified unit tests in both `CSVSuite.scala` and `JsonSuite.scala`.

Author: hyukjinkwon <gurwls223@gmail.com>

Closes #17377 from HyukjinKwon/SPARK-19949.
2017-03-22 09:52:37 -07:00
Prashant Sharma 0caade6340 [SPARK-20027][DOCS] Compilation fix in java docs.
## What changes were proposed in this pull request?

During build/sbt publish-local, build breaks due to javadocs errors. This patch fixes those errors.

## How was this patch tested?

Tested by running the sbt build.

Author: Prashant Sharma <prashsh1@in.ibm.com>

Closes #17358 from ScrapCodes/docs-fix.
2017-03-22 13:52:03 +00:00
Xiao Li 7343a09401 [SPARK-20023][SQL] Output table comment for DESC FORMATTED
### What changes were proposed in this pull request?
Currently, `DESC FORMATTED` did not output the table comment, unlike what `DESC EXTENDED` does. This PR is to fix it.

Also correct the following displayed names in `DESC FORMATTED`, for being consistent with `DESC EXTENDED`
- `"Create Time:"` -> `"Created:"`
- `"Last Access Time:"` -> `"Last Access:"`

### How was this patch tested?
Added test cases in `describe.sql`

Author: Xiao Li <gatorsmile@gmail.com>

Closes #17381 from gatorsmile/descFormattedTableComment.
2017-03-22 19:08:28 +08:00
Tathagata Das c1e87e384d [SPARK-20030][SS] Event-time-based timeout for MapGroupsWithState
## What changes were proposed in this pull request?

Adding event time based timeout. The user sets the timeout timestamp directly using `KeyedState.setTimeoutTimestamp`. The keys times out when the watermark crosses the timeout timestamp.

## How was this patch tested?
Unit tests

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

Closes #17361 from tdas/SPARK-20030.
2017-03-21 21:27:08 -07:00
Kunal Khamar 2d73fcced0 [SPARK-20051][SS] Fix StreamSuite flaky test - recover from v2.1 checkpoint
## What changes were proposed in this pull request?

There is a race condition between calling stop on a streaming query and deleting directories in `withTempDir` that causes test to fail, fixing to do lazy deletion using delete on shutdown JVM hook.

## How was this patch tested?

- Unit test
  - repeated 300 runs with no failure

Author: Kunal Khamar <kkhamar@outlook.com>

Closes #17382 from kunalkhamar/partition-bugfix.
2017-03-21 18:56:14 -07:00
hyukjinkwon 9281a3d504 [SPARK-19919][SQL] Defer throwing the exception for empty paths in CSV datasource into DataSource
## What changes were proposed in this pull request?

This PR proposes to defer throwing the exception within `DataSource`.

Currently, if other datasources fail to infer the schema, it returns `None` and then this is being validated in `DataSource` as below:

```
scala> spark.read.json("emptydir")
org.apache.spark.sql.AnalysisException: Unable to infer schema for JSON. It must be specified manually.;
```

```
scala> spark.read.orc("emptydir")
org.apache.spark.sql.AnalysisException: Unable to infer schema for ORC. It must be specified manually.;
```

```
scala> spark.read.parquet("emptydir")
org.apache.spark.sql.AnalysisException: Unable to infer schema for Parquet. It must be specified manually.;
```

However, CSV it checks it within the datasource implementation and throws another exception message as below:

```
scala> spark.read.csv("emptydir")
java.lang.IllegalArgumentException: requirement failed: Cannot infer schema from an empty set of files
```

We could remove this duplicated check and validate this in one place in the same way with the same message.

## How was this patch tested?

Unit test in `CSVSuite` and manual test.

Author: hyukjinkwon <gurwls223@gmail.com>

Closes #17256 from HyukjinKwon/SPARK-19919.
2017-03-22 08:41:46 +08:00
Will Manning a04dcde8cb clarify array_contains function description
## What changes were proposed in this pull request?

The description in the comment for array_contains is vague/incomplete (i.e., doesn't mention that it returns `null` if the array is `null`); this PR fixes that.

## How was this patch tested?

No testing, since it merely changes a comment.

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

Author: Will Manning <lwwmanning@gmail.com>

Closes #17380 from lwwmanning/patch-1.
2017-03-22 00:40:48 +01:00
zhaorongsheng 7dbc162f12 [SPARK-20017][SQL] change the nullability of function 'StringToMap' from 'false' to 'true'
## What changes were proposed in this pull request?

Change the nullability of function `StringToMap` from `false` to `true`.

Author: zhaorongsheng <334362872@qq.com>

Closes #17350 from zhaorongsheng/bug-fix_strToMap_NPE.
2017-03-21 11:30:55 -07:00
Xin Wu 4c0ff5f585 [SPARK-19261][SQL] Alter add columns for Hive serde and some datasource tables
## What changes were proposed in this pull request?
Support` ALTER TABLE ADD COLUMNS (...) `syntax for Hive serde and some datasource tables.
In this PR, we consider a few aspects:

1. View is not supported for `ALTER ADD COLUMNS`

2. Since tables created in SparkSQL with Hive DDL syntax will populate table properties with schema information, we need make sure the consistency of the schema before and after ALTER operation in order for future use.

3. For embedded-schema type of format, such as `parquet`, we need to make sure that the predicate on the newly-added columns can be evaluated properly, or pushed down properly. In case of the data file does not have the columns for the newly-added columns, such predicates should return as if the column values are NULLs.

4. For datasource table, this feature does not support the following:
4.1 TEXT format, since there is only one default column `value` is inferred for text format data.
4.2 ORC format, since SparkSQL native ORC reader does not support the difference between user-specified-schema and inferred schema from ORC files.
4.3 Third party datasource types that implements RelationProvider, including the built-in JDBC format, since different implementations by the vendors may have different ways to dealing with schema.
4.4 Other datasource types, such as `parquet`, `json`, `csv`, `hive` are supported.

5. Column names being added can not be duplicate of any existing data column or partition column names. Case sensitivity is taken into consideration according to the sql configuration.

6. This feature also supports In-Memory catalog, while Hive support is turned off.
## How was this patch tested?
Add new test cases

Author: Xin Wu <xinwu@us.ibm.com>

Closes #16626 from xwu0226/alter_add_columns.
2017-03-21 08:49:54 -07:00
wangzhenhua 14865d7ff7 [SPARK-17080][SQL][FOLLOWUP] Improve documentation, change buildJoin method structure and add a debug log
## What changes were proposed in this pull request?

1. Improve documentation for class `Cost` and `JoinReorderDP` and method `buildJoin()`.
2. Change code structure of `buildJoin()` to make the logic clearer.
3. Add a debug-level log to record information for join reordering, including time cost, the number of items and the number of plans in memo.

## How was this patch tested?

Not related.

Author: wangzhenhua <wangzhenhua@huawei.com>

Closes #17353 from wzhfy/reorderFollow.
2017-03-21 08:44:09 -07:00
Xiao Li d2dcd6792f [SPARK-20024][SQL][TEST-MAVEN] SessionCatalog reset need to set the current database of ExternalCatalog
### What changes were proposed in this pull request?
SessionCatalog API setCurrentDatabase does not set the current database of the underlying ExternalCatalog. Thus, weird errors could come in the test suites after we call reset. We need to fix it.

So far, have not found the direct impact in the other code paths because we expect all the SessionCatalog APIs should always use the current database value we managed, unless some of code paths skip it. Thus, we fix it in the test-only function reset().

### How was this patch tested?
Multiple test case failures are observed in mvn and add a test case in SessionCatalogSuite.

Author: Xiao Li <gatorsmile@gmail.com>

Closes #17354 from gatorsmile/useDB.
2017-03-20 22:52:45 -07:00
Wenchen Fan 68d65fae71 [SPARK-19949][SQL] unify bad record handling in CSV and JSON
## What changes were proposed in this pull request?

Currently JSON and CSV have exactly the same logic about handling bad records, this PR tries to abstract it and put it in a upper level to reduce code duplication.

The overall idea is, we make the JSON and CSV parser to throw a BadRecordException, then the upper level, FailureSafeParser, handles bad records according to the parse mode.

Behavior changes:
1. with PERMISSIVE mode, if the number of tokens doesn't match the schema, previously CSV parser will treat it as a legal record and parse as many tokens as possible. After this PR, we treat it as an illegal record, and put the raw record string in a special column, but we still parse as many tokens as possible.
2. all logging is removed as they are not very useful in practice.

## How was this patch tested?

existing tests

Author: Wenchen Fan <wenchen@databricks.com>
Author: hyukjinkwon <gurwls223@gmail.com>
Author: Wenchen Fan <cloud0fan@gmail.com>

Closes #17315 from cloud-fan/bad-record2.
2017-03-20 21:43:14 -07:00
Dongjoon Hyun 21e366aea5 [SPARK-19912][SQL] String literals should be escaped for Hive metastore partition pruning
## What changes were proposed in this pull request?

Since current `HiveShim`'s `convertFilters` does not escape the string literals. There exists the following correctness issues. This PR aims to return the correct result and also shows the more clear exception message.

**BEFORE**

```scala
scala> Seq((1, "p1", "q1"), (2, "p1\" and q=\"q1", "q2")).toDF("a", "p", "q").write.partitionBy("p", "q").saveAsTable("t1")

scala> spark.table("t1").filter($"p" === "p1\" and q=\"q1").select($"a").show
+---+
|  a|
+---+
+---+

scala> spark.table("t1").filter($"p" === "'\"").select($"a").show
java.lang.RuntimeException: Caught Hive MetaException attempting to get partition metadata by filter from ...
```

**AFTER**

```scala
scala> spark.table("t1").filter($"p" === "p1\" and q=\"q1").select($"a").show
+---+
|  a|
+---+
|  2|
+---+

scala> spark.table("t1").filter($"p" === "'\"").select($"a").show
java.lang.UnsupportedOperationException: Partition filter cannot have both `"` and `'` characters
```

## How was this patch tested?

Pass the Jenkins test with new test cases.

Author: Dongjoon Hyun <dongjoon@apache.org>

Closes #17266 from dongjoon-hyun/SPARK-19912.
2017-03-21 12:17:26 +08:00
Takeshi Yamamuro 0ec1db5475 [SPARK-19980][SQL] Add NULL checks in Bean serializer
## What changes were proposed in this pull request?
A Bean serializer in `ExpressionEncoder`  could change values when Beans having NULL. A concrete example is as follows;
```
scala> :paste
class Outer extends Serializable {
  private var cls: Inner = _
  def setCls(c: Inner): Unit = cls = c
  def getCls(): Inner = cls
}

class Inner extends Serializable {
  private var str: String = _
  def setStr(s: String): Unit = str = str
  def getStr(): String = str
}

scala> Seq("""{"cls":null}""", """{"cls": {"str":null}}""").toDF().write.text("data")
scala> val encoder = Encoders.bean(classOf[Outer])
scala> val schema = encoder.schema
scala> val df = spark.read.schema(schema).json("data").as[Outer](encoder)
scala> df.show
+------+
|   cls|
+------+
|[null]|
|  null|
+------+

scala> df.map(x => x)(encoder).show()
+------+
|   cls|
+------+
|[null]|
|[null]|     // <-- Value changed
+------+
```

This is because the Bean serializer does not have the NULL-check expressions that the serializer of Scala's product types has. Actually, this value change does not happen in Scala's product types;

```
scala> :paste
case class Outer(cls: Inner)
case class Inner(str: String)

scala> val encoder = Encoders.product[Outer]
scala> val schema = encoder.schema
scala> val df = spark.read.schema(schema).json("data").as[Outer](encoder)
scala> df.show
+------+
|   cls|
+------+
|[null]|
|  null|
+------+

scala> df.map(x => x)(encoder).show()
+------+
|   cls|
+------+
|[null]|
|  null|
+------+
```

This pr added the NULL-check expressions in Bean serializer along with the serializer of Scala's product types.

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

Author: Takeshi Yamamuro <yamamuro@apache.org>

Closes #17347 from maropu/SPARK-19980.
2017-03-21 11:17:34 +08:00
wangzhenhua e9c91badce [SPARK-20010][SQL] Sort information is lost after sort merge join
## What changes were proposed in this pull request?

After sort merge join for inner join, now we only keep left key ordering. However, after inner join, right key has the same value and order as left key. So if we need another smj on right key, we will unnecessarily add a sort which causes additional cost.

As a more complicated example, A join B on A.key = B.key join C on B.key = C.key join D on A.key = D.key. We will unnecessarily add a sort on B.key when join {A, B} and C, and add a sort on A.key when join {A, B, C} and D.

To fix this, we need to propagate all sorted information (equivalent expressions) from bottom up through `outputOrdering` and `SortOrder`.

## How was this patch tested?

Test cases are added.

Author: wangzhenhua <wangzhenhua@huawei.com>

Closes #17339 from wzhfy/sortEnhance.
2017-03-21 10:43:17 +08:00
Zheng RuiFeng 10691d36de [SPARK-19573][SQL] Make NaN/null handling consistent in approxQuantile
## What changes were proposed in this pull request?
update `StatFunctions.multipleApproxQuantiles` to handle NaN/null

## How was this patch tested?
existing tests and added tests

Author: Zheng RuiFeng <ruifengz@foxmail.com>

Closes #16971 from zhengruifeng/quantiles_nan.
2017-03-20 18:25:59 -07:00
Dongjoon Hyun fc7554599a [SPARK-19970][SQL] Table owner should be USER instead of PRINCIPAL in kerberized clusters
## What changes were proposed in this pull request?

In the kerberized hadoop cluster, when Spark creates tables, the owner of tables are filled with PRINCIPAL strings instead of USER names. This is inconsistent with Hive and causes problems when using [ROLE](https://cwiki.apache.org/confluence/display/Hive/SQL+Standard+Based+Hive+Authorization) in Hive. We had better to fix this.

**BEFORE**
```scala
scala> sql("create table t(a int)").show
scala> sql("desc formatted t").show(false)
...
|Owner:                      |sparkEXAMPLE.COM                                         |       |
```

**AFTER**
```scala
scala> sql("create table t(a int)").show
scala> sql("desc formatted t").show(false)
...
|Owner:                      |spark                                         |       |
```

## How was this patch tested?

Manually do `create table` and `desc formatted` because this happens in Kerberized clusters.

Author: Dongjoon Hyun <dongjoon@apache.org>

Closes #17311 from dongjoon-hyun/SPARK-19970.
2017-03-20 10:07:31 -07:00
windpiger 7ce30e00b2 [SPARK-19990][SQL][TEST-MAVEN] create a temp file for file in test.jar's resource when run mvn test accross different modules
## What changes were proposed in this pull request?

After we have merged the `HiveDDLSuite` and `DDLSuite` in [SPARK-19235](https://issues.apache.org/jira/browse/SPARK-19235), we have two subclasses of `DDLSuite`, that is `HiveCatalogedDDLSuite` and `InMemoryCatalogDDLSuite`.

While `DDLSuite` is in `sql/core module`, and `HiveCatalogedDDLSuite` is in `sql/hive module`, if we mvn test
`HiveCatalogedDDLSuite`, it will run the test in its parent class `DDLSuite`, this will cause some test case failed which will get and use the test file path in `sql/core module` 's `resource`.

Because the test file path getted will start with 'jar:' like "jar:file:/home/jenkins/workspace/spark-master-test-maven-hadoop-2.6/sql/core/target/spark-sql_2.11-2.2.0-SNAPSHOT-tests.jar!/test-data/cars.csv", which will failed when new Path() in datasource.scala

This PR fix this by copy file from resource to  a temp dir.

## How was this patch tested?
N/A

Author: windpiger <songjun@outlook.com>

Closes #17338 from windpiger/fixtestfailemvn.
2017-03-20 21:36:00 +08:00
Ioana Delaney 8163911594 [SPARK-17791][SQL] Join reordering using star schema detection
## What changes were proposed in this pull request?

Star schema consists of one or more fact tables referencing a number of dimension tables. In general, queries against star schema are expected to run fast because of the established RI constraints among the tables. This design proposes a join reordering based on natural, generally accepted heuristics for star schema queries:
- Finds the star join with the largest fact table and places it on the driving arm of the left-deep join. This plan avoids large tables on the inner, and thus favors hash joins.
- Applies the most selective dimensions early in the plan to reduce the amount of data flow.

The design document was included in SPARK-17791.

Link to the google doc: [StarSchemaDetection](https://docs.google.com/document/d/1UAfwbm_A6wo7goHlVZfYK99pqDMEZUumi7pubJXETEA/edit?usp=sharing)

## How was this patch tested?

A new test suite StarJoinSuite.scala was implemented.

Author: Ioana Delaney <ioanamdelaney@gmail.com>

Closes #15363 from ioana-delaney/starJoinReord2.
2017-03-20 16:04:58 +08:00
wangzhenhua 965a5abcff [SPARK-19994][SQL] Wrong outputOrdering for right/full outer smj
## What changes were proposed in this pull request?

For right outer join, values of the left key will be filled with nulls if it can't match the value of the right key, so `nullOrdering` of the left key can't be guaranteed. We should output right key order instead of left key order.

For full outer join, neither left key nor right key guarantees `nullOrdering`. We should not output any ordering.

In tests, besides adding three test cases for left/right/full outer sort merge join, this patch also reorganizes code in `PlannerSuite` by putting together tests for `Sort`, and also extracts common logic in Sort tests into a method.

## How was this patch tested?

Corresponding test cases are added.

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

Closes #17331 from wzhfy/wrongOrdering.
2017-03-20 14:37:23 +08:00
hyukjinkwon 0cdcf91145 [SPARK-19849][SQL] Support ArrayType in to_json to produce JSON array
## What changes were proposed in this pull request?

This PR proposes to support an array of struct type in `to_json` as below:

```scala
import org.apache.spark.sql.functions._

val df = Seq(Tuple1(Tuple1(1) :: Nil)).toDF("a")
df.select(to_json($"a").as("json")).show()
```

```
+----------+
|      json|
+----------+
|[{"_1":1}]|
+----------+
```

Currently, it throws an exception as below (a newline manually inserted for readability):

```
org.apache.spark.sql.AnalysisException: cannot resolve 'structtojson(`array`)' due to data type
mismatch: structtojson requires that the expression is a struct expression.;;
```

This allows the roundtrip with `from_json` as below:

```scala
import org.apache.spark.sql.functions._
import org.apache.spark.sql.types._

val schema = ArrayType(StructType(StructField("a", IntegerType) :: Nil))
val df = Seq("""[{"a":1}, {"a":2}]""").toDF("json").select(from_json($"json", schema).as("array"))
df.show()

// Read back.
df.select(to_json($"array").as("json")).show()
```

```
+----------+
|     array|
+----------+
|[[1], [2]]|
+----------+

+-----------------+
|             json|
+-----------------+
|[{"a":1},{"a":2}]|
+-----------------+
```

Also, this PR proposes to rename from `StructToJson` to `StructsToJson ` and `JsonToStruct` to `JsonToStructs`.

## How was this patch tested?

Unit tests in `JsonFunctionsSuite` and `JsonExpressionsSuite` for Scala, doctest for Python and test in `test_sparkSQL.R` for R.

Author: hyukjinkwon <gurwls223@gmail.com>

Closes #17192 from HyukjinKwon/SPARK-19849.
2017-03-19 22:33:01 -07:00
Tathagata Das 990af630d0 [SPARK-19067][SS] Processing-time-based timeout in MapGroupsWithState
## What changes were proposed in this pull request?

When a key does not get any new data in `mapGroupsWithState`, the mapping function is never called on it. So we need a timeout feature that calls the function again in such cases, so that the user can decide whether to continue waiting or clean up (remove state, save stuff externally, etc.).
Timeouts can be either based on processing time or event time. This JIRA is for processing time, but defines the high level API design for both. The usage would look like this.
```
def stateFunction(key: K, value: Iterator[V], state: KeyedState[S]): U = {
  ...
  state.setTimeoutDuration(10000)
  ...
}

dataset					// type is Dataset[T]
  .groupByKey[K](keyingFunc)   // generates KeyValueGroupedDataset[K, T]
  .mapGroupsWithState[S, U](
     func = stateFunction,
     timeout = KeyedStateTimeout.withProcessingTime)	// returns Dataset[U]
```

Note the following design aspects.

- The timeout type is provided as a param in mapGroupsWithState as a parameter global to all the keys. This is so that the planner knows this at planning time, and accordingly optimize the execution based on whether to saves extra info in state or not (e.g. timeout durations or timestamps).

- The exact timeout duration is provided inside the function call so that it can be customized on a per key basis.

- When the timeout occurs for a key, the function is called with no values, and KeyedState.isTimingOut() set to true.

- The timeout is reset for key every time the function is called on the key, that is, when the key has new data, or the key has timed out. So the user has to set the timeout duration everytime the function is called, otherwise there will not be any timeout set.

Guarantees provided on timeout of key, when timeout duration is D ms:
- Timeout will never be called before real clock time has advanced by D ms
- Timeout will be called eventually when there is a trigger with any data in it (i.e. after D ms). So there is a no strict upper bound on when the timeout would occur. For example, if there is no data in the stream (for any key) for a while, then the timeout will not be hit.

Implementation details:
- Added new param to `mapGroupsWithState` for timeout
- Added new method to `StateStore` to filter data based on timeout timestamp
- Changed the internal map type of `HDFSBackedStateStore` from Java's `HashMap` to `ConcurrentHashMap` as the latter allows weakly-consistent fail-safe iterators on the map data. See comments in code for more details.
- Refactored logic of `MapGroupsWithStateExec` to
  - Save timeout info to state store for each key that has data.
  - Then, filter states that should be timed out based on the current batch processing timestamp.
- Moved KeyedState for `o.a.s.sql` to `o.a.s.sql.streaming`. I remember that this was a feedback in the MapGroupsWithState PR that I had forgotten to address.

## How was this patch tested?
New unit tests in
- MapGroupsWithStateSuite for timeouts.
- StateStoreSuite for new APIs in StateStore.

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

Closes #17179 from tdas/mapgroupwithstate-timeout.
2017-03-19 14:07:49 -07:00
Xiao Li 0ee9fbf51a [SPARK-19990][TEST] Use the database after Hive's current Database is dropped
### What changes were proposed in this pull request?
This PR is to fix the following test failure in maven and the PR https://github.com/apache/spark/pull/15363.

> org.apache.spark.sql.hive.orc.OrcSourceSuite SPARK-19459/SPARK-18220: read char/varchar column written by Hive

The[ test history](https://spark-tests.appspot.com/test-details?suite_name=org.apache.spark.sql.hive.orc.OrcSourceSuite&test_name=SPARK-19459%2FSPARK-18220%3A+read+char%2Fvarchar+column+written+by+Hive) shows all the maven builds failed this test case with the same error message.

```
FAILED: SemanticException [Error 10072]: Database does not exist: db2

      org.apache.spark.sql.execution.QueryExecutionException: FAILED: SemanticException [Error 10072]: Database does not exist: db2
      at org.apache.spark.sql.hive.client.HiveClientImpl$$anonfun$runHive$1.apply(HiveClientImpl.scala:637)
      at org.apache.spark.sql.hive.client.HiveClientImpl$$anonfun$runHive$1.apply(HiveClientImpl.scala:621)
      at org.apache.spark.sql.hive.client.HiveClientImpl$$anonfun$withHiveState$1.apply(HiveClientImpl.scala:288)
      at org.apache.spark.sql.hive.client.HiveClientImpl.liftedTree1$1(HiveClientImpl.scala:229)
      at org.apache.spark.sql.hive.client.HiveClientImpl.retryLocked(HiveClientImpl.scala:228)
      at org.apache.spark.sql.hive.client.HiveClientImpl.withHiveState(HiveClientImpl.scala:271)
      at org.apache.spark.sql.hive.client.HiveClientImpl.runHive(HiveClientImpl.scala:621)
      at org.apache.spark.sql.hive.client.HiveClientImpl.runSqlHive(HiveClientImpl.scala:611)
      at org.apache.spark.sql.hive.orc.OrcSuite$$anonfun$7.apply$mcV$sp(OrcSourceSuite.scala:160)
      at org.apache.spark.sql.hive.orc.OrcSuite$$anonfun$7.apply(OrcSourceSuite.scala:155)
      at org.apache.spark.sql.hive.orc.OrcSuite$$anonfun$7.apply(OrcSourceSuite.scala:155)
      at org.scalatest.Transformer$$anonfun$apply$1.apply$mcV$sp(Transformer.scala:22)
      at org.scalatest.OutcomeOf$class.outcomeOf(OutcomeOf.scala:85)
      at org.scalatest.OutcomeOf$.outcomeOf(OutcomeOf.scala:104)
      at org.scalatest.Transformer.apply(Transformer.scala:22)
      at org.scalatest.Transformer.apply(Transformer.scala:20)
      at org.scalatest.FunSuiteLike$$anon$1.apply(FunSuiteLike.scala:166)
      at org.apache.spark.SparkFunSuite.withFixture(SparkFunSuite.scala:68)
      at org.scalatest.FunSuiteLike$class.invokeWithFixture$1(FunSuiteLike.scala:163)
      at org.scalatest.FunSuiteLike$$anonfun$runTest$1.apply(FunSuiteLike.scala:175)
```

### How was this patch tested?
N/A

Author: Xiao Li <gatorsmile@gmail.com>

Closes #17344 from gatorsmile/testtest.
2017-03-19 13:52:22 -07:00
Takeshi Yamamuro ccba622e35 [SPARK-19896][SQL] Throw an exception if case classes have circular references in toDS
## What changes were proposed in this pull request?
If case classes have circular references below, it throws StackOverflowError;
```
scala> :pasge
case class classA(i: Int, cls: classB)
case class classB(cls: classA)

scala> Seq(classA(0, null)).toDS()
java.lang.StackOverflowError
  at scala.reflect.internal.Symbols$Symbol.info(Symbols.scala:1494)
  at scala.reflect.runtime.JavaMirrors$JavaMirror$$anon$1.scala$reflect$runtime$SynchronizedSymbols$SynchronizedSymbol$$super$info(JavaMirrors.scala:66)
  at scala.reflect.runtime.SynchronizedSymbols$SynchronizedSymbol$$anonfun$info$1.apply(SynchronizedSymbols.scala:127)
  at scala.reflect.runtime.SynchronizedSymbols$SynchronizedSymbol$$anonfun$info$1.apply(SynchronizedSymbols.scala:127)
  at scala.reflect.runtime.Gil$class.gilSynchronized(Gil.scala:19)
  at scala.reflect.runtime.JavaUniverse.gilSynchronized(JavaUniverse.scala:16)
  at scala.reflect.runtime.SynchronizedSymbols$SynchronizedSymbol$class.gilSynchronizedIfNotThreadsafe(SynchronizedSymbols.scala:123)
  at scala.reflect.runtime.JavaMirrors$JavaMirror$$anon$1.gilSynchronizedIfNotThreadsafe(JavaMirrors.scala:66)
  at scala.reflect.runtime.SynchronizedSymbols$SynchronizedSymbol$class.info(SynchronizedSymbols.scala:127)
  at scala.reflect.runtime.JavaMirrors$JavaMirror$$anon$1.info(JavaMirrors.scala:66)
  at scala.reflect.internal.Mirrors$RootsBase.getModuleOrClass(Mirrors.scala:48)
  at scala.reflect.internal.Mirrors$RootsBase.getModuleOrClass(Mirrors.scala:45)
  at scala.reflect.internal.Mirrors$RootsBase.getModuleOrClass(Mirrors.scala:45)
  at scala.reflect.internal.Mirrors$RootsBase.getModuleOrClass(Mirrors.scala:45)
  at scala.reflect.internal.Mirrors$RootsBase.getModuleOrClass(Mirrors.scala:45)
```
This pr added code to throw UnsupportedOperationException in that case as follows;
```
scala> :paste
case class A(cls: B)
case class B(cls: A)

scala> Seq(A(null)).toDS()
java.lang.UnsupportedOperationException: cannot have circular references in class, but got the circular reference of class B
  at org.apache.spark.sql.catalyst.ScalaReflection$.org$apache$spark$sql$catalyst$ScalaReflection$$serializerFor(ScalaReflection.scala:627)
  at org.apache.spark.sql.catalyst.ScalaReflection$$anonfun$9.apply(ScalaReflection.scala:644)
  at org.apache.spark.sql.catalyst.ScalaReflection$$anonfun$9.apply(ScalaReflection.scala:632)
  at scala.collection.TraversableLike$$anonfun$flatMap$1.apply(TraversableLike.scala:241)
  at scala.collection.TraversableLike$$anonfun$flatMap$1.apply(TraversableLike.scala:241)
  at scala.collection.immutable.List.foreach(List.scala:381)
  at scala.collection.TraversableLike$class.flatMap(TraversableLike.scala:241)
```

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

Author: Takeshi Yamamuro <yamamuro@apache.org>

Closes #17318 from maropu/SPARK-19896.
2017-03-18 14:40:16 +08:00
wangzhenhua c083b6b7de [SPARK-19915][SQL] Exclude cartesian product candidates to reduce the search space
## What changes were proposed in this pull request?

We have some concerns about removing size in the cost model [in the previous pr](https://github.com/apache/spark/pull/17240). It's a tradeoff between code structure and algorithm completeness. I tend to keep the size and thus create this new pr without changing cost model.

What this pr does:
1. We only consider consecutive inner joinable items, thus excluding cartesian products in reordering procedure. This significantly reduces the search space and memory overhead of memo. Otherwise every combination of items will exist in the memo.
2. This pr also includes a bug fix: if a leaf item is a project(_, child), current solution will miss the project.

## How was this patch tested?

Added test cases.

Author: wangzhenhua <wangzhenhua@huawei.com>

Closes #17286 from wzhfy/joinReorder3.
2017-03-18 14:07:25 +08:00
Jacek Laskowski 6326d406b9 [SQL][MINOR] Fix scaladoc for UDFRegistration
## What changes were proposed in this pull request?

Fix scaladoc for UDFRegistration

## How was this patch tested?

local build

Author: Jacek Laskowski <jacek@japila.pl>

Closes #17337 from jaceklaskowski/udfregistration-scaladoc.
2017-03-17 21:55:10 -07:00
Kunal Khamar 3783539d7a [SPARK-19873][SS] Record num shuffle partitions in offset log and enforce in next batch.
## What changes were proposed in this pull request?

If the user changes the shuffle partition number between batches, Streaming aggregation will fail.

Here are some possible cases:

- Change "spark.sql.shuffle.partitions"
- Use "repartition" and change the partition number in codes
- RangePartitioner doesn't generate deterministic partitions. Right now it's safe as we disallow sort before aggregation. Not sure if we will add some operators using RangePartitioner in future.

## How was this patch tested?

- Unit tests
- Manual tests
  - forward compatibility tested by using the new `OffsetSeqMetadata` json with Spark v2.1.0

Author: Kunal Khamar <kkhamar@outlook.com>

Closes #17216 from kunalkhamar/num-partitions.
2017-03-17 16:16:22 -07:00
Takeshi Yamamuro 7de66bae58 [SPARK-19967][SQL] Add from_json in FunctionRegistry
## What changes were proposed in this pull request?
This pr added entries in `FunctionRegistry` and supported `from_json` in SQL.

## How was this patch tested?
Added tests in `JsonFunctionsSuite` and `SQLQueryTestSuite`.

Author: Takeshi Yamamuro <yamamuro@apache.org>

Closes #17320 from maropu/SPARK-19967.
2017-03-17 14:51:59 -07:00
Andrew Ray 13538cf3dd [SPARK-19882][SQL] Pivot with null as a distinct pivot value throws NPE
## What changes were proposed in this pull request?

Allows null values of the pivot column to be included in the pivot values list without throwing NPE

Note this PR was made as an alternative to #17224 but preserves the two phase aggregate operation that is needed for good performance.

## How was this patch tested?

Additional unit test

Author: Andrew Ray <ray.andrew@gmail.com>

Closes #17226 from aray/pivot-null.
2017-03-17 16:43:42 +08:00
Reynold Xin 8537c00e0a [SPARK-19987][SQL] Pass all filters into FileIndex
## What changes were proposed in this pull request?
This is a tiny teeny refactoring to pass data filters also to the FileIndex, so FileIndex can have a more global view on predicates.

## How was this patch tested?
Change should be covered by existing test cases.

Author: Reynold Xin <rxin@databricks.com>

Closes #17322 from rxin/SPARK-19987.
2017-03-16 18:31:57 -07:00
Liwei Lin 2ea214dd05 [SPARK-19721][SS] Good error message for version mismatch in log files
## Problem

There are several places where we write out version identifiers in various logs for structured streaming (usually `v1`). However, in the places where we check for this, we throw a confusing error message.

## What changes were proposed in this pull request?

This patch made two major changes:
1. added a `parseVersion(...)` method, and based on this method, fixed the following places the way they did version checking (no other place needed to do this checking):
```
HDFSMetadataLog
  - CompactibleFileStreamLog  ------------> fixed with this patch
    - FileStreamSourceLog  ---------------> inherited the fix of `CompactibleFileStreamLog`
    - FileStreamSinkLog  -----------------> inherited the fix of `CompactibleFileStreamLog`
  - OffsetSeqLog  ------------------------> fixed with this patch
  - anonymous subclass in KafkaSource  ---> fixed with this patch
```

2. changed the type of `FileStreamSinkLog.VERSION`, `FileStreamSourceLog.VERSION` etc. from `String` to `Int`, so that we can identify newer versions via `version > 1` instead of `version != "v1"`
    - note this didn't break any backwards compatibility -- we are still writing out `"v1"` and reading back `"v1"`

## Exception message with this patch
```
java.lang.IllegalStateException: Failed to read log file /private/var/folders/nn/82rmvkk568sd8p3p8tb33trw0000gn/T/spark-86867b65-0069-4ef1-b0eb-d8bd258ff5b8/0. UnsupportedLogVersion: maximum supported log version is v1, but encountered v99. The log file was produced by a newer version of Spark and cannot be read by this version. Please upgrade.
	at org.apache.spark.sql.execution.streaming.HDFSMetadataLog.get(HDFSMetadataLog.scala:202)
	at org.apache.spark.sql.execution.streaming.OffsetSeqLogSuite$$anonfun$3$$anonfun$apply$mcV$sp$2.apply(OffsetSeqLogSuite.scala:78)
	at org.apache.spark.sql.execution.streaming.OffsetSeqLogSuite$$anonfun$3$$anonfun$apply$mcV$sp$2.apply(OffsetSeqLogSuite.scala:75)
	at org.apache.spark.sql.test.SQLTestUtils$class.withTempDir(SQLTestUtils.scala:133)
	at org.apache.spark.sql.execution.streaming.OffsetSeqLogSuite.withTempDir(OffsetSeqLogSuite.scala:26)
	at org.apache.spark.sql.execution.streaming.OffsetSeqLogSuite$$anonfun$3.apply$mcV$sp(OffsetSeqLogSuite.scala:75)
	at org.apache.spark.sql.execution.streaming.OffsetSeqLogSuite$$anonfun$3.apply(OffsetSeqLogSuite.scala:75)
	at org.apache.spark.sql.execution.streaming.OffsetSeqLogSuite$$anonfun$3.apply(OffsetSeqLogSuite.scala:75)
	at org.scalatest.Transformer$$anonfun$apply$1.apply$mcV$sp(Transformer.scala:22)
	at org.scalatest.OutcomeOf$class.outcomeOf(OutcomeOf.scala:85)
```

## How was this patch tested?

unit tests

Author: Liwei Lin <lwlin7@gmail.com>

Closes #17070 from lw-lin/better-msg.
2017-03-16 13:05:36 -07:00
windpiger 8e8f898335 [SPARK-19945][SQL] add test suite for SessionCatalog with HiveExternalCatalog
## What changes were proposed in this pull request?

Currently `SessionCatalogSuite` is only for `InMemoryCatalog`, there is no suite for `HiveExternalCatalog`.
And there are some ddl function is not proper to test in `ExternalCatalogSuite`, because some logic are not full implement in `ExternalCatalog`, these ddl functions are full implement in `SessionCatalog`(e.g. merge the same logic from `ExternalCatalog` up to `SessionCatalog` ).
It is better to test it in `SessionCatalogSuite` for this situation.

So we should add a test suite for `SessionCatalog` with `HiveExternalCatalog`

The main change is that in `SessionCatalogSuite` add two functions:
`withBasicCatalog` and `withEmptyCatalog`
And replace the code like  `val catalog = new SessionCatalog(newBasicCatalog)` with above two functions

## How was this patch tested?
add `HiveExternalSessionCatalogSuite`

Author: windpiger <songjun@outlook.com>

Closes #17287 from windpiger/sessioncatalogsuit.
2017-03-16 11:34:13 -07:00
Xiao Li 1472cac4bb [SPARK-19830][SQL] Add parseTableSchema API to ParserInterface
### What changes were proposed in this pull request?

Specifying the table schema in DDL formats is needed for different scenarios. For example,
- [specifying the schema in SQL function `from_json` using DDL formats](https://issues.apache.org/jira/browse/SPARK-19637), which is suggested by marmbrus ,
- [specifying the customized JDBC data types](https://github.com/apache/spark/pull/16209).

These two PRs need users to use the JSON format to specify the table schema. This is not user friendly.

This PR is to provide a `parseTableSchema` API in `ParserInterface`.

### How was this patch tested?
Added a test suite `TableSchemaParserSuite`

Author: Xiao Li <gatorsmile@gmail.com>

Closes #17171 from gatorsmile/parseDDLStmt.
2017-03-16 12:06:20 +08:00
Takeshi Yamamuro 21f333c635 [SPARK-19751][SQL] Throw an exception if bean class has one's own class in fields
## What changes were proposed in this pull request?
The current master throws `StackOverflowError` in `createDataFrame`/`createDataset` if bean has one's own class in fields;
```
public class SelfClassInFieldBean implements Serializable {
  private SelfClassInFieldBean child;
  ...
}
```
This pr added code to throw `UnsupportedOperationException` in that case as soon as possible.

## How was this patch tested?
Added tests in `JavaDataFrameSuite` and `JavaDatasetSuite`.

Author: Takeshi Yamamuro <yamamuro@apache.org>

Closes #17188 from maropu/SPARK-19751.
2017-03-16 08:50:01 +08:00
windpiger fc9314671c [SPARK-19961][SQL][MINOR] unify a erro msg when drop databse for HiveExternalCatalog and InMemoryCatalog
## What changes were proposed in this pull request?

unify a exception erro msg for dropdatabase when the database still have some tables for HiveExternalCatalog and InMemoryCatalog
## How was this patch tested?
N/A

Author: windpiger <songjun@outlook.com>

Closes #17305 from windpiger/unifyErromsg.
2017-03-16 08:44:57 +08:00
Juliusz Sompolski 339b237dc1 [SPARK-19948] Document that saveAsTable uses catalog as source of truth for table existence.
It is quirky behaviour that saveAsTable to e.g. a JDBC source with SaveMode other
than Overwrite will nevertheless overwrite the table in the external source,
if that table was not a catalog table.

Author: Juliusz Sompolski <julek@databricks.com>

Closes #17289 from juliuszsompolski/saveAsTableDoc.
2017-03-16 08:20:47 +08:00
Liang-Chi Hsieh 7d734a6583 [SPARK-19931][SQL] InMemoryTableScanExec should rewrite output partitioning and ordering when aliasing output attributes
## What changes were proposed in this pull request?

Now `InMemoryTableScanExec` simply takes the `outputPartitioning` and `outputOrdering` from the associated `InMemoryRelation`'s `child.outputPartitioning` and `outputOrdering`.

However, `InMemoryTableScanExec` can alias the output attributes. In this case, its `outputPartitioning` and `outputOrdering` are not correct and its parent operators can't correctly determine its data distribution.

## How was this patch tested?

Jenkins tests.

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

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

Closes #17175 from viirya/ensure-no-unnecessary-shuffle.
2017-03-16 08:18:36 +08:00
Dongjoon Hyun 54a3697f1f [MINOR][CORE] Fix a info message of prunePartitions
## What changes were proposed in this pull request?

`PrunedInMemoryFileIndex.prunePartitions` shows `pruned NaN% partitions` for the following case.

```scala
scala> Seq.empty[(String, String)].toDF("a", "p").write.partitionBy("p").saveAsTable("t1")

scala> sc.setLogLevel("INFO")

scala> spark.table("t1").filter($"p" === "1").select($"a").show
...
17/03/13 00:33:04 INFO PrunedInMemoryFileIndex: Selected 0 partitions out of 0, pruned NaN% partitions.
```

After this PR, the message looks like this.
```scala
17/03/15 10:39:48 INFO PrunedInMemoryFileIndex: Selected 0 partitions out of 0, pruned 0 partitions.
```

## How was this patch tested?

Pass the Jenkins with the existing tests.

Author: Dongjoon Hyun <dongjoon@apache.org>

Closes #17273 from dongjoon-hyun/SPARK-EMPTY-PARTITION.
2017-03-15 15:01:16 -07:00
Tejas Patil 02c274eaba [SPARK-13450] Introduce ExternalAppendOnlyUnsafeRowArray. Change CartesianProductExec, SortMergeJoin, WindowExec to use it
## What issue does this PR address ?

Jira: https://issues.apache.org/jira/browse/SPARK-13450

In `SortMergeJoinExec`, rows of the right relation having the same value for a join key are buffered in-memory. In case of skew, this causes OOMs (see comments in SPARK-13450 for more details). Heap dump from a failed job confirms this : https://issues.apache.org/jira/secure/attachment/12846382/heap-dump-analysis.png . While its possible to increase the heap size to workaround, Spark should be resilient to such issues as skews can happen arbitrarily.

## Change proposed in this pull request

- Introduces `ExternalAppendOnlyUnsafeRowArray`
  - It holds `UnsafeRow`s in-memory upto a certain threshold.
  - After the threshold is hit, it switches to `UnsafeExternalSorter` which enables spilling of the rows to disk. It does NOT sort the data.
  - Allows iterating the array multiple times. However, any alteration to the array (using `add` or `clear`) will invalidate the existing iterator(s)
- `WindowExec` was already using `UnsafeExternalSorter` to support spilling. Changed it to use the new array
- Changed `SortMergeJoinExec` to use the new array implementation
  - NOTE: I have not changed FULL OUTER JOIN to use this new array implementation. Changing that will need more surgery and I will rather put up a separate PR for that once this gets in.
- Changed `CartesianProductExec` to use the new array implementation

#### Note for reviewers

The diff can be divided into 3 parts. My motive behind having all the changes in a single PR was to demonstrate that the API is sane and supports 2 use cases. If reviewing as 3 separate PRs would help, I am happy to make the split.

## How was this patch tested ?

#### Unit testing
- Added unit tests `ExternalAppendOnlyUnsafeRowArray` to validate all its APIs and access patterns
- Added unit test for `SortMergeExec`
  - with and without spill for inner join, left outer join, right outer join to confirm that the spill threshold config behaves as expected and output is as expected.
  - This PR touches the scanning logic in `SortMergeExec` for _all_ joins (except FULL OUTER JOIN). However, I expect existing test cases to cover that there is no regression in correctness.
- Added unit test for `WindowExec` to check behavior of spilling and correctness of results.

#### Stress testing
- Confirmed that OOM is gone by running against a production job which used to OOM
- Since I cannot share details about prod workload externally, created synthetic data to mimic the issue. Ran before and after the fix to demonstrate the issue and query success with this PR

Generating the synthetic data

```
./bin/spark-shell --driver-memory=6G

import org.apache.spark.sql._
val hc = SparkSession.builder.master("local").getOrCreate()

hc.sql("DROP TABLE IF EXISTS spark_13450_large_table").collect
hc.sql("DROP TABLE IF EXISTS spark_13450_one_row_table").collect

val df1 = (0 until 1).map(i => ("10", "100", i.toString, (i * 2).toString)).toDF("i", "j", "str1", "str2")
df1.write.format("org.apache.spark.sql.hive.orc.OrcFileFormat").bucketBy(100, "i", "j").sortBy("i", "j").saveAsTable("spark_13450_one_row_table")

val df2 = (0 until 3000000).map(i => ("10", "100", i.toString, (i * 2).toString)).toDF("i", "j", "str1", "str2")
df2.write.format("org.apache.spark.sql.hive.orc.OrcFileFormat").bucketBy(100, "i", "j").sortBy("i", "j").saveAsTable("spark_13450_large_table")
```

Ran this against trunk VS local build with this PR. OOM repros with trunk and with the fix this query runs fine.

```
./bin/spark-shell --driver-java-options="-XX:+HeapDumpOnOutOfMemoryError -XX:HeapDumpPath=/tmp/spark.driver.heapdump.hprof"

import org.apache.spark.sql._
val hc = SparkSession.builder.master("local").getOrCreate()
hc.sql("SET spark.sql.autoBroadcastJoinThreshold=1")
hc.sql("SET spark.sql.sortMergeJoinExec.buffer.spill.threshold=10000")

hc.sql("DROP TABLE IF EXISTS spark_13450_result").collect
hc.sql("""
  CREATE TABLE spark_13450_result
  AS
  SELECT
    a.i AS a_i, a.j AS a_j, a.str1 AS a_str1, a.str2 AS a_str2,
    b.i AS b_i, b.j AS b_j, b.str1 AS b_str1, b.str2 AS b_str2
  FROM
    spark_13450_one_row_table a
  JOIN
    spark_13450_large_table b
  ON
    a.i=b.i AND
    a.j=b.j
""")
```

## Performance comparison

### Macro-benchmark

I ran a SMB join query over two real world tables (2 trillion rows (40 TB) and 6 million rows (120 GB)). Note that this dataset does not have skew so no spill happened. I saw improvement in CPU time by 2-4% over version without this PR. This did not add up as I was expected some regression. I think allocating array of capacity of 128 at the start (instead of starting with default size 16) is the sole reason for the perf. gain : https://github.com/tejasapatil/spark/blob/SPARK-13450_smb_buffer_oom/sql/core/src/main/scala/org/apache/spark/sql/execution/ExternalAppendOnlyUnsafeRowArray.scala#L43 . I could remove that and rerun, but effectively the change will be deployed in this form and I wanted to see the effect of it over large workload.

### Micro-benchmark

Two types of benchmarking can be found in `ExternalAppendOnlyUnsafeRowArrayBenchmark`:

[A] Comparing `ExternalAppendOnlyUnsafeRowArray` against raw `ArrayBuffer` when all rows fit in-memory and there is no spill

```
Array with 1000 rows:                    Best/Avg Time(ms)    Rate(M/s)   Per Row(ns)   Relative
------------------------------------------------------------------------------------------------
ArrayBuffer                                   7821 / 7941         33.5          29.8       1.0X
ExternalAppendOnlyUnsafeRowArray              8798 / 8819         29.8          33.6       0.9X

Array with 30000 rows:                   Best/Avg Time(ms)    Rate(M/s)   Per Row(ns)   Relative
------------------------------------------------------------------------------------------------
ArrayBuffer                                 19200 / 19206         25.6          39.1       1.0X
ExternalAppendOnlyUnsafeRowArray            19558 / 19562         25.1          39.8       1.0X

Array with 100000 rows:                  Best/Avg Time(ms)    Rate(M/s)   Per Row(ns)   Relative
------------------------------------------------------------------------------------------------
ArrayBuffer                                   5949 / 6028         17.2          58.1       1.0X
ExternalAppendOnlyUnsafeRowArray              6078 / 6138         16.8          59.4       1.0X
```

[B] Comparing `ExternalAppendOnlyUnsafeRowArray` against raw `UnsafeExternalSorter` when there is spilling of data

```
Spilling with 1000 rows:                 Best/Avg Time(ms)    Rate(M/s)   Per Row(ns)   Relative
------------------------------------------------------------------------------------------------
UnsafeExternalSorter                          9239 / 9470         28.4          35.2       1.0X
ExternalAppendOnlyUnsafeRowArray              8857 / 8909         29.6          33.8       1.0X

Spilling with 10000 rows:                Best/Avg Time(ms)    Rate(M/s)   Per Row(ns)   Relative
------------------------------------------------------------------------------------------------
UnsafeExternalSorter                             4 /    5         39.3          25.5       1.0X
ExternalAppendOnlyUnsafeRowArray                 5 /    6         29.8          33.5       0.8X
```

Author: Tejas Patil <tejasp@fb.com>

Closes #16909 from tejasapatil/SPARK-13450_smb_buffer_oom.
2017-03-15 20:18:39 +01:00
jiangxingbo ee36bc1c90 [SPARK-19877][SQL] Restrict the nested level of a view
## What changes were proposed in this pull request?

We should restrict the nested level of a view, to avoid stack overflow exception during the view resolution.

## How was this patch tested?

Add new test case in `SQLViewSuite`.

Author: jiangxingbo <jiangxb1987@gmail.com>

Closes #17241 from jiangxb1987/view-depth.
2017-03-14 23:57:54 -07:00
Liwei Lin e1ac553402 [SPARK-19817][SS] Make it clear that timeZone is a general option in DataStreamReader/Writer
## What changes were proposed in this pull request?

As timezone setting can also affect partition values, it works for all formats, we should make it clear.

## How was this patch tested?

N/A

Author: Liwei Lin <lwlin7@gmail.com>

Closes #17299 from lw-lin/timezone.
2017-03-14 22:30:16 -07:00
Xiao Li f9a93b1b4a [SPARK-18112][SQL] Support reading data from Hive 2.1 metastore
### What changes were proposed in this pull request?
This PR is to support reading data from Hive 2.1 metastore. Need to update shim class because of the Hive API changes caused by the following three Hive JIRAs:
- [HIVE-12730 MetadataUpdater: provide a mechanism to edit the basic statistics of a table (or a partition)](https://issues.apache.org/jira/browse/HIVE-12730)
- [Hive-13341 Stats state is not captured correctly: differentiate load table and create table](https://issues.apache.org/jira/browse/HIVE-13341)
- [HIVE-13622 WriteSet tracking optimizations](https://issues.apache.org/jira/browse/HIVE-13622)

There are three new fields added to Hive APIs.
- `boolean hasFollowingStatsTask`. We always set it to `false`. This is to keep the existing behavior unchanged (starting from 0.13), no matter which Hive metastore client version users choose. If we set it to `true`, the basic table statistics is not collected by Hive. For example,

```SQL
	CREATE TABLE tbl AS SELECT 1 AS a
```
When setting `hasFollowingStatsTask ` to `false`, the table properties is like
```
	Properties: [numFiles=1, transient_lastDdlTime=1489513927, totalSize=2]
```
When setting `hasFollowingStatsTask ` to `true`, the table properties is like
```
	Properties: [transient_lastDdlTime=1489513563]
```

- `AcidUtils.Operation operation`. Obviously, we do not support ACID. Thus, we set it to `AcidUtils.Operation.NOT_ACID`.
- `EnvironmentContext environmentContext`. So far, this is always set to `null`. This was introduced for supporting DDL `alter table s update statistics set ('numRows'='NaN')`. Using this DDL, users can specify the statistics. So far, our Spark SQL does not need it, because we use different table properties to store our generated statistics values. However, when Spark SQL issues ALTER TABLE DDL statements, Hive metastore always automatically invalidate the Hive-generated statistics.

In the follow-up PR, we can fix it by explicitly adding a property to `environmentContext`.
```JAVA
putToProperties(StatsSetupConst.STATS_GENERATED, StatsSetupConst.USER)
```
Another alternative is to set `DO_NOT_UPDATE_STATS`to `TRUE`. See the Hive JIRA: https://issues.apache.org/jira/browse/HIVE-15653. We will not address it in this PR.

### How was this patch tested?
Added test cases to VersionsSuite.scala

Author: Xiao Li <gatorsmile@gmail.com>

Closes #17232 from gatorsmile/Hive21.
2017-03-15 10:53:58 +08:00
hyukjinkwon d1f6c64c4b [SPARK-19828][R] Support array type in from_json in R
## What changes were proposed in this pull request?

Since we could not directly define the array type in R, this PR proposes to support array types in R as string types that are used in `structField` as below:

```R
jsonArr <- "[{\"name\":\"Bob\"}, {\"name\":\"Alice\"}]"
df <- as.DataFrame(list(list("people" = jsonArr)))
collect(select(df, alias(from_json(df$people, "array<struct<name:string>>"), "arrcol")))
```

prints

```R
      arrcol
1 Bob, Alice
```

## How was this patch tested?

Unit tests in `test_sparkSQL.R`.

Author: hyukjinkwon <gurwls223@gmail.com>

Closes #17178 from HyukjinKwon/SPARK-19828.
2017-03-14 19:51:25 -07:00
hyukjinkwon 8fb2a02e2c [SPARK-19918][SQL] Use TextFileFormat in implementation of TextInputJsonDataSource
## What changes were proposed in this pull request?

This PR proposes to use text datasource when Json schema inference.

This basically proposes the similar approach in https://github.com/apache/spark/pull/15813 If we use Dataset for initial loading when inferring the schema, there are advantages. Please refer SPARK-18362

It seems JSON one was supposed to be fixed together but taken out according to https://github.com/apache/spark/pull/15813

> A similar problem also affects the JSON file format and this patch originally fixed that as well, but I've decided to split that change into a separate patch so as not to conflict with changes in another JSON PR.

Also, this seems affecting some functionalities because it does not use `FileScanRDD`. This problem is described in SPARK-19885 (but it was CSV's case).

## How was this patch tested?

Existing tests should cover this and manual test by `spark.read.json(path)` and check the UI.

Author: hyukjinkwon <gurwls223@gmail.com>

Closes #17255 from HyukjinKwon/json-filescanrdd.
2017-03-15 10:19:19 +08:00
Wenchen Fan dacc382f0c [SPARK-19887][SQL] dynamic partition keys can be null or empty string
## What changes were proposed in this pull request?

When dynamic partition value is null or empty string, we should write the data to a directory like `a=__HIVE_DEFAULT_PARTITION__`, when we read the data back, we should respect this special directory name and treat it as null.

This is the same behavior of impala, see https://issues.apache.org/jira/browse/IMPALA-252

## How was this patch tested?

new regression test

Author: Wenchen Fan <wenchen@databricks.com>

Closes #17277 from cloud-fan/partition.
2017-03-15 08:24:41 +08:00
Takuya UESHIN 7ded39c223 [SPARK-19817][SQL] Make it clear that timeZone option is a general option in DataFrameReader/Writer.
## What changes were proposed in this pull request?

As timezone setting can also affect partition values, it works for all formats, we should make it clear.

## How was this patch tested?

Existing tests.

Author: Takuya UESHIN <ueshin@databricks.com>

Closes #17281 from ueshin/issues/SPARK-19817.
2017-03-14 13:57:23 -07:00
Nattavut Sutyanyong 6eac96823c [SPARK-18966][SQL] NOT IN subquery with correlated expressions may return incorrect result
## What changes were proposed in this pull request?

This PR fixes the following problem:
````
Seq((1, 2)).toDF("a1", "a2").createOrReplaceTempView("a")
Seq[(java.lang.Integer, java.lang.Integer)]((1, null)).toDF("b1", "b2").createOrReplaceTempView("b")

// The expected result is 1 row of (1,2) as shown in the next statement.
sql("select * from a where a1 not in (select b1 from b where b2 = a2)").show
+---+---+
| a1| a2|
+---+---+
+---+---+

sql("select * from a where a1 not in (select b1 from b where b2 = 2)").show
+---+---+
| a1| a2|
+---+---+
|  1|  2|
+---+---+
````
There are a number of scenarios to consider:

1. When the correlated predicate yields a match (i.e., B.B2 = A.A2)
1.1. When the NOT IN expression yields a match (i.e., A.A1 = B.B1)
1.2. When the NOT IN expression yields no match (i.e., A.A1 = B.B1 returns false)
1.3. When A.A1 is null
1.4. When B.B1 is null
1.4.1. When A.A1 is not null
1.4.2. When A.A1 is null

2. When the correlated predicate yields no match (i.e.,B.B2 = A.A2 is false or unknown)
2.1. When B.B2 is null and A.A2 is null
2.2. When B.B2 is null and A.A2 is not null
2.3. When the value of A.A2 does not match any of B.B2

````
 A.A1   A.A2      B.B1   B.B2
-----  -----     -----  -----
    1      1         1      1    (1.1)
    2      1                     (1.2)
 null      1                     (1.3)

    1      3      null      3    (1.4.1)
 null      3                     (1.4.2)

    1   null         1   null    (2.1)
 null      2                     (2.2 & 2.3)
````

We can divide the evaluation of the above correlated NOT IN subquery into 2 groups:-

Group 1: The rows in A when there is a match from the correlated predicate (A.A1 = B.B1)

In this case, the result of the subquery is not empty and the semantics of the NOT IN depends solely on the evaluation of the equality comparison of the columns of NOT IN, i.e., A1 = B1, which says

- If A.A1 is null, the row is filtered (1.3 and 1.4.2)
- If A.A1 = B.B1, the row is filtered (1.1)
- If B.B1 is null, any rows of A in the same group (A.A2 = B.B2) is filtered (1.4.1 & 1.4.2)
- Otherwise, the row is qualified.

Hence, in this group, the result is the row from (1.2).

Group 2: The rows in A when there is no match from the correlated predicate (A.A2 = B.B2)

In this case, all the rows in A, including the rows where A.A1, are qualified because the subquery returns an empty set and by the semantics of the NOT IN, all rows from the parent side qualifies as the result set, that is, the rows from (2.1, 2.2 and 2.3).

In conclusion, the correct result set of the above query is
````
 A.A1   A.A2
-----  -----
    2      1    (1.2)
    1   null    (2.1)
 null      2    (2.2 & 2.3)
````
## How was this patch tested?
unit tests, regression tests, and new test cases focusing on the problem being fixed.

Author: Nattavut Sutyanyong <nsy.can@gmail.com>

Closes #17294 from nsyca/18966.
2017-03-14 20:34:59 +01:00
Herman van Hovell e04c05cf41 [SPARK-19933][SQL] Do not change output of a subquery
## What changes were proposed in this pull request?
The `RemoveRedundantAlias` rule can change the output attributes (the expression id's to be precise) of a query by eliminating the redundant alias producing them. This is no problem for a regular query, but can cause problems for correlated subqueries: The attributes produced by the subquery are used in the parent plan; changing them will break the parent plan.

This PR fixes this by wrapping a subquery in a `Subquery` top level node when it gets optimized. The `RemoveRedundantAlias` rule now recognizes `Subquery` and makes sure that the output attributes of the `Subquery` node are retained.

## How was this patch tested?
Added a test case to `RemoveRedundantAliasAndProjectSuite` and added a regression test to `SubquerySuite`.

Author: Herman van Hovell <hvanhovell@databricks.com>

Closes #17278 from hvanhovell/SPARK-19933.
2017-03-14 18:52:16 +01:00
Takeshi Yamamuro 6325a2f82a [SPARK-19923][SQL] Remove unnecessary type conversions per call in Hive
## What changes were proposed in this pull request?
This pr removed unnecessary type conversions per call in Hive: https://github.com/apache/spark/blob/master/sql/hive/src/main/scala/org/apache/spark/sql/hive/hiveUDFs.scala#L116

## How was this patch tested?
Existing tests

Author: Takeshi Yamamuro <yamamuro@apache.org>

Closes #17264 from maropu/SPARK-19923.
2017-03-14 18:51:05 +01:00
jiangxingbo a02a0b1703 [SPARK-18961][SQL] Support SHOW TABLE EXTENDED ... PARTITION statement
## What changes were proposed in this pull request?

We should support the statement `SHOW TABLE EXTENDED LIKE 'table_identifier' PARTITION(partition_spec)`, just like that HIVE does.
When partition is specified, the `SHOW TABLE EXTENDED` command should output the information of the partitions instead of the tables.
Note that in this statement, we require exact matched partition spec. For example:
```
CREATE TABLE show_t1(a String, b Int) PARTITIONED BY (c String, d String);
ALTER TABLE show_t1 ADD PARTITION (c='Us', d=1) PARTITION (c='Us', d=22);

-- Output the extended information of Partition(c='Us', d=1)
SHOW TABLE EXTENDED LIKE 'show_t1' PARTITION(c='Us', d=1);
-- Throw an AnalysisException
SHOW TABLE EXTENDED LIKE 'show_t1' PARTITION(c='Us');
```

## How was this patch tested?
Add new test sqls in file `show-tables.sql`.
Add new test case in `DDLSuite`.

Author: jiangxingbo <jiangxb1987@gmail.com>

Closes #16373 from jiangxb1987/show-partition-extended.
2017-03-14 10:13:50 -07:00
Herman van Hovell 1c7275efa7 [SPARK-18874][SQL] Fix 2.10 build after moving the subquery rules to optimization
## What changes were proposed in this pull request?
Commit 4ce970d714 in accidentally broke the 2.10 build for Spark. This PR fixes this by simplifying the offending pattern match.

## How was this patch tested?
Existing tests.

Author: Herman van Hovell <hvanhovell@databricks.com>

Closes #17288 from hvanhovell/SPARK-18874.
2017-03-14 14:02:48 +01:00
Herman van Hovell a0b92f73fe [SPARK-19850][SQL] Allow the use of aliases in SQL function calls
## What changes were proposed in this pull request?
We currently cannot use aliases in SQL function calls. This is inconvenient when you try to create a struct. This SQL query for example `select struct(1, 2) st`, will create a struct with column names `col1` and `col2`. This is even more problematic when we want to append a field to an existing struct. For example if we want to a field to struct `st` we would issue the following SQL query `select struct(st.*, 1) as st from src`, the result will be struct `st` with an a column with a non descriptive name `col3` (if `st` itself has 2 fields).

This PR proposes to change this by allowing the use of aliased expression in function parameters. For example `select struct(1 as a, 2 as b) st`, will create a struct with columns `a` & `b`.

## How was this patch tested?
Added a test to `ExpressionParserSuite` and added a test file for `SQLQueryTestSuite`.

Author: Herman van Hovell <hvanhovell@databricks.com>

Closes #17245 from hvanhovell/SPARK-19850.
2017-03-14 12:49:30 +01:00
Reynold Xin 0ee38a39e4 [SPARK-19944][SQL] Move SQLConf from sql/core to sql/catalyst
## What changes were proposed in this pull request?
This patch moves SQLConf from sql/core to sql/catalyst. To minimize the changes, the patch used type alias to still keep CatalystConf (as a type alias) and SimpleCatalystConf (as a concrete class that extends SQLConf).

Motivation for the change is that it is pretty weird to have SQLConf only in sql/core and then we have to duplicate config options that impact optimizer/analyzer in sql/catalyst using CatalystConf.

## How was this patch tested?
N/A

Author: Reynold Xin <rxin@databricks.com>

Closes #17285 from rxin/SPARK-19944.
2017-03-14 19:02:30 +08:00
Nattavut Sutyanyong 4ce970d714 [SPARK-18874][SQL] First phase: Deferring the correlated predicate pull up to Optimizer phase
## What changes were proposed in this pull request?
Currently Analyzer as part of ResolveSubquery, pulls up the correlated predicates to its
originating SubqueryExpression. The subquery plan is then transformed to remove the correlated
predicates after they are moved up to the outer plan. In this PR, the task of pulling up
correlated predicates is deferred to Optimizer. This is the initial work that will allow us to
support the form of correlated subqueries that we don't support today. The design document
from nsyca can be found in the following link :
[DesignDoc](https://docs.google.com/document/d/1QDZ8JwU63RwGFS6KVF54Rjj9ZJyK33d49ZWbjFBaIgU/edit#)

The brief description of code changes (hopefully to aid with code review) can be be found in the
following link:
[CodeChanges](https://docs.google.com/document/d/18mqjhL9V1An-tNta7aVE13HkALRZ5GZ24AATA-Vqqf0/edit#)

## How was this patch tested?
The test case PRs were submitted earlier using.
[16337](https://github.com/apache/spark/pull/16337) [16759](https://github.com/apache/spark/pull/16759) [16841](https://github.com/apache/spark/pull/16841) [16915](https://github.com/apache/spark/pull/16915) [16798](https://github.com/apache/spark/pull/16798) [16712](https://github.com/apache/spark/pull/16712) [16710](https://github.com/apache/spark/pull/16710) [16760](https://github.com/apache/spark/pull/16760) [16802](https://github.com/apache/spark/pull/16802)

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

Closes #16954 from dilipbiswal/SPARK-18874.
2017-03-14 10:37:10 +01:00
Xiao Li 415f9f3423 [SPARK-19921][SQL][TEST] Enable end-to-end testing using different Hive metastore versions.
### What changes were proposed in this pull request?

To improve the quality of our Spark SQL in different Hive metastore versions, this PR is to enable end-to-end testing using different versions. This PR allows the test cases in sql/hive to pass the existing Hive client to create a SparkSession.
- Since Derby does not allow concurrent connections, the pre-built Hive clients use different database from the TestHive's built-in 1.2.1 client.
- Since our test cases in sql/hive only can create a single Spark context in the same JVM, the newly created SparkSession share the same spark context with the existing TestHive's corresponding SparkSession.

### How was this patch tested?
Fixed the existing test cases.

Author: Xiao Li <gatorsmile@gmail.com>

Closes #17260 from gatorsmile/versionSuite.
2017-03-14 14:19:02 +08:00
Xiao Li 4dc3a8171c [SPARK-19924][SQL] Handle InvocationTargetException for all Hive Shim
### What changes were proposed in this pull request?
Since we are using shim for most Hive metastore APIs, the exceptions thrown by the underlying method of Method.invoke() are wrapped by `InvocationTargetException`. Instead of doing it one by one, we should handle all of them in the `withClient`. If any of them is missing, the error message could looks unfriendly. For example, below is an example for dropping tables.

```
Expected exception org.apache.spark.sql.AnalysisException to be thrown, but java.lang.reflect.InvocationTargetException was thrown.
ScalaTestFailureLocation: org.apache.spark.sql.catalyst.catalog.ExternalCatalogSuite$$anonfun$14 at (ExternalCatalogSuite.scala:193)
org.scalatest.exceptions.TestFailedException: Expected exception org.apache.spark.sql.AnalysisException to be thrown, but java.lang.reflect.InvocationTargetException was thrown.
	at org.scalatest.Assertions$class.newAssertionFailedException(Assertions.scala:496)
	at org.scalatest.FunSuite.newAssertionFailedException(FunSuite.scala:1555)
	at org.scalatest.Assertions$class.intercept(Assertions.scala:1004)
	at org.scalatest.FunSuite.intercept(FunSuite.scala:1555)
	at org.apache.spark.sql.catalyst.catalog.ExternalCatalogSuite$$anonfun$14.apply$mcV$sp(ExternalCatalogSuite.scala:193)
	at org.apache.spark.sql.catalyst.catalog.ExternalCatalogSuite$$anonfun$14.apply(ExternalCatalogSuite.scala:183)
	at org.apache.spark.sql.catalyst.catalog.ExternalCatalogSuite$$anonfun$14.apply(ExternalCatalogSuite.scala:183)
	at org.scalatest.Transformer$$anonfun$apply$1.apply$mcV$sp(Transformer.scala:22)
	at org.scalatest.OutcomeOf$class.outcomeOf(OutcomeOf.scala:85)
	at org.scalatest.OutcomeOf$.outcomeOf(OutcomeOf.scala:104)
	at org.scalatest.Transformer.apply(Transformer.scala:22)
	at org.scalatest.Transformer.apply(Transformer.scala:20)
	at org.scalatest.FunSuiteLike$$anon$1.apply(FunSuiteLike.scala:166)
	at org.apache.spark.SparkFunSuite.withFixture(SparkFunSuite.scala:68)
	at org.scalatest.FunSuiteLike$class.invokeWithFixture$1(FunSuiteLike.scala:163)
	at org.scalatest.FunSuiteLike$$anonfun$runTest$1.apply(FunSuiteLike.scala:175)
	at org.scalatest.FunSuiteLike$$anonfun$runTest$1.apply(FunSuiteLike.scala:175)
	at org.scalatest.SuperEngine.runTestImpl(Engine.scala:306)
	at org.scalatest.FunSuiteLike$class.runTest(FunSuiteLike.scala:175)
	at org.apache.spark.sql.catalyst.catalog.ExternalCatalogSuite.org$scalatest$BeforeAndAfterEach$$super$runTest(ExternalCatalogSuite.scala:40)
	at org.scalatest.BeforeAndAfterEach$class.runTest(BeforeAndAfterEach.scala:255)
	at org.apache.spark.sql.catalyst.catalog.ExternalCatalogSuite.runTest(ExternalCatalogSuite.scala:40)
	at org.scalatest.FunSuiteLike$$anonfun$runTests$1.apply(FunSuiteLike.scala:208)
	at org.scalatest.FunSuiteLike$$anonfun$runTests$1.apply(FunSuiteLike.scala:208)
	at org.scalatest.SuperEngine$$anonfun$traverseSubNodes$1$1.apply(Engine.scala:413)
	at org.scalatest.SuperEngine$$anonfun$traverseSubNodes$1$1.apply(Engine.scala:401)
	at scala.collection.immutable.List.foreach(List.scala:381)
	at org.scalatest.SuperEngine.traverseSubNodes$1(Engine.scala:401)
	at org.scalatest.SuperEngine.org$scalatest$SuperEngine$$runTestsInBranch(Engine.scala:396)
	at org.scalatest.SuperEngine.runTestsImpl(Engine.scala:483)
	at org.scalatest.FunSuiteLike$class.runTests(FunSuiteLike.scala:208)
	at org.scalatest.FunSuite.runTests(FunSuite.scala:1555)
	at org.scalatest.Suite$class.run(Suite.scala:1424)
	at org.scalatest.FunSuite.org$scalatest$FunSuiteLike$$super$run(FunSuite.scala:1555)
	at org.scalatest.FunSuiteLike$$anonfun$run$1.apply(FunSuiteLike.scala:212)
	at org.scalatest.FunSuiteLike$$anonfun$run$1.apply(FunSuiteLike.scala:212)
	at org.scalatest.SuperEngine.runImpl(Engine.scala:545)
	at org.scalatest.FunSuiteLike$class.run(FunSuiteLike.scala:212)
	at org.apache.spark.SparkFunSuite.org$scalatest$BeforeAndAfterAll$$super$run(SparkFunSuite.scala:31)
	at org.scalatest.BeforeAndAfterAll$class.liftedTree1$1(BeforeAndAfterAll.scala:257)
	at org.scalatest.BeforeAndAfterAll$class.run(BeforeAndAfterAll.scala:256)
	at org.apache.spark.SparkFunSuite.run(SparkFunSuite.scala:31)
	at org.scalatest.tools.SuiteRunner.run(SuiteRunner.scala:55)
	at org.scalatest.tools.Runner$$anonfun$doRunRunRunDaDoRunRun$3.apply(Runner.scala:2563)
	at org.scalatest.tools.Runner$$anonfun$doRunRunRunDaDoRunRun$3.apply(Runner.scala:2557)
	at scala.collection.immutable.List.foreach(List.scala:381)
	at org.scalatest.tools.Runner$.doRunRunRunDaDoRunRun(Runner.scala:2557)
	at org.scalatest.tools.Runner$$anonfun$runOptionallyWithPassFailReporter$2.apply(Runner.scala:1044)
	at org.scalatest.tools.Runner$$anonfun$runOptionallyWithPassFailReporter$2.apply(Runner.scala:1043)
	at org.scalatest.tools.Runner$.withClassLoaderAndDispatchReporter(Runner.scala:2722)
	at org.scalatest.tools.Runner$.runOptionallyWithPassFailReporter(Runner.scala:1043)
	at org.scalatest.tools.Runner$.run(Runner.scala:883)
	at org.scalatest.tools.Runner.run(Runner.scala)
	at org.jetbrains.plugins.scala.testingSupport.scalaTest.ScalaTestRunner.runScalaTest2(ScalaTestRunner.java:138)
	at org.jetbrains.plugins.scala.testingSupport.scalaTest.ScalaTestRunner.main(ScalaTestRunner.java:28)
	at sun.reflect.NativeMethodAccessorImpl.invoke0(Native Method)
	at sun.reflect.NativeMethodAccessorImpl.invoke(NativeMethodAccessorImpl.java:62)
	at sun.reflect.DelegatingMethodAccessorImpl.invoke(DelegatingMethodAccessorImpl.java:43)
	at java.lang.reflect.Method.invoke(Method.java:498)
	at com.intellij.rt.execution.application.AppMain.main(AppMain.java:147)
Caused by: java.lang.reflect.InvocationTargetException
	at sun.reflect.NativeMethodAccessorImpl.invoke0(Native Method)
	at sun.reflect.NativeMethodAccessorImpl.invoke(NativeMethodAccessorImpl.java:62)
	at sun.reflect.DelegatingMethodAccessorImpl.invoke(DelegatingMethodAccessorImpl.java:43)
	at java.lang.reflect.Method.invoke(Method.java:498)
	at org.apache.spark.sql.hive.client.Shim_v0_14.dropTable(HiveShim.scala:736)
	at org.apache.spark.sql.hive.client.HiveClientImpl$$anonfun$dropTable$1.apply$mcV$sp(HiveClientImpl.scala:451)
	at org.apache.spark.sql.hive.client.HiveClientImpl$$anonfun$dropTable$1.apply(HiveClientImpl.scala:451)
	at org.apache.spark.sql.hive.client.HiveClientImpl$$anonfun$dropTable$1.apply(HiveClientImpl.scala:451)
	at org.apache.spark.sql.hive.client.HiveClientImpl$$anonfun$withHiveState$1.apply(HiveClientImpl.scala:287)
	at org.apache.spark.sql.hive.client.HiveClientImpl.liftedTree1$1(HiveClientImpl.scala:228)
	at org.apache.spark.sql.hive.client.HiveClientImpl.retryLocked(HiveClientImpl.scala:227)
	at org.apache.spark.sql.hive.client.HiveClientImpl.withHiveState(HiveClientImpl.scala:270)
	at org.apache.spark.sql.hive.client.HiveClientImpl.dropTable(HiveClientImpl.scala:450)
	at org.apache.spark.sql.hive.HiveExternalCatalog$$anonfun$dropTable$1.apply$mcV$sp(HiveExternalCatalog.scala:456)
	at org.apache.spark.sql.hive.HiveExternalCatalog$$anonfun$dropTable$1.apply(HiveExternalCatalog.scala:454)
	at org.apache.spark.sql.hive.HiveExternalCatalog$$anonfun$dropTable$1.apply(HiveExternalCatalog.scala:454)
	at org.apache.spark.sql.hive.HiveExternalCatalog.withClient(HiveExternalCatalog.scala:94)
	at org.apache.spark.sql.hive.HiveExternalCatalog.dropTable(HiveExternalCatalog.scala:454)
	at org.apache.spark.sql.catalyst.catalog.ExternalCatalogSuite$$anonfun$14$$anonfun$apply$mcV$sp$8.apply$mcV$sp(ExternalCatalogSuite.scala:194)
	at org.apache.spark.sql.catalyst.catalog.ExternalCatalogSuite$$anonfun$14$$anonfun$apply$mcV$sp$8.apply(ExternalCatalogSuite.scala:194)
	at org.apache.spark.sql.catalyst.catalog.ExternalCatalogSuite$$anonfun$14$$anonfun$apply$mcV$sp$8.apply(ExternalCatalogSuite.scala:194)
	at org.scalatest.Assertions$class.intercept(Assertions.scala:997)
	... 57 more
Caused by: org.apache.hadoop.hive.ql.metadata.HiveException: NoSuchObjectException(message:db2.unknown_table table not found)
	at org.apache.hadoop.hive.ql.metadata.Hive.dropTable(Hive.java:1038)
	... 79 more
Caused by: NoSuchObjectException(message:db2.unknown_table table not found)
	at org.apache.hadoop.hive.metastore.HiveMetaStore$HMSHandler.get_table_core(HiveMetaStore.java:1808)
	at org.apache.hadoop.hive.metastore.HiveMetaStore$HMSHandler.get_table(HiveMetaStore.java:1778)
	at sun.reflect.NativeMethodAccessorImpl.invoke0(Native Method)
	at sun.reflect.NativeMethodAccessorImpl.invoke(NativeMethodAccessorImpl.java:62)
	at sun.reflect.DelegatingMethodAccessorImpl.invoke(DelegatingMethodAccessorImpl.java:43)
	at java.lang.reflect.Method.invoke(Method.java:498)
	at org.apache.hadoop.hive.metastore.RetryingHMSHandler.invoke(RetryingHMSHandler.java:107)
	at com.sun.proxy.$Proxy10.get_table(Unknown Source)
	at org.apache.hadoop.hive.metastore.HiveMetaStoreClient.getTable(HiveMetaStoreClient.java:1208)
	at org.apache.hadoop.hive.ql.metadata.SessionHiveMetaStoreClient.getTable(SessionHiveMetaStoreClient.java:131)
	at org.apache.hadoop.hive.metastore.HiveMetaStoreClient.dropTable(HiveMetaStoreClient.java:952)
	at org.apache.hadoop.hive.metastore.HiveMetaStoreClient.dropTable(HiveMetaStoreClient.java:904)
	at sun.reflect.NativeMethodAccessorImpl.invoke0(Native Method)
	at sun.reflect.NativeMethodAccessorImpl.invoke(NativeMethodAccessorImpl.java:62)
	at sun.reflect.DelegatingMethodAccessorImpl.invoke(DelegatingMethodAccessorImpl.java:43)
	at java.lang.reflect.Method.invoke(Method.java:498)
	at org.apache.hadoop.hive.metastore.RetryingMetaStoreClient.invoke(RetryingMetaStoreClient.java:156)
	at com.sun.proxy.$Proxy11.dropTable(Unknown Source)
	at org.apache.hadoop.hive.ql.metadata.Hive.dropTable(Hive.java:1035)
	... 79 more
```

After unwrapping the exception, the message is like
```
org.apache.hadoop.hive.ql.metadata.HiveException: NoSuchObjectException(message:db2.unknown_table table not found);
org.apache.spark.sql.AnalysisException: org.apache.hadoop.hive.ql.metadata.HiveException: NoSuchObjectException(message:db2.unknown_table table not found);
	at org.apache.spark.sql.hive.HiveExternalCatalog.withClient(HiveExternalCatalog.scala:100)
	at org.apache.spark.sql.hive.HiveExternalCatalog.dropTable(HiveExternalCatalog.scala:460)
	at org.apache.spark.sql.catalyst.catalog.ExternalCatalogSuite$$anonfun$14.apply$mcV$sp(ExternalCatalogSuite.scala:193)
	at org.apache.spark.sql.catalyst.catalog.ExternalCatalogSuite$$anonfun$14.apply(ExternalCatalogSuite.scala:183)
	at org.apache.spark.sql.catalyst.catalog.ExternalCatalogSuite$$anonfun$14.apply(ExternalCatalogSuite.scala:183)
	at org.scalatest.Transformer$$anonfun$apply$1.apply$mcV$sp(Transformer.scala:22)
...
```

### How was this patch tested?
Covered by the existing test case in `test("drop table when database/table does not exist")` in `ExternalCatalogSuite`.

Author: Xiao Li <gatorsmile@gmail.com>

Closes #17265 from gatorsmile/InvocationTargetException.
2017-03-14 12:06:01 +08:00
Wenchen Fan 05887fc3d8 [SPARK-19916][SQL] simplify bad file handling
## What changes were proposed in this pull request?

We should only have one centre place to try catch the exception for corrupted files.

## How was this patch tested?

existing test

Author: Wenchen Fan <wenchen@databricks.com>

Closes #17253 from cloud-fan/bad-file.
2017-03-12 23:16:45 -07:00
Tejas Patil 9456688547 [SPARK-17495][SQL] Support date, timestamp and interval types in Hive hash
## What changes were proposed in this pull request?

- Timestamp hashing is done as per [TimestampWritable.hashCode()](ff67cdda1c/serde/src/java/org/apache/hadoop/hive/serde2/io/TimestampWritable.java (L406)) in Hive
- Interval hashing is done as per [HiveIntervalDayTime.hashCode()](ff67cdda1c/storage-api/src/java/org/apache/hadoop/hive/common/type/HiveIntervalDayTime.java (L178)). Note that there are inherent differences in how Hive and Spark store intervals under the hood which limits the ability to be in completely sync with hive's hashing function. I have explained this in the method doc.
- Date type was already supported. This PR adds test for that.

## How was this patch tested?

Added unit tests

Author: Tejas Patil <tejasp@fb.com>

Closes #17062 from tejasapatil/SPARK-17495_time_related_types.
2017-03-12 20:08:44 -07:00
uncleGen e29a74d5b1 [DOCS][SS] fix structured streaming python example
## What changes were proposed in this pull request?

- SS python example: `TypeError: 'xxx' object is not callable`
- some other doc issue.

## How was this patch tested?

Jenkins.

Author: uncleGen <hustyugm@gmail.com>

Closes #17257 from uncleGen/docs-ss-python.
2017-03-12 08:29:37 +00:00
windpiger f6fdf92d0d [SPARK-19723][SQL] create datasource table with an non-existent location should work
## What changes were proposed in this pull request?

This JIRA is a follow up work after [SPARK-19583](https://issues.apache.org/jira/browse/SPARK-19583)

As we discussed in that [PR](https://github.com/apache/spark/pull/16938)

The following DDL for datasource table with an non-existent location should work:
```
CREATE TABLE ... (PARTITIONED BY ...) LOCATION path
```
Currently it will throw exception that path not exists for datasource table for datasource table

## How was this patch tested?
unit test added

Author: windpiger <songjun@outlook.com>

Closes #17055 from windpiger/CTDataSourcePathNotExists.
2017-03-10 20:59:32 -08:00
Wenchen Fan fb9beda546 [SPARK-19893][SQL] should not run DataFrame set oprations with map type
## What changes were proposed in this pull request?

In spark SQL, map type can't be used in equality test/comparison, and `Intersect`/`Except`/`Distinct` do need equality test for all columns, we should not allow map type in `Intersect`/`Except`/`Distinct`.

## How was this patch tested?

new regression test

Author: Wenchen Fan <wenchen@databricks.com>

Closes #17236 from cloud-fan/map.
2017-03-10 16:14:22 -08:00
Cheng Lian ffee4f1cef [SPARK-19905][SQL] Bring back Dataset.inputFiles for Hive SerDe tables
## What changes were proposed in this pull request?

`Dataset.inputFiles` works by matching `FileRelation`s in the query plan. In Spark 2.1, Hive SerDe tables are represented by `MetastoreRelation`, which inherits from `FileRelation`. However, in Spark 2.2, Hive SerDe tables are now represented by `CatalogRelation`, which doesn't inherit from `FileRelation` anymore, due to the unification of Hive SerDe tables and data source tables. This change breaks `Dataset.inputFiles` for Hive SerDe tables.

This PR tries to fix this issue by explicitly matching `CatalogRelation`s that are Hive SerDe tables in `Dataset.inputFiles`. Note that we can't make `CatalogRelation` inherit from `FileRelation` since not all `CatalogRelation`s are file based (e.g., JDBC data source tables).

## How was this patch tested?

New test case added in `HiveDDLSuite`.

Author: Cheng Lian <lian@databricks.com>

Closes #17247 from liancheng/spark-19905-hive-table-input-files.
2017-03-10 15:19:32 -08:00
Budde bc30351404 [SPARK-19611][SQL] Preserve metastore field order when merging inferred schema
## What changes were proposed in this pull request?

The ```HiveMetastoreCatalog.mergeWithMetastoreSchema()``` method added in #16944 may
not preserve the same field order as the metastore schema in some cases, which can cause
queries to fail. This change ensures that the metastore field order is preserved.

## How was this patch tested?

A test for ensuring that metastore order is preserved was added to ```HiveSchemaInferenceSuite.```
The particular failure usecase from #16944 was tested manually as well.

Author: Budde <budde@amazon.com>

Closes #17249 from budde/PreserveMetastoreFieldOrder.
2017-03-10 15:18:37 -08:00
Carson Wang dd9049e049 [SPARK-19620][SQL] Fix incorrect exchange coordinator id in the physical plan
## What changes were proposed in this pull request?
When adaptive execution is enabled, an exchange coordinator is used in the Exchange operators. For Join, the same exchange coordinator is used for its two Exchanges. But the physical plan shows two different coordinator Ids which is confusing.

This PR is to fix the incorrect exchange coordinator id in the physical plan. The coordinator object instead of the `Option[ExchangeCoordinator]` should be used to generate the identity hash code of the same coordinator.

## How was this patch tested?
Before the patch, the physical plan shows two different exchange coordinator id for Join.
```
== Physical Plan ==
*Project [key1#3L, value2#12L]
+- *SortMergeJoin [key1#3L], [key2#11L], Inner
   :- *Sort [key1#3L ASC NULLS FIRST], false, 0
   :  +- Exchange(coordinator id: 1804587700) hashpartitioning(key1#3L, 10), coordinator[target post-shuffle partition size: 67108864]
   :     +- *Project [(id#0L % 500) AS key1#3L]
   :        +- *Filter isnotnull((id#0L % 500))
   :           +- *Range (0, 1000, step=1, splits=Some(10))
   +- *Sort [key2#11L ASC NULLS FIRST], false, 0
      +- Exchange(coordinator id: 793927319) hashpartitioning(key2#11L, 10), coordinator[target post-shuffle partition size: 67108864]
         +- *Project [(id#8L % 500) AS key2#11L, id#8L AS value2#12L]
            +- *Filter isnotnull((id#8L % 500))
               +- *Range (0, 1000, step=1, splits=Some(10))
```
After the patch, two exchange coordinator id are the same.

Author: Carson Wang <carson.wang@intel.com>

Closes #16952 from carsonwang/FixCoordinatorId.
2017-03-10 11:13:26 -08:00
Kazuaki Ishizaki fcb68e0f5d [SPARK-19786][SQL] Facilitate loop optimizations in a JIT compiler regarding range()
## What changes were proposed in this pull request?

This PR improves performance of operations with `range()` by changing Java code generated by Catalyst. This PR is inspired by the [blog article](https://databricks.com/blog/2017/02/16/processing-trillion-rows-per-second-single-machine-can-nested-loop-joins-fast.html).

This PR changes generated code in the following two points.
1. Replace a while-loop with long instance variables a for-loop with int local varibles
2. Suppress generation of `shouldStop()` method if this method is unnecessary (e.g. `append()` is not generated).

These points facilitates compiler optimizations in a JIT compiler by feeding the simplified Java code into the JIT compiler. The performance is improved by 7.6x.

Benchmark program:
```java
val N = 1 << 29
val iters = 2
val benchmark = new Benchmark("range.count", N * iters)
benchmark.addCase(s"with this PR") { i =>
  var n = 0
  var len = 0
  while (n < iters) {
    len += sparkSession.range(N).selectExpr("count(id)").collect.length
    n += 1
  }
}
benchmark.run
```

Performance result without this PR
```
OpenJDK 64-Bit Server VM 1.8.0_111-8u111-b14-2ubuntu0.16.04.2-b14 on Linux 4.4.0-47-generic
Intel(R) Xeon(R) CPU E5-2667 v3  3.20GHz
range.count:                             Best/Avg Time(ms)    Rate(M/s)   Per Row(ns)   Relative
------------------------------------------------------------------------------------------------
w/o this PR                                   1349 / 1356        796.2           1.3       1.0X
```

Performance result with this PR
```
OpenJDK 64-Bit Server VM 1.8.0_111-8u111-b14-2ubuntu0.16.04.2-b14 on Linux 4.4.0-47-generic
Intel(R) Xeon(R) CPU E5-2667 v3  3.20GHz
range.count:                             Best/Avg Time(ms)    Rate(M/s)   Per Row(ns)   Relative
------------------------------------------------------------------------------------------------
with this PR                                   177 /  271       6065.3           0.2       1.0X
```

Here is a comparison between generated code w/o and with this PR. Only the method ```agg_doAggregateWithoutKey``` is changed.

Generated code without this PR
```java

/* 005 */ final class GeneratedIterator extends org.apache.spark.sql.execution.BufferedRowIterator {
/* 006 */   private Object[] references;
/* 007 */   private scala.collection.Iterator[] inputs;
/* 008 */   private boolean agg_initAgg;
/* 009 */   private boolean agg_bufIsNull;
/* 010 */   private long agg_bufValue;
/* 011 */   private org.apache.spark.sql.execution.metric.SQLMetric range_numOutputRows;
/* 012 */   private org.apache.spark.sql.execution.metric.SQLMetric range_numGeneratedRows;
/* 013 */   private boolean range_initRange;
/* 014 */   private long range_number;
/* 015 */   private TaskContext range_taskContext;
/* 016 */   private InputMetrics range_inputMetrics;
/* 017 */   private long range_batchEnd;
/* 018 */   private long range_numElementsTodo;
/* 019 */   private scala.collection.Iterator range_input;
/* 020 */   private UnsafeRow range_result;
/* 021 */   private org.apache.spark.sql.catalyst.expressions.codegen.BufferHolder range_holder;
/* 022 */   private org.apache.spark.sql.catalyst.expressions.codegen.UnsafeRowWriter range_rowWriter;
/* 023 */   private org.apache.spark.sql.execution.metric.SQLMetric agg_numOutputRows;
/* 024 */   private org.apache.spark.sql.execution.metric.SQLMetric agg_aggTime;
/* 025 */   private UnsafeRow agg_result;
/* 026 */   private org.apache.spark.sql.catalyst.expressions.codegen.BufferHolder agg_holder;
/* 027 */   private org.apache.spark.sql.catalyst.expressions.codegen.UnsafeRowWriter agg_rowWriter;
/* 028 */
/* 029 */   public GeneratedIterator(Object[] references) {
/* 030 */     this.references = references;
/* 031 */   }
/* 032 */
/* 033 */   public void init(int index, scala.collection.Iterator[] inputs) {
/* 034 */     partitionIndex = index;
/* 035 */     this.inputs = inputs;
/* 036 */     agg_initAgg = false;
/* 037 */
/* 038 */     this.range_numOutputRows = (org.apache.spark.sql.execution.metric.SQLMetric) references[0];
/* 039 */     this.range_numGeneratedRows = (org.apache.spark.sql.execution.metric.SQLMetric) references[1];
/* 040 */     range_initRange = false;
/* 041 */     range_number = 0L;
/* 042 */     range_taskContext = TaskContext.get();
/* 043 */     range_inputMetrics = range_taskContext.taskMetrics().inputMetrics();
/* 044 */     range_batchEnd = 0;
/* 045 */     range_numElementsTodo = 0L;
/* 046 */     range_input = inputs[0];
/* 047 */     range_result = new UnsafeRow(1);
/* 048 */     this.range_holder = new org.apache.spark.sql.catalyst.expressions.codegen.BufferHolder(range_result, 0);
/* 049 */     this.range_rowWriter = new org.apache.spark.sql.catalyst.expressions.codegen.UnsafeRowWriter(range_holder, 1);
/* 050 */     this.agg_numOutputRows = (org.apache.spark.sql.execution.metric.SQLMetric) references[2];
/* 051 */     this.agg_aggTime = (org.apache.spark.sql.execution.metric.SQLMetric) references[3];
/* 052 */     agg_result = new UnsafeRow(1);
/* 053 */     this.agg_holder = new org.apache.spark.sql.catalyst.expressions.codegen.BufferHolder(agg_result, 0);
/* 054 */     this.agg_rowWriter = new org.apache.spark.sql.catalyst.expressions.codegen.UnsafeRowWriter(agg_holder, 1);
/* 055 */
/* 056 */   }
/* 057 */
/* 058 */   private void agg_doAggregateWithoutKey() throws java.io.IOException {
/* 059 */     // initialize aggregation buffer
/* 060 */     agg_bufIsNull = false;
/* 061 */     agg_bufValue = 0L;
/* 062 */
/* 063 */     // initialize Range
/* 064 */     if (!range_initRange) {
/* 065 */       range_initRange = true;
/* 066 */       initRange(partitionIndex);
/* 067 */     }
/* 068 */
/* 069 */     while (true) {
/* 070 */       while (range_number != range_batchEnd) {
/* 071 */         long range_value = range_number;
/* 072 */         range_number += 1L;
/* 073 */
/* 074 */         // do aggregate
/* 075 */         // common sub-expressions
/* 076 */
/* 077 */         // evaluate aggregate function
/* 078 */         boolean agg_isNull1 = false;
/* 079 */
/* 080 */         long agg_value1 = -1L;
/* 081 */         agg_value1 = agg_bufValue + 1L;
/* 082 */         // update aggregation buffer
/* 083 */         agg_bufIsNull = false;
/* 084 */         agg_bufValue = agg_value1;
/* 085 */
/* 086 */         if (shouldStop()) return;
/* 087 */       }
/* 088 */
/* 089 */       if (range_taskContext.isInterrupted()) {
/* 090 */         throw new TaskKilledException();
/* 091 */       }
/* 092 */
/* 093 */       long range_nextBatchTodo;
/* 094 */       if (range_numElementsTodo > 1000L) {
/* 095 */         range_nextBatchTodo = 1000L;
/* 096 */         range_numElementsTodo -= 1000L;
/* 097 */       } else {
/* 098 */         range_nextBatchTodo = range_numElementsTodo;
/* 099 */         range_numElementsTodo = 0;
/* 100 */         if (range_nextBatchTodo == 0) break;
/* 101 */       }
/* 102 */       range_numOutputRows.add(range_nextBatchTodo);
/* 103 */       range_inputMetrics.incRecordsRead(range_nextBatchTodo);
/* 104 */
/* 105 */       range_batchEnd += range_nextBatchTodo * 1L;
/* 106 */     }
/* 107 */
/* 108 */   }
/* 109 */
/* 110 */   private void initRange(int idx) {
/* 111 */     java.math.BigInteger index = java.math.BigInteger.valueOf(idx);
/* 112 */     java.math.BigInteger numSlice = java.math.BigInteger.valueOf(2L);
/* 113 */     java.math.BigInteger numElement = java.math.BigInteger.valueOf(10000L);
/* 114 */     java.math.BigInteger step = java.math.BigInteger.valueOf(1L);
/* 115 */     java.math.BigInteger start = java.math.BigInteger.valueOf(0L);
/* 117 */
/* 118 */     java.math.BigInteger st = index.multiply(numElement).divide(numSlice).multiply(step).add(start);
/* 119 */     if (st.compareTo(java.math.BigInteger.valueOf(Long.MAX_VALUE)) > 0) {
/* 120 */       range_number = Long.MAX_VALUE;
/* 121 */     } else if (st.compareTo(java.math.BigInteger.valueOf(Long.MIN_VALUE)) < 0) {
/* 122 */       range_number = Long.MIN_VALUE;
/* 123 */     } else {
/* 124 */       range_number = st.longValue();
/* 125 */     }
/* 126 */     range_batchEnd = range_number;
/* 127 */
/* 128 */     java.math.BigInteger end = index.add(java.math.BigInteger.ONE).multiply(numElement).divide(numSlice)
/* 129 */     .multiply(step).add(start);
/* 130 */     if (end.compareTo(java.math.BigInteger.valueOf(Long.MAX_VALUE)) > 0) {
/* 131 */       partitionEnd = Long.MAX_VALUE;
/* 132 */     } else if (end.compareTo(java.math.BigInteger.valueOf(Long.MIN_VALUE)) < 0) {
/* 133 */       partitionEnd = Long.MIN_VALUE;
/* 134 */     } else {
/* 135 */       partitionEnd = end.longValue();
/* 136 */     }
/* 137 */
/* 138 */     java.math.BigInteger startToEnd = java.math.BigInteger.valueOf(partitionEnd).subtract(
/* 139 */       java.math.BigInteger.valueOf(range_number));
/* 140 */     range_numElementsTodo  = startToEnd.divide(step).longValue();
/* 141 */     if (range_numElementsTodo < 0) {
/* 142 */       range_numElementsTodo = 0;
/* 143 */     } else if (startToEnd.remainder(step).compareTo(java.math.BigInteger.valueOf(0L)) != 0) {
/* 144 */       range_numElementsTodo++;
/* 145 */     }
/* 146 */   }
/* 147 */
/* 148 */   protected void processNext() throws java.io.IOException {
/* 149 */     while (!agg_initAgg) {
/* 150 */       agg_initAgg = true;
/* 151 */       long agg_beforeAgg = System.nanoTime();
/* 152 */       agg_doAggregateWithoutKey();
/* 153 */       agg_aggTime.add((System.nanoTime() - agg_beforeAgg) / 1000000);
/* 154 */
/* 155 */       // output the result
/* 156 */
/* 157 */       agg_numOutputRows.add(1);
/* 158 */       agg_rowWriter.zeroOutNullBytes();
/* 159 */
/* 160 */       if (agg_bufIsNull) {
/* 161 */         agg_rowWriter.setNullAt(0);
/* 162 */       } else {
/* 163 */         agg_rowWriter.write(0, agg_bufValue);
/* 164 */       }
/* 165 */       append(agg_result);
/* 166 */     }
/* 167 */   }
/* 168 */ }
```

Generated code with this PR
```java
/* 005 */ final class GeneratedIterator extends org.apache.spark.sql.execution.BufferedRowIterator {
/* 006 */   private Object[] references;
/* 007 */   private scala.collection.Iterator[] inputs;
/* 008 */   private boolean agg_initAgg;
/* 009 */   private boolean agg_bufIsNull;
/* 010 */   private long agg_bufValue;
/* 011 */   private org.apache.spark.sql.execution.metric.SQLMetric range_numOutputRows;
/* 012 */   private org.apache.spark.sql.execution.metric.SQLMetric range_numGeneratedRows;
/* 013 */   private boolean range_initRange;
/* 014 */   private long range_number;
/* 015 */   private TaskContext range_taskContext;
/* 016 */   private InputMetrics range_inputMetrics;
/* 017 */   private long range_batchEnd;
/* 018 */   private long range_numElementsTodo;
/* 019 */   private scala.collection.Iterator range_input;
/* 020 */   private UnsafeRow range_result;
/* 021 */   private org.apache.spark.sql.catalyst.expressions.codegen.BufferHolder range_holder;
/* 022 */   private org.apache.spark.sql.catalyst.expressions.codegen.UnsafeRowWriter range_rowWriter;
/* 023 */   private org.apache.spark.sql.execution.metric.SQLMetric agg_numOutputRows;
/* 024 */   private org.apache.spark.sql.execution.metric.SQLMetric agg_aggTime;
/* 025 */   private UnsafeRow agg_result;
/* 026 */   private org.apache.spark.sql.catalyst.expressions.codegen.BufferHolder agg_holder;
/* 027 */   private org.apache.spark.sql.catalyst.expressions.codegen.UnsafeRowWriter agg_rowWriter;
/* 028 */
/* 029 */   public GeneratedIterator(Object[] references) {
/* 030 */     this.references = references;
/* 031 */   }
/* 032 */
/* 033 */   public void init(int index, scala.collection.Iterator[] inputs) {
/* 034 */     partitionIndex = index;
/* 035 */     this.inputs = inputs;
/* 036 */     agg_initAgg = false;
/* 037 */
/* 038 */     this.range_numOutputRows = (org.apache.spark.sql.execution.metric.SQLMetric) references[0];
/* 039 */     this.range_numGeneratedRows = (org.apache.spark.sql.execution.metric.SQLMetric) references[1];
/* 040 */     range_initRange = false;
/* 041 */     range_number = 0L;
/* 042 */     range_taskContext = TaskContext.get();
/* 043 */     range_inputMetrics = range_taskContext.taskMetrics().inputMetrics();
/* 044 */     range_batchEnd = 0;
/* 045 */     range_numElementsTodo = 0L;
/* 046 */     range_input = inputs[0];
/* 047 */     range_result = new UnsafeRow(1);
/* 048 */     this.range_holder = new org.apache.spark.sql.catalyst.expressions.codegen.BufferHolder(range_result, 0);
/* 049 */     this.range_rowWriter = new org.apache.spark.sql.catalyst.expressions.codegen.UnsafeRowWriter(range_holder, 1);
/* 050 */     this.agg_numOutputRows = (org.apache.spark.sql.execution.metric.SQLMetric) references[2];
/* 051 */     this.agg_aggTime = (org.apache.spark.sql.execution.metric.SQLMetric) references[3];
/* 052 */     agg_result = new UnsafeRow(1);
/* 053 */     this.agg_holder = new org.apache.spark.sql.catalyst.expressions.codegen.BufferHolder(agg_result, 0);
/* 054 */     this.agg_rowWriter = new org.apache.spark.sql.catalyst.expressions.codegen.UnsafeRowWriter(agg_holder, 1);
/* 055 */
/* 056 */   }
/* 057 */
/* 058 */   private void agg_doAggregateWithoutKey() throws java.io.IOException {
/* 059 */     // initialize aggregation buffer
/* 060 */     agg_bufIsNull = false;
/* 061 */     agg_bufValue = 0L;
/* 062 */
/* 063 */     // initialize Range
/* 064 */     if (!range_initRange) {
/* 065 */       range_initRange = true;
/* 066 */       initRange(partitionIndex);
/* 067 */     }
/* 068 */
/* 069 */     while (true) {
/* 070 */       long range_range = range_batchEnd - range_number;
/* 071 */       if (range_range != 0L) {
/* 072 */         int range_localEnd = (int)(range_range / 1L);
/* 073 */         for (int range_localIdx = 0; range_localIdx < range_localEnd; range_localIdx++) {
/* 074 */           long range_value = ((long)range_localIdx * 1L) + range_number;
/* 075 */
/* 076 */           // do aggregate
/* 077 */           // common sub-expressions
/* 078 */
/* 079 */           // evaluate aggregate function
/* 080 */           boolean agg_isNull1 = false;
/* 081 */
/* 082 */           long agg_value1 = -1L;
/* 083 */           agg_value1 = agg_bufValue + 1L;
/* 084 */           // update aggregation buffer
/* 085 */           agg_bufIsNull = false;
/* 086 */           agg_bufValue = agg_value1;
/* 087 */
/* 088 */           // shouldStop check is eliminated
/* 089 */         }
/* 090 */         range_number = range_batchEnd;
/* 091 */       }
/* 092 */
/* 093 */       if (range_taskContext.isInterrupted()) {
/* 094 */         throw new TaskKilledException();
/* 095 */       }
/* 096 */
/* 097 */       long range_nextBatchTodo;
/* 098 */       if (range_numElementsTodo > 1000L) {
/* 099 */         range_nextBatchTodo = 1000L;
/* 100 */         range_numElementsTodo -= 1000L;
/* 101 */       } else {
/* 102 */         range_nextBatchTodo = range_numElementsTodo;
/* 103 */         range_numElementsTodo = 0;
/* 104 */         if (range_nextBatchTodo == 0) break;
/* 105 */       }
/* 106 */       range_numOutputRows.add(range_nextBatchTodo);
/* 107 */       range_inputMetrics.incRecordsRead(range_nextBatchTodo);
/* 108 */
/* 109 */       range_batchEnd += range_nextBatchTodo * 1L;
/* 110 */     }
/* 111 */
/* 112 */   }
/* 113 */
/* 114 */   private void initRange(int idx) {
/* 115 */     java.math.BigInteger index = java.math.BigInteger.valueOf(idx);
/* 116 */     java.math.BigInteger numSlice = java.math.BigInteger.valueOf(2L);
/* 117 */     java.math.BigInteger numElement = java.math.BigInteger.valueOf(10000L);
/* 118 */     java.math.BigInteger step = java.math.BigInteger.valueOf(1L);
/* 119 */     java.math.BigInteger start = java.math.BigInteger.valueOf(0L);
/* 120 */     long partitionEnd;
/* 121 */
/* 122 */     java.math.BigInteger st = index.multiply(numElement).divide(numSlice).multiply(step).add(start);
/* 123 */     if (st.compareTo(java.math.BigInteger.valueOf(Long.MAX_VALUE)) > 0) {
/* 124 */       range_number = Long.MAX_VALUE;
/* 125 */     } else if (st.compareTo(java.math.BigInteger.valueOf(Long.MIN_VALUE)) < 0) {
/* 126 */       range_number = Long.MIN_VALUE;
/* 127 */     } else {
/* 128 */       range_number = st.longValue();
/* 129 */     }
/* 130 */     range_batchEnd = range_number;
/* 131 */
/* 132 */     java.math.BigInteger end = index.add(java.math.BigInteger.ONE).multiply(numElement).divide(numSlice)
/* 133 */     .multiply(step).add(start);
/* 134 */     if (end.compareTo(java.math.BigInteger.valueOf(Long.MAX_VALUE)) > 0) {
/* 135 */       partitionEnd = Long.MAX_VALUE;
/* 136 */     } else if (end.compareTo(java.math.BigInteger.valueOf(Long.MIN_VALUE)) < 0) {
/* 137 */       partitionEnd = Long.MIN_VALUE;
/* 138 */     } else {
/* 139 */       partitionEnd = end.longValue();
/* 140 */     }
/* 141 */
/* 142 */     java.math.BigInteger startToEnd = java.math.BigInteger.valueOf(partitionEnd).subtract(
/* 143 */       java.math.BigInteger.valueOf(range_number));
/* 144 */     range_numElementsTodo  = startToEnd.divide(step).longValue();
/* 145 */     if (range_numElementsTodo < 0) {
/* 146 */       range_numElementsTodo = 0;
/* 147 */     } else if (startToEnd.remainder(step).compareTo(java.math.BigInteger.valueOf(0L)) != 0) {
/* 148 */       range_numElementsTodo++;
/* 149 */     }
/* 150 */   }
/* 151 */
/* 152 */   protected void processNext() throws java.io.IOException {
/* 153 */     while (!agg_initAgg) {
/* 154 */       agg_initAgg = true;
/* 155 */       long agg_beforeAgg = System.nanoTime();
/* 156 */       agg_doAggregateWithoutKey();
/* 157 */       agg_aggTime.add((System.nanoTime() - agg_beforeAgg) / 1000000);
/* 158 */
/* 159 */       // output the result
/* 160 */
/* 161 */       agg_numOutputRows.add(1);
/* 162 */       agg_rowWriter.zeroOutNullBytes();
/* 163 */
/* 164 */       if (agg_bufIsNull) {
/* 165 */         agg_rowWriter.setNullAt(0);
/* 166 */       } else {
/* 167 */         agg_rowWriter.write(0, agg_bufValue);
/* 168 */       }
/* 169 */       append(agg_result);
/* 170 */     }
/* 171 */   }
/* 172 */ }
```

A part of suppressing `shouldStop()` was originally developed by inouehrs

## How was this patch tested?

Add new tests into `DataFrameRangeSuite`

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

Closes #17122 from kiszk/SPARK-19786.
2017-03-10 18:04:37 +01:00
Tyson Condie 501b711199 [SPARK-19891][SS] Await Batch Lock notified on stream execution exit
## What changes were proposed in this pull request?

We need to notify the await batch lock when the stream exits early e.g., when an exception has been thrown.

## How was this patch tested?

Current tests that throw exceptions at runtime will finish faster as a result of this update.

zsxwing

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

Author: Tyson Condie <tcondie@gmail.com>

Closes #17231 from tcondie/kafka-writer.
2017-03-09 23:02:13 -08:00
Kazuaki Ishizaki 5949e6c447 [SPARK-19008][SQL] Improve performance of Dataset.map by eliminating boxing/unboxing
## What changes were proposed in this pull request?

This PR improve performance of Dataset.map() for primitive types by removing boxing/unbox operations. This is based on [the discussion](https://github.com/apache/spark/pull/16391#discussion_r93788919) with cloud-fan.

Current Catalyst generates a method call to a `apply()` method of an anonymous function written in Scala. The types of an argument and return value are `java.lang.Object`. As a result, each method call for a primitive value involves a pair of unboxing and boxing for calling this `apply()` method and a pair of boxing and unboxing for returning from this `apply()` method.

This PR directly calls a specialized version of a `apply()` method without boxing and unboxing. For example, if types of an arguments ant return value is `int`, this PR generates a method call to `apply$mcII$sp`. This PR supports any combination of `Int`, `Long`, `Float`, and `Double`.

The following is a benchmark result using [this program](https://github.com/apache/spark/pull/16391/files) with 4.7x. Here is a Dataset part of this program.

Without this PR
```
OpenJDK 64-Bit Server VM 1.8.0_111-8u111-b14-2ubuntu0.16.04.2-b14 on Linux 4.4.0-47-generic
Intel(R) Xeon(R) CPU E5-2667 v3  3.20GHz
back-to-back map:                        Best/Avg Time(ms)    Rate(M/s)   Per Row(ns)   Relative
------------------------------------------------------------------------------------------------
RDD                                           1923 / 1952         52.0          19.2       1.0X
DataFrame                                      526 /  548        190.2           5.3       3.7X
Dataset                                       3094 / 3154         32.3          30.9       0.6X
```

With this PR
```
OpenJDK 64-Bit Server VM 1.8.0_111-8u111-b14-2ubuntu0.16.04.2-b14 on Linux 4.4.0-47-generic
Intel(R) Xeon(R) CPU E5-2667 v3  3.20GHz
back-to-back map:                        Best/Avg Time(ms)    Rate(M/s)   Per Row(ns)   Relative
------------------------------------------------------------------------------------------------
RDD                                           1883 / 1892         53.1          18.8       1.0X
DataFrame                                      502 /  642        199.1           5.0       3.7X
Dataset                                        657 /  784        152.2           6.6       2.9X
```

```java
  def backToBackMap(spark: SparkSession, numRows: Long, numChains: Int): Benchmark = {
    import spark.implicits._
    val rdd = spark.sparkContext.range(0, numRows)
    val ds = spark.range(0, numRows)
    val func = (l: Long) => l + 1
    val benchmark = new Benchmark("back-to-back map", numRows)
...
    benchmark.addCase("Dataset") { iter =>
      var res = ds.as[Long]
      var i = 0
      while (i < numChains) {
        res = res.map(func)
        i += 1
      }
      res.queryExecution.toRdd.foreach(_ => Unit)
    }
    benchmark
  }
```

A motivating example
```java
Seq(1, 2, 3).toDS.map(i => i * 7).show
```

Generated code without this PR
```java
/* 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 inputadapter_input;
/* 009 */   private UnsafeRow deserializetoobject_result;
/* 010 */   private org.apache.spark.sql.catalyst.expressions.codegen.BufferHolder deserializetoobject_holder;
/* 011 */   private org.apache.spark.sql.catalyst.expressions.codegen.UnsafeRowWriter deserializetoobject_rowWriter;
/* 012 */   private int mapelements_argValue;
/* 013 */   private UnsafeRow mapelements_result;
/* 014 */   private org.apache.spark.sql.catalyst.expressions.codegen.BufferHolder mapelements_holder;
/* 015 */   private org.apache.spark.sql.catalyst.expressions.codegen.UnsafeRowWriter mapelements_rowWriter;
/* 016 */   private UnsafeRow serializefromobject_result;
/* 017 */   private org.apache.spark.sql.catalyst.expressions.codegen.BufferHolder serializefromobject_holder;
/* 018 */   private org.apache.spark.sql.catalyst.expressions.codegen.UnsafeRowWriter serializefromobject_rowWriter;
/* 019 */
/* 020 */   public GeneratedIterator(Object[] references) {
/* 021 */     this.references = references;
/* 022 */   }
/* 023 */
/* 024 */   public void init(int index, scala.collection.Iterator[] inputs) {
/* 025 */     partitionIndex = index;
/* 026 */     this.inputs = inputs;
/* 027 */     inputadapter_input = inputs[0];
/* 028 */     deserializetoobject_result = new UnsafeRow(1);
/* 029 */     this.deserializetoobject_holder = new org.apache.spark.sql.catalyst.expressions.codegen.BufferHolder(deserializetoobject_result, 0);
/* 030 */     this.deserializetoobject_rowWriter = new org.apache.spark.sql.catalyst.expressions.codegen.UnsafeRowWriter(deserializetoobject_holder, 1);
/* 031 */
/* 032 */     mapelements_result = new UnsafeRow(1);
/* 033 */     this.mapelements_holder = new org.apache.spark.sql.catalyst.expressions.codegen.BufferHolder(mapelements_result, 0);
/* 034 */     this.mapelements_rowWriter = new org.apache.spark.sql.catalyst.expressions.codegen.UnsafeRowWriter(mapelements_holder, 1);
/* 035 */     serializefromobject_result = new UnsafeRow(1);
/* 036 */     this.serializefromobject_holder = new org.apache.spark.sql.catalyst.expressions.codegen.BufferHolder(serializefromobject_result, 0);
/* 037 */     this.serializefromobject_rowWriter = new org.apache.spark.sql.catalyst.expressions.codegen.UnsafeRowWriter(serializefromobject_holder, 1);
/* 038 */
/* 039 */   }
/* 040 */
/* 041 */   protected void processNext() throws java.io.IOException {
/* 042 */     while (inputadapter_input.hasNext() && !stopEarly()) {
/* 043 */       InternalRow inputadapter_row = (InternalRow) inputadapter_input.next();
/* 044 */       int inputadapter_value = inputadapter_row.getInt(0);
/* 045 */
/* 046 */       boolean mapelements_isNull = true;
/* 047 */       int mapelements_value = -1;
/* 048 */       if (!false) {
/* 049 */         mapelements_argValue = inputadapter_value;
/* 050 */
/* 051 */         mapelements_isNull = false;
/* 052 */         if (!mapelements_isNull) {
/* 053 */           Object mapelements_funcResult = null;
/* 054 */           mapelements_funcResult = ((scala.Function1) references[0]).apply(mapelements_argValue);
/* 055 */           if (mapelements_funcResult == null) {
/* 056 */             mapelements_isNull = true;
/* 057 */           } else {
/* 058 */             mapelements_value = (Integer) mapelements_funcResult;
/* 059 */           }
/* 060 */
/* 061 */         }
/* 062 */
/* 063 */       }
/* 064 */
/* 065 */       serializefromobject_rowWriter.zeroOutNullBytes();
/* 066 */
/* 067 */       if (mapelements_isNull) {
/* 068 */         serializefromobject_rowWriter.setNullAt(0);
/* 069 */       } else {
/* 070 */         serializefromobject_rowWriter.write(0, mapelements_value);
/* 071 */       }
/* 072 */       append(serializefromobject_result);
/* 073 */       if (shouldStop()) return;
/* 074 */     }
/* 075 */   }
/* 076 */ }
```

Generated code with this PR (lines 48-56 are changed)
```java
/* 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 inputadapter_input;
/* 009 */   private UnsafeRow deserializetoobject_result;
/* 010 */   private org.apache.spark.sql.catalyst.expressions.codegen.BufferHolder deserializetoobject_holder;
/* 011 */   private org.apache.spark.sql.catalyst.expressions.codegen.UnsafeRowWriter deserializetoobject_rowWriter;
/* 012 */   private int mapelements_argValue;
/* 013 */   private UnsafeRow mapelements_result;
/* 014 */   private org.apache.spark.sql.catalyst.expressions.codegen.BufferHolder mapelements_holder;
/* 015 */   private org.apache.spark.sql.catalyst.expressions.codegen.UnsafeRowWriter mapelements_rowWriter;
/* 016 */   private UnsafeRow serializefromobject_result;
/* 017 */   private org.apache.spark.sql.catalyst.expressions.codegen.BufferHolder serializefromobject_holder;
/* 018 */   private org.apache.spark.sql.catalyst.expressions.codegen.UnsafeRowWriter serializefromobject_rowWriter;
/* 019 */
/* 020 */   public GeneratedIterator(Object[] references) {
/* 021 */     this.references = references;
/* 022 */   }
/* 023 */
/* 024 */   public void init(int index, scala.collection.Iterator[] inputs) {
/* 025 */     partitionIndex = index;
/* 026 */     this.inputs = inputs;
/* 027 */     inputadapter_input = inputs[0];
/* 028 */     deserializetoobject_result = new UnsafeRow(1);
/* 029 */     this.deserializetoobject_holder = new org.apache.spark.sql.catalyst.expressions.codegen.BufferHolder(deserializetoobject_result, 0);
/* 030 */     this.deserializetoobject_rowWriter = new org.apache.spark.sql.catalyst.expressions.codegen.UnsafeRowWriter(deserializetoobject_holder, 1);
/* 031 */
/* 032 */     mapelements_result = new UnsafeRow(1);
/* 033 */     this.mapelements_holder = new org.apache.spark.sql.catalyst.expressions.codegen.BufferHolder(mapelements_result, 0);
/* 034 */     this.mapelements_rowWriter = new org.apache.spark.sql.catalyst.expressions.codegen.UnsafeRowWriter(mapelements_holder, 1);
/* 035 */     serializefromobject_result = new UnsafeRow(1);
/* 036 */     this.serializefromobject_holder = new org.apache.spark.sql.catalyst.expressions.codegen.BufferHolder(serializefromobject_result, 0);
/* 037 */     this.serializefromobject_rowWriter = new org.apache.spark.sql.catalyst.expressions.codegen.UnsafeRowWriter(serializefromobject_holder, 1);
/* 038 */
/* 039 */   }
/* 040 */
/* 041 */   protected void processNext() throws java.io.IOException {
/* 042 */     while (inputadapter_input.hasNext() && !stopEarly()) {
/* 043 */       InternalRow inputadapter_row = (InternalRow) inputadapter_input.next();
/* 044 */       int inputadapter_value = inputadapter_row.getInt(0);
/* 045 */
/* 046 */       boolean mapelements_isNull = true;
/* 047 */       int mapelements_value = -1;
/* 048 */       if (!false) {
/* 049 */         mapelements_argValue = inputadapter_value;
/* 050 */
/* 051 */         mapelements_isNull = false;
/* 052 */         if (!mapelements_isNull) {
/* 053 */           mapelements_value = ((scala.Function1) references[0]).apply$mcII$sp(mapelements_argValue);
/* 054 */         }
/* 055 */
/* 056 */       }
/* 057 */
/* 058 */       serializefromobject_rowWriter.zeroOutNullBytes();
/* 059 */
/* 060 */       if (mapelements_isNull) {
/* 061 */         serializefromobject_rowWriter.setNullAt(0);
/* 062 */       } else {
/* 063 */         serializefromobject_rowWriter.write(0, mapelements_value);
/* 064 */       }
/* 065 */       append(serializefromobject_result);
/* 066 */       if (shouldStop()) return;
/* 067 */     }
/* 068 */   }
/* 069 */ }
```

Java bytecode for methods for `i => i * 7`
```java
$ javap -c Test\$\$anonfun\$5\$\$anonfun\$apply\$mcV\$sp\$1.class
Compiled from "Test.scala"
public final class org.apache.spark.sql.Test$$anonfun$5$$anonfun$apply$mcV$sp$1 extends scala.runtime.AbstractFunction1$mcII$sp implements scala.Serializable {
  public static final long serialVersionUID;

  public final int apply(int);
    Code:
       0: aload_0
       1: iload_1
       2: invokevirtual #18                 // Method apply$mcII$sp:(I)I
       5: ireturn

  public int apply$mcII$sp(int);
    Code:
       0: iload_1
       1: bipush        7
       3: imul
       4: ireturn

  public final java.lang.Object apply(java.lang.Object);
    Code:
       0: aload_0
       1: aload_1
       2: invokestatic  #29                 // Method scala/runtime/BoxesRunTime.unboxToInt:(Ljava/lang/Object;)I
       5: invokevirtual #31                 // Method apply:(I)I
       8: invokestatic  #35                 // Method scala/runtime/BoxesRunTime.boxToInteger:(I)Ljava/lang/Integer;
      11: areturn

  public org.apache.spark.sql.Test$$anonfun$5$$anonfun$apply$mcV$sp$1(org.apache.spark.sql.Test$$anonfun$5);
    Code:
       0: aload_0
       1: invokespecial #42                 // Method scala/runtime/AbstractFunction1$mcII$sp."<init>":()V
       4: return
}
```
## How was this patch tested?

Added new test suites to `DatasetPrimitiveSuite`.

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

Closes #17172 from kiszk/SPARK-19008.
2017-03-09 22:58:52 -08:00
Budde f79371ad86 [SPARK-19611][SQL] Introduce configurable table schema inference
## Summary of changes

Add a new configuration option that allows Spark SQL to infer a case-sensitive schema from a Hive Metastore table's data files when a case-sensitive schema can't be read from the table properties.

- Add spark.sql.hive.caseSensitiveInferenceMode param to SQLConf
- Add schemaPreservesCase field to CatalogTable (set to false when schema can't
  successfully be read from Hive table props)
- Perform schema inference in HiveMetastoreCatalog if schemaPreservesCase is
  false, depending on spark.sql.hive.caseSensitiveInferenceMode
- Add alterTableSchema() method to the ExternalCatalog interface
- Add HiveSchemaInferenceSuite tests
- Refactor and move ParquetFileForamt.meregeMetastoreParquetSchema() as
  HiveMetastoreCatalog.mergeWithMetastoreSchema
- Move schema merging tests from ParquetSchemaSuite to HiveSchemaInferenceSuite

[JIRA for this change](https://issues.apache.org/jira/browse/SPARK-19611)

## How was this patch tested?

The tests in ```HiveSchemaInferenceSuite``` should verify that schema inference is working as expected. ```ExternalCatalogSuite``` has also been extended to cover the new ```alterTableSchema()``` API.

Author: Budde <budde@amazon.com>

Closes #16944 from budde/SPARK-19611.
2017-03-09 12:55:33 -08:00
Jeff Zhang cabe1df860 [SPARK-12334][SQL][PYSPARK] Support read from multiple input paths for orc file in DataFrameReader.orc
Beside the issue in spark api, also fix 2 minor issues in pyspark
- support read from multiple input paths for orc
- support read from multiple input paths for text

Author: Jeff Zhang <zjffdu@apache.org>

Closes #10307 from zjffdu/SPARK-12334.
2017-03-09 11:44:34 -08:00
uncleGen 30b18e6936 [SPARK-19861][SS] watermark should not be a negative time.
## What changes were proposed in this pull request?

`watermark` should not be negative. This behavior is invalid, check it before real run.

## How was this patch tested?

add new unit test.

Author: uncleGen <hustyugm@gmail.com>
Author: dylon <hustyugm@gmail.com>

Closes #17202 from uncleGen/SPARK-19861.
2017-03-09 11:07:31 -08:00
Liwei Lin 40da4d181d [SPARK-19715][STRUCTURED STREAMING] Option to Strip Paths in FileSource
## What changes were proposed in this pull request?

Today, we compare the whole path when deciding if a file is new in the FileSource for structured streaming. However, this would cause false negatives in the case where the path has changed in a cosmetic way (i.e. changing `s3n` to `s3a`).

This patch adds an option `fileNameOnly` that causes the new file check to be based only on the filename (but still store the whole path in the log).

## Usage

```scala
spark
  .readStream
  .option("fileNameOnly", true)
  .text("s3n://bucket/dir1/dir2")
  .writeStream
  ...
```
## How was this patch tested?

Added a test case

Author: Liwei Lin <lwlin7@gmail.com>

Closes #17120 from lw-lin/filename-only.
2017-03-09 11:02:44 -08:00
Jason White 206030bd12 [SPARK-19561][SQL] add int case handling for TimestampType
## What changes were proposed in this pull request?

Add handling of input of type `Int` for dataType `TimestampType` to `EvaluatePython.scala`. Py4J serializes ints smaller than MIN_INT or larger than MAX_INT to Long, which are handled correctly already, but values between MIN_INT and MAX_INT are serialized to Int.

These range limits correspond to roughly half an hour on either side of the epoch. As a result, PySpark doesn't allow TimestampType values to be created in this range.

Alternatives attempted: patching the `TimestampType.toInternal` function to cast return values to `long`, so Py4J would always serialize them to Scala Long. Python3 does not have a `long` type, so this approach failed on Python3.

## How was this patch tested?

Added a new PySpark-side test that fails without the change.

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

Resubmission of https://github.com/apache/spark/pull/16896. The original PR didn't go through Jenkins and broke the build. davies dongjoon-hyun

cloud-fan Could you kick off a Jenkins run for me? It passed everything for me locally, but it's possible something has changed in the last few weeks.

Author: Jason White <jason.white@shopify.com>

Closes #17200 from JasonMWhite/SPARK-19561.
2017-03-09 10:34:54 -08:00
windpiger 274973d2a3 [SPARK-19763][SQL] qualified external datasource table location stored in catalog
## What changes were proposed in this pull request?

If we create a external datasource table with a non-qualified location , we should qualified it to store in catalog.

```
CREATE TABLE t(a string)
USING parquet
LOCATION '/path/xx'

CREATE TABLE t1(a string, b string)
USING parquet
PARTITIONED BY(b)
LOCATION '/path/xx'
```

when we get the table from catalog, the location should be qualified, e.g.'file:/path/xxx'
## How was this patch tested?
unit test added

Author: windpiger <songjun@outlook.com>

Closes #17095 from windpiger/tablepathQualified.
2017-03-09 01:18:17 -08:00
uncleGen eeb1d6db87 [SPARK-19859][SS][FOLLOW-UP] The new watermark should override the old one.
## What changes were proposed in this pull request?

A follow up to SPARK-19859:

- extract the calculation of `delayMs` and reuse it.
- update EventTimeWatermarkExec
- use the correct `delayMs` in EventTimeWatermark

## How was this patch tested?

Jenkins.

Author: uncleGen <hustyugm@gmail.com>

Closes #17221 from uncleGen/SPARK-19859.
2017-03-08 23:23:10 -08:00
Xiao Li 09829be621 [SPARK-19235][SQL][TESTS] Enable Test Cases in DDLSuite with Hive Metastore
### What changes were proposed in this pull request?
So far, the test cases in DDLSuites only verify the behaviors of InMemoryCatalog. That means, they do not cover the scenarios using HiveExternalCatalog. Thus, we need to improve the existing test suite to run these cases using Hive metastore.

When porting these test cases, a bug of `SET LOCATION` is found. `path` is not set when the location is changed.

After this PR, a few changes are made, as summarized below,
- `DDLSuite` becomes an abstract class. Both `InMemoryCatalogedDDLSuite` and `HiveCatalogedDDLSuite` extend it. `InMemoryCatalogedDDLSuite` is using `InMemoryCatalog`. `HiveCatalogedDDLSuite` is using `HiveExternalCatalog`.
- `InMemoryCatalogedDDLSuite` contains all the existing test cases in `DDLSuite`.
- `HiveCatalogedDDLSuite` contains a subset of `DDLSuite`. The following test cases are excluded:

1. The following test cases only make sense for `InMemoryCatalog`:
```
  test("desc table for parquet data source table using in-memory catalog")
  test("create a managed Hive source table") {
  test("create an external Hive source table")
  test("Create Hive Table As Select")
```

2. The following test cases are unable to be ported because we are unable to alter table provider when using Hive metastore. In the future PRs we need to improve the test cases so that altering table provider is not needed:
```
  test("alter table: set location (datasource table)")
  test("alter table: set properties (datasource table)")
  test("alter table: unset properties (datasource table)")
  test("alter table: set serde (datasource table)")
  test("alter table: set serde partition (datasource table)")
  test("alter table: change column (datasource table)")
  test("alter table: add partition (datasource table)")
  test("alter table: drop partition (datasource table)")
  test("alter table: rename partition (datasource table)")
  test("drop table - data source table")
```

**TODO** : in the future PRs, we need to remove `HiveDDLSuite` and move the test cases to either `DDLSuite`,  `InMemoryCatalogedDDLSuite` or `HiveCatalogedDDLSuite`.

### How was this patch tested?
N/A

Author: Xiao Li <gatorsmile@gmail.com>
Author: gatorsmile <gatorsmile@gmail.com>

Closes #16592 from gatorsmile/refactorDDLSuite.
2017-03-08 23:12:10 -08:00
Dilip Biswal d809ceed97 [MINOR][SQL] The analyzer rules are fired twice for cases when AnalysisException is raised from analyzer.
## What changes were proposed in this pull request?
In general we have a checkAnalysis phase which validates the logical plan and throws AnalysisException on semantic errors. However we also can throw AnalysisException from a few analyzer rules like ResolveSubquery.

I found that we fire up the analyzer rules twice for the queries that throw AnalysisException from one of the analyzer rules. This is a very minor fix. We don't have to strictly fix it. I just got confused seeing the rule getting fired two times when i was not expecting it.

## How was this patch tested?

Tested manually.

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

Closes #17214 from dilipbiswal/analyis_twice.
2017-03-08 17:33:49 -08:00
Burak Yavuz a3648b5d4f [SPARK-19813] maxFilesPerTrigger combo latestFirst may miss old files in combination with maxFileAge in FileStreamSource
## What changes were proposed in this pull request?

**The Problem**
There is a file stream source option called maxFileAge which limits how old the files can be, relative the latest file that has been seen. This is used to limit the files that need to be remembered as "processed". Files older than the latest processed files are ignored. This values is by default 7 days.
This causes a problem when both
latestFirst = true
maxFilesPerTrigger > total files to be processed.
Here is what happens in all combinations
1) latestFirst = false - Since files are processed in order, there wont be any unprocessed file older than the latest processed file. All files will be processed.
2) latestFirst = true AND maxFilesPerTrigger is not set - The maxFileAge thresholding mechanism takes one batch initialize. If maxFilesPerTrigger is not, then all old files get processed in the first batch, and so no file is left behind.
3) latestFirst = true AND maxFilesPerTrigger is set to X - The first batch process the latest X files. That sets the threshold latest file - maxFileAge, so files older than this threshold will never be considered for processing.
The bug is with case 3.

**The Solution**

Ignore `maxFileAge` when both `maxFilesPerTrigger` and `latestFirst` are set.

## How was this patch tested?

Regression test in `FileStreamSourceSuite`

Author: Burak Yavuz <brkyvz@gmail.com>

Closes #17153 from brkyvz/maxFileAge.
2017-03-08 14:35:07 -08:00
hyukjinkwon 455129020c [SPARK-15463][SQL] Add an API to load DataFrame from Dataset[String] storing CSV
## What changes were proposed in this pull request?

This PR proposes to add an API that loads `DataFrame` from `Dataset[String]` storing csv.

It allows pre-processing before loading into CSV, which means allowing a lot of workarounds for many narrow cases, for example, as below:

- Case 1 - pre-processing

  ```scala
  val df = spark.read.text("...")
  // Pre-processing with this.
  spark.read.csv(df.as[String])
  ```

- Case 2 - use other input formats

  ```scala
  val rdd = spark.sparkContext.newAPIHadoopFile("/file.csv.lzo",
    classOf[com.hadoop.mapreduce.LzoTextInputFormat],
    classOf[org.apache.hadoop.io.LongWritable],
    classOf[org.apache.hadoop.io.Text])
  val stringRdd = rdd.map(pair => new String(pair._2.getBytes, 0, pair._2.getLength))

  spark.read.csv(stringRdd.toDS)
  ```

## How was this patch tested?

Added tests in `CSVSuite` and build with Scala 2.10.

```
./dev/change-scala-version.sh 2.10
./build/mvn -Pyarn -Phadoop-2.4 -Dscala-2.10 -DskipTests clean package
```

Author: hyukjinkwon <gurwls223@gmail.com>

Closes #16854 from HyukjinKwon/SPARK-15463.
2017-03-08 13:43:09 -08:00
Kunal Khamar 6570cfd7ab [SPARK-19540][SQL] Add ability to clone SparkSession wherein cloned session has an identical copy of the SessionState
Forking a newSession() from SparkSession currently makes a new SparkSession that does not retain SessionState (i.e. temporary tables, SQL config, registered functions etc.) This change adds a method cloneSession() which creates a new SparkSession with a copy of the parent's SessionState.

Subsequent changes to base session are not propagated to cloned session, clone is independent after creation.
If the base is changed after clone has been created, say user registers new UDF, then the new UDF will not be available inside the clone. Same goes for configs and temp tables.

Unit tests

Author: Kunal Khamar <kkhamar@outlook.com>
Author: Shixiong Zhu <shixiong@databricks.com>

Closes #16826 from kunalkhamar/fork-sparksession.
2017-03-08 13:20:45 -08:00
Shixiong Zhu 1bf9012380 [SPARK-19858][SS] Add output mode to flatMapGroupsWithState and disallow invalid cases
## What changes were proposed in this pull request?

Add a output mode parameter to `flatMapGroupsWithState` and just define `mapGroupsWithState` as `flatMapGroupsWithState(Update)`.

`UnsupportedOperationChecker` is modified to disallow unsupported cases.

- Batch mapGroupsWithState or flatMapGroupsWithState is always allowed.
- For streaming (map/flatMap)GroupsWithState, see the following table:

| Operators  | Supported Query Output Mode |
| ------------- | ------------- |
| flatMapGroupsWithState(Update) without aggregation  | Update |
| flatMapGroupsWithState(Update) with aggregation  | None |
| flatMapGroupsWithState(Append) without aggregation  | Append |
| flatMapGroupsWithState(Append) before aggregation  | Append, Update, Complete |
| flatMapGroupsWithState(Append) after aggregation  | None |
| Multiple flatMapGroupsWithState(Append)s  | Append |
| Multiple mapGroupsWithStates  | None |
| Mxing mapGroupsWithStates  and flatMapGroupsWithStates | None |
| Other cases of multiple flatMapGroupsWithState | None |

## How was this patch tested?

The added unit tests. Here are the tests related to (map/flatMap)GroupsWithState:
```
[info] - batch plan - flatMapGroupsWithState - flatMapGroupsWithState(Append) on batch relation: supported (1 millisecond)
[info] - batch plan - flatMapGroupsWithState - multiple flatMapGroupsWithState(Append)s on batch relation: supported (0 milliseconds)
[info] - batch plan - flatMapGroupsWithState - flatMapGroupsWithState(Update) on batch relation: supported (0 milliseconds)
[info] - batch plan - flatMapGroupsWithState - multiple flatMapGroupsWithState(Update)s on batch relation: supported (0 milliseconds)
[info] - streaming plan - flatMapGroupsWithState - flatMapGroupsWithState(Update) on streaming relation without aggregation in update mode: supported (2 milliseconds)
[info] - streaming plan - flatMapGroupsWithState - flatMapGroupsWithState(Update) on streaming relation without aggregation in append mode: not supported (7 milliseconds)
[info] - streaming plan - flatMapGroupsWithState - flatMapGroupsWithState(Update) on streaming relation without aggregation in complete mode: not supported (5 milliseconds)
[info] - streaming plan - flatMapGroupsWithState - flatMapGroupsWithState(Update) on streaming relation with aggregation in Append mode: not supported (11 milliseconds)
[info] - streaming plan - flatMapGroupsWithState - flatMapGroupsWithState(Update) on streaming relation with aggregation in Update mode: not supported (5 milliseconds)
[info] - streaming plan - flatMapGroupsWithState - flatMapGroupsWithState(Update) on streaming relation with aggregation in Complete mode: not supported (5 milliseconds)
[info] - streaming plan - flatMapGroupsWithState - flatMapGroupsWithState(Append) on streaming relation without aggregation in append mode: supported (1 millisecond)
[info] - streaming plan - flatMapGroupsWithState - flatMapGroupsWithState(Append) on streaming relation without aggregation in update mode: not supported (6 milliseconds)
[info] - streaming plan - flatMapGroupsWithState - flatMapGroupsWithState(Append) on streaming relation before aggregation in Append mode: supported (1 millisecond)
[info] - streaming plan - flatMapGroupsWithState - flatMapGroupsWithState(Append) on streaming relation before aggregation in Update mode: supported (0 milliseconds)
[info] - streaming plan - flatMapGroupsWithState - flatMapGroupsWithState(Append) on streaming relation before aggregation in Complete mode: supported (1 millisecond)
[info] - streaming plan - flatMapGroupsWithState - flatMapGroupsWithState(Append) on streaming relation after aggregation in Append mode: not supported (6 milliseconds)
[info] - streaming plan - flatMapGroupsWithState - flatMapGroupsWithState(Append) on streaming relation after aggregation in Update mode: not supported (4 milliseconds)
[info] - streaming plan - flatMapGroupsWithState - flatMapGroupsWithState(Update) on streaming relation in complete mode: not supported (2 milliseconds)
[info] - streaming plan - flatMapGroupsWithState - flatMapGroupsWithState(Append) on batch relation inside streaming relation in Append output mode: supported (1 millisecond)
[info] - streaming plan - flatMapGroupsWithState - flatMapGroupsWithState(Append) on batch relation inside streaming relation in Update output mode: supported (1 millisecond)
[info] - streaming plan - flatMapGroupsWithState - flatMapGroupsWithState(Update) on batch relation inside streaming relation in Append output mode: supported (0 milliseconds)
[info] - streaming plan - flatMapGroupsWithState - flatMapGroupsWithState(Update) on batch relation inside streaming relation in Update output mode: supported (0 milliseconds)
[info] - streaming plan - flatMapGroupsWithState - multiple flatMapGroupsWithStates on streaming relation and all are in append mode: supported (2 milliseconds)
[info] - streaming plan - flatMapGroupsWithState -  multiple flatMapGroupsWithStates on s streaming relation but some are not in append mode: not supported (7 milliseconds)
[info] - streaming plan - mapGroupsWithState - mapGroupsWithState on streaming relation without aggregation in append mode: not supported (3 milliseconds)
[info] - streaming plan - mapGroupsWithState - mapGroupsWithState on streaming relation without aggregation in complete mode: not supported (3 milliseconds)
[info] - streaming plan - mapGroupsWithState - mapGroupsWithState on streaming relation with aggregation in Append mode: not supported (6 milliseconds)
[info] - streaming plan - mapGroupsWithState - mapGroupsWithState on streaming relation with aggregation in Update mode: not supported (3 milliseconds)
[info] - streaming plan - mapGroupsWithState - mapGroupsWithState on streaming relation with aggregation in Complete mode: not supported (4 milliseconds)
[info] - streaming plan - mapGroupsWithState - multiple mapGroupsWithStates on streaming relation and all are in append mode: not supported (4 milliseconds)
[info] - streaming plan - mapGroupsWithState - mixing mapGroupsWithStates and flatMapGroupsWithStates on streaming relation: not supported (4 milliseconds)
```

Author: Shixiong Zhu <shixiong@databricks.com>

Closes #17197 from zsxwing/mapgroups-check.
2017-03-08 13:18:07 -08:00
Wojtek Szymanski e9e2c612d5 [SPARK-19727][SQL] Fix for round function that modifies original column
## What changes were proposed in this pull request?

Fix for SQL round function that modifies original column when underlying data frame is created from a local product.

    import org.apache.spark.sql.functions._

    case class NumericRow(value: BigDecimal)

    val df = spark.createDataFrame(Seq(NumericRow(BigDecimal("1.23456789"))))

    df.show()
    +--------------------+
    |               value|
    +--------------------+
    |1.234567890000000000|
    +--------------------+

    df.withColumn("value_rounded", round('value)).show()

    // before
    +--------------------+-------------+
    |               value|value_rounded|
    +--------------------+-------------+
    |1.000000000000000000|            1|
    +--------------------+-------------+

    // after
    +--------------------+-------------+
    |               value|value_rounded|
    +--------------------+-------------+
    |1.234567890000000000|            1|
    +--------------------+-------------+

## How was this patch tested?

New unit test added to existing suite `org.apache.spark.sql.MathFunctionsSuite`

Author: Wojtek Szymanski <wk.szymanski@gmail.com>

Closes #17075 from wojtek-szymanski/SPARK-19727.
2017-03-08 12:36:16 -08:00
windpiger f3387d9748 [SPARK-19864][SQL][TEST] provide a makeQualifiedPath functions to optimize some code
## What changes were proposed in this pull request?

Currently there are lots of places to make the path qualified, it is better to provide a function to do this, then the code will be more simple.

## How was this patch tested?
N/A

Author: windpiger <songjun@outlook.com>

Closes #17204 from windpiger/addQualifiledPathUtil.
2017-03-08 10:48:53 -08:00
Xiao Li 9a6ac7226f [SPARK-19601][SQL] Fix CollapseRepartition rule to preserve shuffle-enabled Repartition
### What changes were proposed in this pull request?

Observed by felixcheung  in https://github.com/apache/spark/pull/16739, when users use the shuffle-enabled `repartition` API, they expect the partition they got should be the exact number they provided, even if they call shuffle-disabled `coalesce` later.

Currently, `CollapseRepartition` rule does not consider whether shuffle is enabled or not. Thus, we got the following unexpected result.

```Scala
    val df = spark.range(0, 10000, 1, 5)
    val df2 = df.repartition(10)
    assert(df2.coalesce(13).rdd.getNumPartitions == 5)
    assert(df2.coalesce(7).rdd.getNumPartitions == 5)
    assert(df2.coalesce(3).rdd.getNumPartitions == 3)
```

This PR is to fix the issue. We preserve shuffle-enabled Repartition.

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

Author: Xiao Li <gatorsmile@gmail.com>

Closes #16933 from gatorsmile/CollapseRepartition.
2017-03-08 09:36:01 -08:00
jiangxingbo 5f7d835d38 [SPARK-19865][SQL] remove the view identifier in SubqueryAlias
## What changes were proposed in this pull request?

Since we have a `View` node now, we can remove the view identifier in `SubqueryAlias`, which was used to indicate a view node before.

## How was this patch tested?

Update the related test cases.

Author: jiangxingbo <jiangxb1987@gmail.com>

Closes #17210 from jiangxb1987/SubqueryAlias.
2017-03-08 16:18:17 +01:00
wangzhenhua e44274870d [SPARK-17080][SQL] join reorder
## What changes were proposed in this pull request?

Reorder the joins using a dynamic programming algorithm (Selinger paper):
First we put all items (basic joined nodes) into level 1, then we build all two-way joins at level 2 from plans at level 1 (single items), then build all 3-way joins from plans at previous levels (two-way joins and single items), then 4-way joins ... etc, until we build all n-way joins and pick the best plan among them.

When building m-way joins, we only keep the best plan (with the lowest cost) for the same set of m items. E.g., for 3-way joins, we keep only the best plan for items {A, B, C} among plans (A J B) J C, (A J C) J B and (B J C) J A. Thus, the plans maintained for each level when reordering four items A, B, C, D are as follows:
```
level 1: p({A}), p({B}), p({C}), p({D})
level 2: p({A, B}), p({A, C}), p({A, D}), p({B, C}), p({B, D}), p({C, D})
level 3: p({A, B, C}), p({A, B, D}), p({A, C, D}), p({B, C, D})
level 4: p({A, B, C, D})
```
where p({A, B, C, D}) is the final output plan.

For cost evaluation, since physical costs for operators are not available currently, we use cardinalities and sizes to compute costs.

## How was this patch tested?
add test cases

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

Closes #17138 from wzhfy/joinReorder.
2017-03-08 16:01:28 +01:00
Yuming Wang 3f9f9180c2 [SPARK-19693][SQL] Make the SET mapreduce.job.reduces automatically converted to spark.sql.shuffle.partitions
## What changes were proposed in this pull request?
Make the `SET mapreduce.job.reduces` automatically converted to `spark.sql.shuffle.partitions`, it's similar to `SET mapred.reduce.tasks`.

## How was this patch tested?

unit tests

Author: Yuming Wang <wgyumg@gmail.com>

Closes #17020 from wangyum/SPARK-19693.
2017-03-08 11:31:01 +00:00
Michael Armbrust 314e48a358 [SPARK-18055][SQL] Use correct mirror in ExpresionEncoder
Previously, we were using the mirror of passed in `TypeTag` when reflecting to build an encoder.  This fails when the outer class is built in (i.e. `Seq`'s default mirror is based on root classloader) but inner classes (i.e. `A` in `Seq[A]`) are defined in the REPL or a library.

This patch changes us to always reflect based on a mirror created using the context classloader.

Author: Michael Armbrust <michael@databricks.com>

Closes #17201 from marmbrus/replSeqEncoder.
2017-03-08 01:32:42 -08:00
Shixiong Zhu d8830c5039 [SPARK-19859][SS] The new watermark should override the old one
## What changes were proposed in this pull request?

The new watermark should override the old one. Otherwise, we just pick up the first column which has a watermark, it may be unexpected.

## How was this patch tested?

The new test.

Author: Shixiong Zhu <shixiong@databricks.com>

Closes #17199 from zsxwing/SPARK-19859.
2017-03-07 20:34:55 -08:00
Shixiong Zhu ca849ac4e8 [SPARK-19841][SS] watermarkPredicate should filter based on keys
## What changes were proposed in this pull request?

`StreamingDeduplicateExec.watermarkPredicate` should filter based on keys. Otherwise, it may generate a wrong answer if the watermark column in `keyExpression` has a different position in the row.

`StateStoreSaveExec` has the same codes but its parent can makes sure the watermark column positions in `keyExpression` and `row` are the same.

## How was this patch tested?

The added test.

Author: Shixiong Zhu <shixiong@databricks.com>

Closes #17183 from zsxwing/SPARK-19841.
2017-03-07 20:32:51 -08:00
jiangxingbo b9783a92f7 [SPARK-18389][SQL] Disallow cyclic view reference
## What changes were proposed in this pull request?

Disallow cyclic view references, a cyclic view reference may be created by the following queries:
```
CREATE VIEW testView AS SELECT id FROM tbl
CREATE VIEW testView2 AS SELECT id FROM testView
ALTER VIEW testView AS SELECT * FROM testView2
```
In the above example, a reference cycle (testView -> testView2 -> testView) exsits.

We disallow cyclic view references by checking that in ALTER VIEW command, when the `analyzedPlan` contains the same `View` node with the altered view, we should prevent the behavior and throw an AnalysisException.

## How was this patch tested?

Test by `SQLViewSuite.test("correctly handle a cyclic view reference")`.

Author: jiangxingbo <jiangxb1987@gmail.com>

Closes #17152 from jiangxb1987/cyclic-view.
2017-03-07 20:25:38 -08:00
Tejas Patil c96d14abae [SPARK-19843][SQL] UTF8String => (int / long) conversion expensive for invalid inputs
## What changes were proposed in this pull request?

Jira : https://issues.apache.org/jira/browse/SPARK-19843

Created wrapper classes (`IntWrapper`, `LongWrapper`) to wrap the result of parsing (which are primitive types). In case of problem in parsing, the method would return a boolean.

## How was this patch tested?

- Added new unit tests
- Ran a prod job which had conversion from string -> int and verified the outputs

## Performance

Tiny regression when all strings are valid integers

```
conversion to int:       Best/Avg Time(ms)    Rate(M/s)   Per Row(ns)   Relative
--------------------------------------------------------------------------------
trunk                         502 /  522         33.4          29.9       1.0X
SPARK-19843                   493 /  503         34.0          29.4       1.0X
```

Huge gain when all strings are invalid integers
```
conversion to int:      Best/Avg Time(ms)    Rate(M/s)   Per Row(ns)   Relative
-------------------------------------------------------------------------------
trunk                     33913 / 34219          0.5        2021.4       1.0X
SPARK-19843                  154 /  162        108.8           9.2     220.0X
```

Author: Tejas Patil <tejasp@fb.com>

Closes #17184 from tejasapatil/SPARK-19843_is_numeric_maybe.
2017-03-07 20:19:30 -08:00
Wenchen Fan c05baabf10 [SPARK-19765][SPARK-18549][SQL] UNCACHE TABLE should un-cache all cached plans that refer to this table
## What changes were proposed in this pull request?

When un-cache a table, we should not only remove the cache entry for this table, but also un-cache any other cached plans that refer to this table.

This PR also includes some refactors:

1. use `java.util.LinkedList` to store the cache entries, so that it's safer to remove elements while iterating
2. rename `invalidateCache` to `recacheByPlan`, which is more obvious about what it does.

## How was this patch tested?

new regression test

Author: Wenchen Fan <wenchen@databricks.com>

Closes #17097 from cloud-fan/cache.
2017-03-07 09:21:58 -08:00
Takeshi Yamamuro 030acdd1f0 [SPARK-19637][SQL] Add to_json in FunctionRegistry
## What changes were proposed in this pull request?
This pr added entries  in `FunctionRegistry` and supported `to_json` in SQL.

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

Author: Takeshi Yamamuro <yamamuro@apache.org>

Closes #16981 from maropu/SPARK-19637.
2017-03-07 09:00:14 -08:00
wangzhenhua 932196d9e3 [SPARK-17075][SQL][FOLLOWUP] fix filter estimation issues
## What changes were proposed in this pull request?

1. support boolean type in binary expression estimation.
2. deal with compound Not conditions.
3. avoid convert BigInt/BigDecimal directly to double unless it's within range (0, 1).
4. reorganize test code.

## How was this patch tested?

modify related test cases.

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

Closes #17148 from wzhfy/fixFilter.
2017-03-06 23:53:53 -08:00